You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by jolynch <gi...@git.apache.org> on 2018/10/14 22:22:05 UTC

[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

GitHub user jolynch opened a pull request:

    https://github.com/apache/cassandra/pull/283

    CASSANDRA-14459: DynamicEndpointSnitch should never prefer latent replicas

    This change incorporates the feedback from Ariel and Jason as part of https://issues.apache.org/jira/browse/CASSANDRA-14459.
    
    The following is introduced:
    
    1. Fully pluggable DynamicEndpointSnitch so that we can continue experimenting with new implementations
    2. Instead of resetting every 10 minutes, the DES uses active latency probes for replicas that it was asked to rank but has no recent data on. These are rate limited by default to a single probe per second. These latency probes, while not perfect, will correctly detect nodes that are latent due to network conditions, JVM instability (gc/safepoint pauses), and Read threadpool exhaustion.
    3. A new opt-in implementation of the DES which uses an exponential moving average instead of a Histogram. Both statistical measures try to develop a noise reduced sample with different tradeoffs, but the main one in favor of DES is that it reacts to extreme outliars faster (e.g. if a node is actively timing out and dropping messages) and generates about 100x less garbage than the histogram approach.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/jolynch/cassandra CASSANDRA-14459

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/cassandra/pull/283.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #283
    
----
commit 850952dac3a7988252cb09072f5dbd226bda3430
Author: Joseph Lynch <jo...@...>
Date:   2018-10-01T13:30:58Z

    Avoid dropping all data in DynamicSnitch reset
    
    Instead of throwing away all measurements every ten minutes, now we keep
    the minimum value and allow "bad" measurements such as EchoMessage
    responses to be kept just when the sample size is small (right after a
    reset).
    
    This prevents nodes from talking accross datacenters and makes it so that when
    nodes start up they get a latency landscape during the first round of gossip

commit 700f8c2e81221b4b18b6e012cfd33525d4861a91
Author: Joseph Lynch <jo...@...>
Date:   2018-07-20T07:08:28Z

    Send pings on a scheduled basis rather than from Gossiper

commit c6760e63b3682b00d11b0a8019cc9b7fda8b199f
Author: Joseph Lynch <jo...@...>
Date:   2018-10-11T19:26:44Z

    Makes the DES plugable and refactors it to be cleaner
    
    In particular separates the DES components that manage updating the
    scores from all the rest, allowing us to experiemnt safely with e.g.
    EMAs instead of Histograms and other new approaches.

commit bb34644ef46d14332ca4f5fa561bf8411eab148f
Author: Joseph Lynch <jo...@...>
Date:   2018-10-12T23:13:29Z

    Add pluggable EMA based Snitch
    
    Also refactors the test suite to test both implementations as well as
    more closely testing the latency probe algorithm.

commit 753e4b86bde34194a5997c84046a1ceb67455337
Author: Joseph Lynch <jo...@...>
Date:   2018-10-14T20:39:16Z

    Make the DES more testable and benchmark the EMA vs Histogram approach
    
    Using -prof gc I was able to show that the EMA approach is about 4-5x
    faster and between 70-400x less garbage generated. Essentially the EMA
    reacts a little bit slower than the histgoram, but is more tolerant of
    noise and generlly is way more performant.

----


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r239884211
  
    --- Diff: src/java/org/apache/cassandra/net/MessagingService.java ---
    @@ -729,20 +730,26 @@ void markTimeout(InetAddressAndPort addr)
         /**
          * Track latency information for the dynamic snitch
          *
    -     * @param cb      the callback associated with this message -- this lets us know if it's a message type we're interested in
    -     * @param address the host that replied to the message
    -     * @param latency
    +     * @param cb            the callback associated with this message -- this lets us know if it's a message type we're interested in
    +     * @param address       the host that replied to the message
    +     * @param latencyMicros the number of microseconds to record for this host
          */
    -    public void maybeAddLatency(IAsyncCallback cb, InetAddressAndPort address, long latency)
    +    public void maybeAddLatency(IAsyncCallback cb, InetAddressAndPort address, long latencyMicros)
         {
    -        if (cb.isLatencyForSnitch())
    -            addLatency(address, latency);
    +        if (cb.latencyMeasurementType() != LatencyMeasurementType.IGNORE)
    +            addLatency(address, latencyMicros, cb.latencyMeasurementType());
         }
     
    -    public void addLatency(InetAddressAndPort address, long latency)
    +    // Used on the local read path
    +    public void addLatency(InetAddressAndPort address, long latencyMicros)
    --- End diff --
    
    It's a bit implicit that this is exclusively for adding read latency?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r229456772
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -23,128 +23,300 @@
     import java.net.UnknownHostException;
     import java.util.*;
     import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ExecutionException;
     import java.util.concurrent.ScheduledFuture;
     import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
     import java.util.stream.Collectors;
     
    -import com.codahale.metrics.ExponentiallyDecayingReservoir;
     import javax.management.MBeanServer;
     import javax.management.ObjectName;
     
    -import com.codahale.metrics.Snapshot;
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.util.concurrent.Futures;
    +import com.google.common.util.concurrent.MoreExecutors;
    +import com.google.common.util.concurrent.RateLimiter;
    +import com.google.common.util.concurrent.SettableFuture;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
     import org.apache.cassandra.concurrent.ScheduledExecutors;
     import org.apache.cassandra.config.DatabaseDescriptor;
    +import org.apache.cassandra.exceptions.RequestFailureReason;
     import org.apache.cassandra.gms.ApplicationState;
     import org.apache.cassandra.gms.EndpointState;
     import org.apache.cassandra.gms.Gossiper;
     import org.apache.cassandra.gms.VersionedValue;
    +import org.apache.cassandra.locator.dynamicsnitch.DynamicEndpointSnitchHistogram;
    +import org.apache.cassandra.net.IAsyncCallbackWithFailure;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +import org.apache.cassandra.net.MessageIn;
    +import org.apache.cassandra.net.MessageOut;
     import org.apache.cassandra.net.MessagingService;
    +import org.apache.cassandra.net.PingMessage;
     import org.apache.cassandra.service.StorageService;
     import org.apache.cassandra.utils.FBUtilities;
     
    +import static org.apache.cassandra.net.MessagingService.Verb.PING;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.LARGE_MESSAGE;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.SMALL_MESSAGE;
    +
    +
     /**
      * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector
    + * Note that the subclasses (e.g. {@link DynamicEndpointSnitchHistogram}) are responsible for actually measuring
    + * latency and providing an ISnitchMeasurement implementation back to this class.
      */
    -public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
    +public abstract class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
     {
    -    private static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    -
    -    private static final double ALPHA = 0.75; // set to 0.75 to make EDS more biased to towards the newer values
    -    private static final int WINDOW_SIZE = 100;
    -
    -    private volatile int dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    -    private volatile int dynamicResetInterval = DatabaseDescriptor.getDynamicResetInterval();
    -    private volatile double dynamicBadnessThreshold = DatabaseDescriptor.getDynamicBadnessThreshold();
    +    private static final Logger logger = LoggerFactory.getLogger(DynamicEndpointSnitch.class);
    +
    +    // Latency measurement and ranking. The samples contain latency measurements and the scores are used for ranking
    +    protected boolean registered = false;
    +    protected static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    protected volatile Map<InetAddressAndPort, Double> scores = new HashMap<>();
    +    protected final Map<InetAddressAndPort, AnnotatedMeasurement> samples = new ConcurrentHashMap<>();
    +
    +    // Latency probe functionality for actively probing endpoints that we haven't measured recently but are ranking
    +    public static final long MAX_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_max_probe_interval_ms", 60 * 10 * 1000L);
    +    public static final long MIN_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_min_probe_interval_ms", 60 * 1000L) ;
    +    // The probe rate is set later when configuration is read
    +    protected static final RateLimiter probeRateLimiter = RateLimiter.create(1);
    +    protected static final DebuggableScheduledThreadPoolExecutor latencyProbeExecutor = new DebuggableScheduledThreadPoolExecutor(1, "LatencyProbes", Thread.MIN_PRIORITY);
    +
    +    // User configuration of the snitch tunables
    +    protected volatile int dynamicUpdateInterval = -1;
    +    protected volatile int dynamicLatencyProbeInterval = -1;
    +    protected volatile double dynamicBadnessThreshold = 0;
     
         // the score for a merged set of endpoints must be this much worse than the score for separate endpoints to
         // warrant not merging two ranges into a single range
         private static final double RANGE_MERGING_PREFERENCE = 1.5;
     
         private String mbeanName;
    -    private boolean registered = false;
    -
    -    private volatile HashMap<InetAddressAndPort, Double> scores = new HashMap<>();
    -    private final ConcurrentHashMap<InetAddressAndPort, ExponentiallyDecayingReservoir> samples = new ConcurrentHashMap<>();
    +    private boolean mbeanRegistered = false;
     
         public final IEndpointSnitch subsnitch;
     
    -    private volatile ScheduledFuture<?> updateSchedular;
    -    private volatile ScheduledFuture<?> resetSchedular;
    -
    -    private final Runnable update;
    -    private final Runnable reset;
    +    private volatile ScheduledFuture<?> updateScoresScheduler;
    +    private volatile ScheduledFuture<?> updateSamplesScheduler;
     
         public DynamicEndpointSnitch(IEndpointSnitch snitch)
         {
             this(snitch, null);
         }
     
    -    public DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
    +    protected DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
         {
             mbeanName = "org.apache.cassandra.db:type=DynamicEndpointSnitch";
             if (instance != null)
                 mbeanName += ",instance=" + instance;
             subsnitch = snitch;
    -        update = new Runnable()
    +
    +        if (DatabaseDescriptor.isDaemonInitialized())
             {
    -            public void run()
    -            {
    -                updateScores();
    -            }
    -        };
    -        reset = new Runnable()
    +            applyConfigChanges(DatabaseDescriptor.getDynamicUpdateInterval(),
    +                               DatabaseDescriptor.getDynamicSampleUpdateInterval(),
    +                               DatabaseDescriptor.getDynamicBadnessThreshold());
    +            open();
    +        }
    +    }
    +
    +    /**
    +     * Allows subclasses to inject new ways of measuring latency back to this abstract base class.
    +     */
    +    protected interface ISnitchMeasurement
    +    {
    +        void sample(long value);
    +        double measure();
    +        Iterable<Double> measurements();
    +    }
    +
    +    /**
    +     * Adds some boookeeping that the DES uses over top of the various metrics techniques used by the
    +     * implementations. This is used to allow CASSANDRA-14459 latency probes as well as further experimentation
    +     * on new latency measurement techniques in CASSANDRA-14817
    +     *
    +     * {@link AnnotatedMeasurement#millisSinceLastRequest} is set to zero through
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}. It defaults to
    +     * the maximum interval so that we only start probing once it has been ranked at least once
    +     * {@link AnnotatedMeasurement#millisSinceLastMeasure} is set to zero from
    +     * {@link DynamicEndpointSnitch#receiveTiming(InetAddressAndPort, long, LatencyMeasurementType)}.
    +     *
    +     * {@link AnnotatedMeasurement#millisSinceLastMeasure and {@link AnnotatedMeasurement#probeTimerMillis}
    +     * are incremented via {@link DynamicEndpointSnitch#updateSamples()}
    +     */
    +    protected static class AnnotatedMeasurement
    +    {
    +        // Used to optimally target latency probes only on nodes that are both requested for ranking
    +        // and are not being measured. For example with token aware clients a large portion of the cluster will never
    +        // be ranked at all and therefore we won't probe them.
    +        public AtomicLong millisSinceLastRequest = new AtomicLong(MAX_PROBE_INTERVAL_MS);
    +        public AtomicLong millisSinceLastMeasure = new AtomicLong(0);
    +        public long probeTimerMillis = 0;
    +        public ScheduledFuture<?> probeFuture = null;
    +
    +        // The underlying measurement technique. E.g. a median filter (histogram) or an EMA filter, or ...
    +        public final ISnitchMeasurement measurement;
    +
    +        public AnnotatedMeasurement(ISnitchMeasurement measurement)
    +        {
    +            this.measurement = measurement;
    +        }
    +
    +        @Override
    +        public String toString()
    +        {
    +            return "AnnotatedMeasurement{" +
    +                   "millisSinceLastRequest=" + millisSinceLastRequest +
    +                   ", millisSinceLastMeasure=" + millisSinceLastMeasure +
    +                   ", probeTimerMillis=" + probeTimerMillis +
    +                   ", probeFuturePending=" + (probeFuture != null && !probeFuture.isDone()) +
    +                   ", measurementClass=" + measurement.getClass().getSimpleName() +
    +                   '}';
    +        }
    +    }
    +
    +    /**
    +     * Allows the subclasses to inject ISnitchMeasurement instances back into this common class so that common
    +     * functionality can be handled here
    +     * @param initialValue The initial value of the measurement, some implementations may use this, others may not
    +     * @return a constructed instance of an ISnitchMeasurement interface
    +     */
    +    abstract protected ISnitchMeasurement measurementImpl(long initialValue);
    +
    +    /**
    +     * Records a latency. This MUST be cheap as it is called in the fast path
    +     */
    +    public void receiveTiming(InetAddressAndPort address, long latency, LatencyMeasurementType measurementType)
    +    {
    +        if (measurementType == LatencyMeasurementType.IGNORE)
    +           return;
    +
    +        AnnotatedMeasurement sample = samples.get(address);
    +
    +        if (sample == null)
    +        {
    +            AnnotatedMeasurement maybeNewSample = new AnnotatedMeasurement(measurementImpl(latency));
    +            sample = samples.putIfAbsent(address, maybeNewSample);
    +            if (sample == null)
    +                sample = maybeNewSample;
    +        }
    +
    +        if (measurementType == LatencyMeasurementType.READ && sample.millisSinceLastMeasure.get() > 0)
    +            sample.millisSinceLastMeasure.lazySet(0);
    +
    +        sample.measurement.sample(latency);
    +    }
    +
    +    @VisibleForTesting
    +    protected void reset()
    +    {
    +        scores.clear();
    +        samples.clear();
    +    }
    +
    +    @VisibleForTesting
    +    void updateScores()
    +    {
    +        if (!StorageService.instance.isGossipActive())
    +            return;
    +
    +        if (!registered)
             {
    -            public void run()
    +            if (MessagingService.instance() != null)
                 {
    -                // we do this so that a host considered bad has a chance to recover, otherwise would we never try
    -                // to read from it, which would cause its score to never change
    -                reset();
    +                MessagingService.instance().register(this);
    +                registered = true;
                 }
    -        };
    +        }
     
    -        if (DatabaseDescriptor.isDaemonInitialized())
    +        this.scores = calculateScores();
    +    }
    +
    +    /**
    +     * This is generally expensive and is called periodically (semi-frequently) not on the fast path.
    +     * @return a freshly constructed scores map.
    +     */
    +    public Map<InetAddressAndPort, Double> calculateScores()
    +    {
    +        double maxLatency = 1;
    +
    +        Map<InetAddressAndPort, Double> measurements = new HashMap<>(samples.size());
    +
    +        // We're going to weight the latency for each host against the worst one we see, to
    +        // arrive at sort of a 'badness percentage' for them. First, find the worst for each:
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry : samples.entrySet())
             {
    -            updateSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(update, dynamicUpdateInterval, dynamicUpdateInterval, TimeUnit.MILLISECONDS);
    -            resetSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(reset, dynamicResetInterval, dynamicResetInterval, TimeUnit.MILLISECONDS);
    -            registerMBean();
    +            // This is expensive for e.g. the Histogram, so do it once and cache the result
    +            double measure = entry.getValue().measurement.measure();
    +            if (measure > maxLatency)
    +                maxLatency = measure;
    +            measurements.put(entry.getKey(), measure);
             }
    +
    +        HashMap<InetAddressAndPort, Double> newScores = new HashMap<>(measurements.size());
    +
    +        // now make another pass to do the weighting based on the maximums we found before
    +        for (Map.Entry<InetAddressAndPort, Double> entry : measurements.entrySet())
    +        {
    +            double score = entry.getValue() / maxLatency;
    +            // finally, add the severity without any weighting, since hosts scale this relative to their own load and the size of the task causing the severity.
    +            // "Severity" is basically a measure of compaction activity (CASSANDRA-3722).
    +            if (USE_SEVERITY)
    +                score += getSeverity(entry.getKey());
    +            // lowest score (least amount of badness) wins.
    +            newScores.put(entry.getKey(), score);
    +        }
    +
    +        return newScores;
         }
     
         /**
    -     * Update configuration from {@link DatabaseDescriptor} and estart the update-scheduler and reset-scheduler tasks
    +     * Update configuration from {@link DatabaseDescriptor} and restart the various scheduler tasks
          * if the configured rates for these tasks have changed.
          */
    -    public void applyConfigChanges()
    +    public void applyConfigChanges(int newDynamicUpdateInternal, int newDynamicLatencyProbeInterval, double newDynamicBadnessThreshold)
         {
    -        if (dynamicUpdateInterval != DatabaseDescriptor.getDynamicUpdateInterval())
    +        if (dynamicUpdateInterval != newDynamicUpdateInternal)
             {
    -            dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    +            dynamicUpdateInterval = newDynamicUpdateInternal;
                 if (DatabaseDescriptor.isDaemonInitialized())
                 {
    -                updateSchedular.cancel(false);
    -                updateSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(update, dynamicUpdateInterval, dynamicUpdateInterval, TimeUnit.MILLISECONDS);
    +                if (updateScoresScheduler != null)
    +                    updateScoresScheduler.cancel(false);
    +                updateScoresScheduler = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(this::updateScores, dynamicUpdateInterval, dynamicUpdateInterval, TimeUnit.MILLISECONDS);
                 }
             }
     
    -        if (dynamicResetInterval != DatabaseDescriptor.getDynamicResetInterval())
    +        if (this.dynamicLatencyProbeInterval != newDynamicLatencyProbeInterval)
             {
    -            dynamicResetInterval = DatabaseDescriptor.getDynamicResetInterval();
    +            this.dynamicLatencyProbeInterval = newDynamicLatencyProbeInterval;
                 if (DatabaseDescriptor.isDaemonInitialized())
                 {
    -                resetSchedular.cancel(false);
    -                resetSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(reset, dynamicResetInterval, dynamicResetInterval, TimeUnit.MILLISECONDS);
    +                if (updateSamplesScheduler != null)
    +                    updateSamplesScheduler.cancel(false);
    +
    +                if (newDynamicLatencyProbeInterval > 0)
    +                {
    +                    probeRateLimiter.setRate(dynamicLatencyProbeInterval);
    +                    updateSamplesScheduler = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(this::updateSamples, dynamicLatencyProbeInterval, dynamicLatencyProbeInterval, TimeUnit.MILLISECONDS);
    --- End diff --
    
    Now that we have this a thread for pings we can also do the scheduling work on it.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225327904
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -22,121 +22,194 @@
     import java.net.InetAddress;
     import java.net.UnknownHostException;
     import java.util.*;
    -import java.util.concurrent.ConcurrentHashMap;
     import java.util.concurrent.ScheduledFuture;
     import java.util.concurrent.TimeUnit;
     import java.util.stream.Collectors;
     
    -import com.codahale.metrics.ExponentiallyDecayingReservoir;
     import javax.management.MBeanServer;
     import javax.management.ObjectName;
     
    -import com.codahale.metrics.Snapshot;
    +import com.google.common.annotations.VisibleForTesting;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
     import org.apache.cassandra.concurrent.ScheduledExecutors;
     import org.apache.cassandra.config.DatabaseDescriptor;
     import org.apache.cassandra.gms.ApplicationState;
     import org.apache.cassandra.gms.EndpointState;
     import org.apache.cassandra.gms.Gossiper;
     import org.apache.cassandra.gms.VersionedValue;
    +import org.apache.cassandra.net.IAsyncCallback;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +import org.apache.cassandra.net.MessageIn;
    +import org.apache.cassandra.net.MessageOut;
     import org.apache.cassandra.net.MessagingService;
    +import org.apache.cassandra.net.PingMessage;
     import org.apache.cassandra.service.StorageService;
     import org.apache.cassandra.utils.FBUtilities;
     
    +import static org.apache.cassandra.net.MessagingService.Verb.PING;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.LARGE_MESSAGE;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.SMALL_MESSAGE;
    +
    +
     /**
      * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector
    + * Note that the subclasses (e.g. {@link DynamicEndpointSnitchHistogram}) are responsible for actually measuring
    + * latency and populating the {@link #scores} map.
      */
    -public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
    +public abstract class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
     {
    -    private static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    -
    -    private static final double ALPHA = 0.75; // set to 0.75 to make EDS more biased to towards the newer values
    -    private static final int WINDOW_SIZE = 100;
    -
    -    private volatile int dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    -    private volatile int dynamicResetInterval = DatabaseDescriptor.getDynamicResetInterval();
    -    private volatile double dynamicBadnessThreshold = DatabaseDescriptor.getDynamicBadnessThreshold();
    +    private static final Logger logger = LoggerFactory.getLogger(DynamicEndpointSnitch.class);
    +
    +    // Subclass specific functionality
    +    protected static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    protected boolean registered = false;
    +    // The scores map is updated via copy in updateScores
    +    // We keep it in the base class for performance reasons (so it can be easily aliased)
    +    protected volatile Map<InetAddressAndPort, Double> scores = new HashMap<>();
    +
    +    // Rate limit how often we generate latency probes
    +    protected long nextAllowedProbeGenerationTime;
    --- End diff --
    
    As we discussed I don't think we really need this. We can back off probing nodes we aren't talking to very frequently to get back to the only check a node every 10 minutes behavior. That coupled with not probing nodes we aren't even considering for queries should do get the messaging down.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r238172698
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -154,31 +326,203 @@ private void registerMBean()
     
         public void close()
         {
    -        updateSchedular.cancel(false);
    -        resetSchedular.cancel(false);
    +        if (updateScoresScheduler != null)
    +            updateScoresScheduler.cancel(false);
    +        if (updateSamplesScheduler != null)
    +            updateSamplesScheduler.cancel(false);
    --- End diff --
    
    Ok, I added a cancel+wait method to do this, and added in some synchronization around `close`/`open` that I believe is is needed to ensure that this method works 100% of the time during snitch updates.
    
    This actually fixes a bug I believe exists in trunk where we just hoped that we didn't get an exception on Snitch construction in https://github.com/apache/cassandra/blob/f3609995c09570d523527d9bd0fd69c2bc65d986/src/java/org/apache/cassandra/service/StorageService.java#L4975-L4986


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r236763554
  
    --- Diff: test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java ---
    @@ -60,57 +130,229 @@ private static EndpointsForRange full(InetAddressAndPort... endpoints)
         }
     
         @Test
    -    public void testSnitch() throws InterruptedException, IOException, ConfigurationException
    +    public void testSortedByProximity() throws InterruptedException, IOException, ConfigurationException
         {
    -        // do this because SS needs to be initialized before DES can work properly.
    -        StorageService.instance.unsafeInitialize();
    -        SimpleSnitch ss = new SimpleSnitch();
    -        DynamicEndpointSnitch dsnitch = new DynamicEndpointSnitch(ss, String.valueOf(ss.hashCode()));
    -        InetAddressAndPort self = FBUtilities.getBroadcastAddressAndPort();
    -        InetAddressAndPort host1 = InetAddressAndPort.getByName("127.0.0.2");
    -        InetAddressAndPort host2 = InetAddressAndPort.getByName("127.0.0.3");
    -        InetAddressAndPort host3 = InetAddressAndPort.getByName("127.0.0.4");
    -        InetAddressAndPort host4 = InetAddressAndPort.getByName("127.0.0.5");
    -        List<InetAddressAndPort> hosts = Arrays.asList(host1, host2, host3);
    +        InetAddressAndPort self = hosts[0];
    +        List<InetAddressAndPort> allHosts = Arrays.asList(hosts[1], hosts[2], hosts[3]);
     
             // first, make all hosts equal
    -        setScores(dsnitch, 1, hosts, 10, 10, 10);
    -        EndpointsForRange order = full(host1, host2, host3);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    -
    -        // make host1 a little worse
    -        setScores(dsnitch, 1, hosts, 20, 10, 10);
    -        order = full(host2, host3, host1);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    -
    -        // make host2 as bad as host1
    -        setScores(dsnitch, 2, hosts, 15, 20, 10);
    -        order = full(host3, host1, host2);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    -
    -        // make host3 the worst
    -        setScores(dsnitch, 3, hosts, 10, 10, 30);
    -        order = full(host1, host2, host3);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    -
    -        // make host3 equal to the others
    -        setScores(dsnitch, 5, hosts, 10, 10, 10);
    -        order = full(host1, host2, host3);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    +        setScores(dsnitch, 1, allHosts, 10, 10, 10);
    +        EndpointsForRange order = full(hosts[1], hosts[2], hosts[3]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        // make hosts[1] a little worse
    +        setScores(dsnitch, 2, allHosts, 20, 10, 10);
    +        order = full(hosts[2], hosts[3], hosts[1]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        // make hosts[2] as bad as hosts[1]
    +        setScores(dsnitch, 4, allHosts, 15, 20, 10);
    +        order = full(hosts[3], hosts[1], hosts[2]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        // make hosts[3] the worst
    +        setScores(dsnitch, 10, allHosts, 10, 10, 30);
    +        order = full(hosts[1], hosts[2], hosts[3]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        // make hosts[3] equal to the others
    +        setScores(dsnitch, 15, allHosts, 10, 10, 10);
    +        order = full(hosts[1], hosts[2], hosts[3]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
     
             /// Tests CASSANDRA-6683 improvements
             // make the scores differ enough from the ideal order that we sort by score; under the old
             // dynamic snitch behavior (where we only compared neighbors), these wouldn't get sorted
    -        setScores(dsnitch, 20, hosts, 10, 70, 20);
    -        order = full(host1, host3, host2);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    +        dsnitch.reset();
    +        setScores(dsnitch, 20, allHosts, 10, 70, 20);
    +        order = full(hosts[1], hosts[3], hosts[2]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        order = full(hosts[4], hosts[1], hosts[3], hosts[2]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3], hosts[4])));
    +
    +        setScores(dsnitch, 40, allHosts, 10, 10, 11);
    +        order = full(hosts[4], hosts[1], hosts[2], hosts[3]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3], hosts[4])));
    +    }
    +
    +    // CASSANDRA-14459 improvements to add latency probes instead of resets
    +    @Test
    +    public void testLatencyProbeNeeded()
    +    {
    +        InetAddressAndPort self = hosts[0];
    +
    +        // Four hosts, two are very latent, but all are requested for ranking
    +        dsnitch.receiveTiming(hosts[1], 20, LatencyMeasurementType.READ);
    +        dsnitch.receiveTiming(hosts[2], 10, LatencyMeasurementType.READ);
    +        dsnitch.receiveTiming(hosts[3], 1000, LatencyMeasurementType.READ);
    +        dsnitch.receiveTiming(hosts[4], 1000, LatencyMeasurementType.READ);
    +        dsnitch.updateScores();
    +
    +        EndpointsForRange orderBefore = full(hosts[2], hosts[1], hosts[3], hosts[4]);
    +
    +        Map<InetAddressAndPort, DynamicEndpointSnitch.AnnotatedMeasurement> measurements = dsnitch.getMeasurementsWithPort();
    +        // At this point we haven't ranked any of hosts so we should need no probes even
    +        DynamicEndpointSnitch.calculateProbes(measurements, dsnitch.dynamicLatencyProbeInterval);
    +        assertFalse(measurements.values().stream().anyMatch(m -> m.probeTimerMillis > 0));
    +        assertTrue(measurements.values().stream().allMatch(m -> m.probeFuture == null));
    +
    +        // Two hosts continue receiving traffic but the last two are always ranked by the snitch
    +        for (int i = 0; i < 10; i++)
    +        {
    +            dsnitch.receiveTiming(hosts[1], 20, LatencyMeasurementType.READ);
    +            dsnitch.receiveTiming(hosts[2], 10, LatencyMeasurementType.READ);
    +            Util.assertRCEquals(orderBefore, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3], hosts[4])));
    --- End diff --
    
    It shouldn't change, I'm mostly just calling that for the `sortedByProximity` call (might as well check the return value while we're calling it).
    
    I added in a call to `dsnitch.updateScores()` after the `receiveTimings` just to double triple check.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r236765727
  
    --- Diff: test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java ---
    @@ -19,34 +19,104 @@
     package org.apache.cassandra.locator;
     
     import java.io.IOException;
    +import java.net.UnknownHostException;
     import java.util.*;
     
    -import org.junit.BeforeClass;
    +import org.junit.After;
    +import org.junit.Before;
     import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
     
     import org.apache.cassandra.Util;
     import org.apache.cassandra.config.DatabaseDescriptor;
     import org.apache.cassandra.exceptions.ConfigurationException;
    +import org.apache.cassandra.gms.Gossiper;
    +import org.apache.cassandra.locator.dynamicsnitch.DynamicEndpointSnitchEMA;
    +import org.apache.cassandra.locator.dynamicsnitch.DynamicEndpointSnitchHistogram;
    +import org.apache.cassandra.locator.dynamicsnitch.DynamicEndpointSnitchLegacyHistogram;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +import org.apache.cassandra.net.async.TestScheduledFuture;
     import org.apache.cassandra.service.StorageService;
     import org.apache.cassandra.utils.FBUtilities;
     
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertFalse;
    +import static org.junit.Assert.assertTrue;
    +
    +@RunWith(Parameterized.class)
     public class DynamicEndpointSnitchTest
     {
    +    private static InetAddressAndPort[] hosts;
    +    // Reduce the update interval significantly so that tests run quickly
    +    private static final long UPDATE_INTERVAL_MS = 10;
    +    // Intentially 31 and a little bit instead of 30 seconds flat so this doesn't divide evenly into the default
    +    // MAX_PROBE_INTERVAL_MS. Also pretty high so latency probes don't interfere with the unit tests
    +    private static final long PING_INTERVAL_MS = 31 * 1003;
    +
    +    private final DynamicEndpointSnitch dsnitch;
    +
    +    public DynamicEndpointSnitchTest(DynamicEndpointSnitch dsnitch)
    +    {
    +        this.dsnitch = dsnitch;
    +    }
    +
    +    @Before
    +    public void prepareDES()
    +    {
    +        for (InetAddressAndPort host : hosts)
    +        {
    +            Gossiper.instance.initializeNodeUnsafe(host, UUID.randomUUID(), 1);
    +            Gossiper.instance.realMarkAlive(host, Gossiper.instance.getEndpointStateForEndpoint(host));
    +        }
    +        dsnitch.reset();
    +    }
     
    -    @BeforeClass
    -    public static void setupDD()
    +    @Parameterized.Parameters(name="{index}: {0}")
    +    public static Iterable<?> getDESImplementation() throws UnknownHostException
         {
             DatabaseDescriptor.daemonInitialization();
    +        // do this because SS needs to be initialized before DES can work properly.
    +        StorageService.instance.unsafeInitialize();
    +
    +        hosts = new InetAddressAndPort[] {
    +            FBUtilities.getBroadcastAddressAndPort(),
    +            InetAddressAndPort.getByName("127.0.0.2"),
    +            InetAddressAndPort.getByName("127.0.0.3"),
    +            InetAddressAndPort.getByName("127.0.0.4"),
    +            InetAddressAndPort.getByName("127.0.0.5"),
    +        };
    +
    +        SimpleSnitch ss1 = new SimpleSnitch();
    +        DynamicEndpointSnitch probeDES = new DynamicEndpointSnitchHistogram(ss1, String.valueOf(ss1.hashCode()));
    +        probeDES.applyConfigChanges((int) UPDATE_INTERVAL_MS, (int) PING_INTERVAL_MS, DatabaseDescriptor.getDynamicBadnessThreshold());
    +
    +        SimpleSnitch ss2 = new SimpleSnitch();
    +        DynamicEndpointSnitch oldDES = new DynamicEndpointSnitchLegacyHistogram(ss2, String.valueOf(ss2.hashCode()));
    +        oldDES.applyConfigChanges((int) UPDATE_INTERVAL_MS, (int) PING_INTERVAL_MS, DatabaseDescriptor.getDynamicBadnessThreshold());
    +
    +        SimpleSnitch ss3 = new SimpleSnitch();
    +        DynamicEndpointSnitch emaDES = new DynamicEndpointSnitchEMA(ss3, String.valueOf(ss3.hashCode()));
    +        emaDES.applyConfigChanges((int) UPDATE_INTERVAL_MS, (int) PING_INTERVAL_MS, DatabaseDescriptor.getDynamicBadnessThreshold());
    +
    +        return Arrays.asList(probeDES, oldDES, emaDES);
    +    }
    +
    +    @After
    +    public void resetDES()
    +    {
    +        dsnitch.reset();
         }
     
         private static void setScores(DynamicEndpointSnitch dsnitch, int rounds, List<InetAddressAndPort> hosts, Integer... scores) throws InterruptedException
         {
             for (int round = 0; round < rounds; round++)
             {
                 for (int i = 0; i < hosts.size(); i++)
    -                dsnitch.receiveTiming(hosts.get(i), scores[i]);
    +                dsnitch.receiveTiming(hosts.get(i), scores[i], LatencyMeasurementType.READ);
             }
    -        Thread.sleep(150);
    +        // Slightly higher than the update interval to allow scores to propagate
    +        Thread.sleep(UPDATE_INTERVAL_MS + 10);
    --- End diff --
    
    Yea I mean the status quo was this could be pretty flakey, one of the reasons that I separated out `updateScores` was so that we can forcibly recalculate scores using it. I went ahead and switched this method to just forcibly call `updateScores`, I guess we don't need a test that the threading works?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225352067
  
    --- Diff: src/java/org/apache/cassandra/gms/Gossiper.java ---
    @@ -1055,7 +1056,6 @@ public boolean isLatencyForSnitch()
                 {
                     return false;
                 }
    -
    --- End diff --
    
    ack, got lost in the various additions and removals of whether EchoMessages counted or not


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r240311669
  
    --- Diff: test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java ---
    @@ -18,35 +18,99 @@
     
     package org.apache.cassandra.locator;
     
    -import java.io.IOException;
    +import java.net.UnknownHostException;
    --- End diff --
    
    My code coverage is a bit wonky today, but is sendPingMessage tested? Is it possible?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r229460094
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -154,31 +326,203 @@ private void registerMBean()
     
         public void close()
         {
    -        updateSchedular.cancel(false);
    -        resetSchedular.cancel(false);
    +        if (updateScoresScheduler != null)
    +            updateScoresScheduler.cancel(false);
    +        if (updateSamplesScheduler != null)
    +            updateSamplesScheduler.cancel(false);
    +
    +        for (AnnotatedMeasurement measurement : samples.values())
    +        {
    +            if (measurement.probeFuture != null)
    +                measurement.probeFuture.cancel(false);
    +
    +            measurement.millisSinceLastMeasure.set(0);
    +            measurement.millisSinceLastRequest.set(MAX_PROBE_INTERVAL_MS);
    +            measurement.probeTimerMillis = 0;
    +        }
     
             MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
             try
             {
    -            mbs.unregisterMBean(new ObjectName(mbeanName));
    +            if (mbeanRegistered)
    +                mbs.unregisterMBean(new ObjectName(mbeanName));
             }
             catch (Exception e)
             {
                 throw new RuntimeException(e);
             }
         }
     
    +    /**
    +     * Background task running on the samples dictionary. The default implementation sends latency probes (PING)
    +     * messages to explore nodes that we have not received timings for recently but have ranked in
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}.
    +     */
    +    protected void updateSamples()
    +    {
    +        // Split calculation of probe timers from sending probes for testability
    +        calculateProbes(samples, dynamicLatencyProbeInterval);
    +
    +        if (!StorageService.instance.isGossipActive())
    --- End diff --
    
    Same question here, should this be at the beginning as well?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225697426
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -22,121 +22,194 @@
     import java.net.InetAddress;
     import java.net.UnknownHostException;
     import java.util.*;
    -import java.util.concurrent.ConcurrentHashMap;
     import java.util.concurrent.ScheduledFuture;
     import java.util.concurrent.TimeUnit;
     import java.util.stream.Collectors;
     
    -import com.codahale.metrics.ExponentiallyDecayingReservoir;
     import javax.management.MBeanServer;
     import javax.management.ObjectName;
     
    -import com.codahale.metrics.Snapshot;
    +import com.google.common.annotations.VisibleForTesting;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
     import org.apache.cassandra.concurrent.ScheduledExecutors;
     import org.apache.cassandra.config.DatabaseDescriptor;
     import org.apache.cassandra.gms.ApplicationState;
     import org.apache.cassandra.gms.EndpointState;
     import org.apache.cassandra.gms.Gossiper;
     import org.apache.cassandra.gms.VersionedValue;
    +import org.apache.cassandra.net.IAsyncCallback;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +import org.apache.cassandra.net.MessageIn;
    +import org.apache.cassandra.net.MessageOut;
     import org.apache.cassandra.net.MessagingService;
    +import org.apache.cassandra.net.PingMessage;
     import org.apache.cassandra.service.StorageService;
     import org.apache.cassandra.utils.FBUtilities;
     
    +import static org.apache.cassandra.net.MessagingService.Verb.PING;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.LARGE_MESSAGE;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.SMALL_MESSAGE;
    +
    +
     /**
      * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector
    + * Note that the subclasses (e.g. {@link DynamicEndpointSnitchHistogram}) are responsible for actually measuring
    + * latency and populating the {@link #scores} map.
      */
    -public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
    +public abstract class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
     {
    -    private static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    -
    -    private static final double ALPHA = 0.75; // set to 0.75 to make EDS more biased to towards the newer values
    -    private static final int WINDOW_SIZE = 100;
    -
    -    private volatile int dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    -    private volatile int dynamicResetInterval = DatabaseDescriptor.getDynamicResetInterval();
    -    private volatile double dynamicBadnessThreshold = DatabaseDescriptor.getDynamicBadnessThreshold();
    +    private static final Logger logger = LoggerFactory.getLogger(DynamicEndpointSnitch.class);
    +
    +    // Subclass specific functionality
    +    protected static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    protected boolean registered = false;
    +    // The scores map is updated via copy in updateScores
    +    // We keep it in the base class for performance reasons (so it can be easily aliased)
    +    protected volatile Map<InetAddressAndPort, Double> scores = new HashMap<>();
    +
    +    // Rate limit how often we generate latency probes
    +    protected long nextAllowedProbeGenerationTime;
    +    protected long nextProbeGenerationTime;
    +    protected int currentProbePosition = 0;
    +    protected final List<InetAddressAndPort> latencyProbeSequence = new ArrayList<>();
    +
    +    // DES general functionality
    +    protected volatile int dynamicUpdateInterval = -1;
    +    protected volatile int dynamicLatencyProbeInterval = -1;
    +    protected volatile double dynamicBadnessThreshold = 0;
     
         // the score for a merged set of endpoints must be this much worse than the score for separate endpoints to
         // warrant not merging two ranges into a single range
         private static final double RANGE_MERGING_PREFERENCE = 1.5;
     
         private String mbeanName;
    -    private boolean registered = false;
    -
    -    private volatile HashMap<InetAddressAndPort, Double> scores = new HashMap<>();
    -    private final ConcurrentHashMap<InetAddressAndPort, ExponentiallyDecayingReservoir> samples = new ConcurrentHashMap<>();
    +    private boolean mbeanRegistered = false;
     
         public final IEndpointSnitch subsnitch;
     
    -    private volatile ScheduledFuture<?> updateSchedular;
    -    private volatile ScheduledFuture<?> resetSchedular;
    +    private volatile ScheduledFuture<?> updateScheduler;
    +    private volatile ScheduledFuture<?> latencyProbeScheduler;
     
    -    private final Runnable update;
    -    private final Runnable reset;
    +    private final Runnable update = this::updateScores;
    +    private final Runnable latencyProbe = this::maybeSendLatencyProbe;
     
         public DynamicEndpointSnitch(IEndpointSnitch snitch)
         {
             this(snitch, null);
         }
     
    -    public DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
    +    protected DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
         {
             mbeanName = "org.apache.cassandra.db:type=DynamicEndpointSnitch";
             if (instance != null)
                 mbeanName += ",instance=" + instance;
             subsnitch = snitch;
    -        update = new Runnable()
    -        {
    -            public void run()
    -            {
    -                updateScores();
    -            }
    -        };
    -        reset = new Runnable()
    -        {
    -            public void run()
    -            {
    -                // we do this so that a host considered bad has a chance to recover, otherwise would we never try
    -                // to read from it, which would cause its score to never change
    -                reset();
    -            }
    -        };
     
             if (DatabaseDescriptor.isDaemonInitialized())
             {
    -            updateSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(update, dynamicUpdateInterval, dynamicUpdateInterval, TimeUnit.MILLISECONDS);
    -            resetSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(reset, dynamicResetInterval, dynamicResetInterval, TimeUnit.MILLISECONDS);
    +            applyConfigChanges();
                 registerMBean();
             }
    +        nextProbeGenerationTime = System.nanoTime();
    +    }
    +
    +    /**
    +     * Records a latency. This MUST be cheap as it is called in the fast path
    +     */
    +    abstract public void receiveTiming(InetAddressAndPort address, long latency, LatencyMeasurementType measurementType);
    +
    +    /**
    +     * Called periodically from {@link DynamicEndpointSnitch#updateScores()}. This method returns a new score map
    +     * that will be assigned to the {@link DynamicEndpointSnitch#scores} map.
    +     * This is generally more expensive than {@link DynamicEndpointSnitch#receiveTiming}
    +     */
    +    abstract public Map<InetAddressAndPort, Double> calculateScores();
    --- End diff --
    
    Additionally, the might not end up being the super class for future snitch implementations since they can implement IEndpointSnitch and replace whatever they want. All we have to worry about today is solving for EMA and Histogram I think.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225336309
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitchEMA.java ---
    @@ -0,0 +1,181 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.cassandra.locator;
    +
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.concurrent.ConcurrentHashMap;
    +
    +import org.apache.cassandra.gms.Gossiper;
    +import org.apache.cassandra.metrics.ExponentialMovingAverage;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +
    +
    +/**
    + * A dynamic snitching implementation that uses Exponentially Decaying Histograms to prefer or
    + * de-prefer hosts
    + *
    + * This was the default implementation prior to Cassandra 4.0 and is being left as the default
    + * in 4.0
    + */
    +public class DynamicEndpointSnitchEMA extends DynamicEndpointSnitch
    +{
    +    // A ~10 sample EMA
    +    private static final double EMA_ALPHA = 0.10;
    +
    +    private final ConcurrentHashMap<InetAddressAndPort, AnnotatedEMA> samples = new ConcurrentHashMap<>();
    +
    +    /**
    +     * Adds two boolean markers to the ExponentialMovingAverage for telling if the data has been
    +     * updated or requested recently.
    +     *
    +     * recentlyMeasured is updated through {@link AnnotatedEMA#update(long, boolean)}
    +     * recentlyRequested is updated through {@link DynamicEndpointSnitch#markRequested}
    +     *
    +     * Both markers are periodically reset via {@link DynamicEndpointSnitch#latencyProbeNeeded(long)}
    +     */
    +    private static class AnnotatedEMA extends ExponentialMovingAverage
    +    {
    +        volatile boolean recentlyRequested = false;
    +        volatile boolean recentlyMeasured = false;
    +
    +        AnnotatedEMA(double alpha, double initialValue)
    +        {
    +            super(alpha, initialValue);
    +        }
    +
    +        public void update(long value, boolean isRealRead)
    +        {
    +            recentlyMeasured = recentlyMeasured || isRealRead;
    --- End diff --
    
    This can be |=. This isn't checking of recently measured is already set before writing which means it's always going to pull the cache line in for write.
    
    isRealRead sounds a little awkward and not proscriptive. What we are really trying to say is that it should count as recently measured. I think it might be clearer if it simply didn't override update.
    
    I am also a developing fan of enums instead of booleans since true/false is just not that informative.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r240299652
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -214,19 +600,27 @@ public String getDatacenter(InetAddressAndPort endpoint)
             {
                 if (subsnitchScore > (sortedScoreIterator.next() * (1.0 + dynamicBadnessThreshold)))
                 {
    -                return sortedByProximityWithScore(address, replicas);
    +                return sortedByProximityWithScore(address, replicas, aliasedScores);
                 }
             }
     
             return replicas;
         }
     
         // Compare endpoints given an immutable snapshot of the scores
    -    private int compareEndpoints(InetAddressAndPort target, Replica a1, Replica a2, Map<InetAddressAndPort, Double> scores)
    +    public int compareEndpoints(InetAddressAndPort target, Replica a1, Replica a2)
    --- End diff --
    
    This comment seems wrong and the entire scenario is a little confusing?
    Why can't this be removed?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225298351
  
    --- Diff: conf/cassandra.yaml ---
    @@ -947,10 +947,10 @@ endpoint_snitch: SimpleSnitch
     
     # controls how often to perform the more expensive part of host score
     # calculation
    -dynamic_snitch_update_interval_in_ms: 100 
    -# controls how often to reset all host scores, allowing a bad host to
    -# possibly recover
    -dynamic_snitch_reset_interval_in_ms: 600000
    +dynamic_snitch_update_interval_in_ms: 100
    +# controls how often this node is permitted to send a latency probe to
    +# determine if previously slow nodes have recovered. Set to -1 to disable
    +dynamic_snitch_latency_probe_interval_in_ms: 1000
    --- End diff --
    
    Add dynamic snitch class name as a commented out option?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r236776225
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -154,31 +326,203 @@ private void registerMBean()
     
         public void close()
         {
    -        updateSchedular.cancel(false);
    -        resetSchedular.cancel(false);
    +        if (updateScoresScheduler != null)
    +            updateScoresScheduler.cancel(false);
    +        if (updateSamplesScheduler != null)
    +            updateSamplesScheduler.cancel(false);
    +
    +        for (AnnotatedMeasurement measurement : samples.values())
    +        {
    +            if (measurement.probeFuture != null)
    +                measurement.probeFuture.cancel(false);
    +
    +            measurement.millisSinceLastMeasure.set(0);
    +            measurement.millisSinceLastRequest.set(MAX_PROBE_INTERVAL_MS);
    +            measurement.probeTimerMillis = 0;
    +        }
     
             MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
             try
             {
    -            mbs.unregisterMBean(new ObjectName(mbeanName));
    +            if (mbeanRegistered)
    +                mbs.unregisterMBean(new ObjectName(mbeanName));
             }
             catch (Exception e)
             {
                 throw new RuntimeException(e);
             }
         }
     
    +    /**
    +     * Background task running on the samples dictionary. The default implementation sends latency probes (PING)
    +     * messages to explore nodes that we have not received timings for recently but have ranked in
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}.
    +     */
    +    protected void updateSamples()
    +    {
    +        // Split calculation of probe timers from sending probes for testability
    +        calculateProbes(samples, dynamicLatencyProbeInterval);
    +
    +        if (!StorageService.instance.isGossipActive())
    +            return;
    +
    +        schedulePings(samples);
    +    }
    +
    +    /**
    +     * This method mutates the passed AnnotatedMeasurements to implement capped exponential backoff per endpoint.
    +     *
    +     * The algorithm is as follows:
    +     * 1. All samples get their millisSinceLastMeasure and millisSinceLastRequest fields
    +     *    incremented by the passed interval
    +     * 2. Any recently requested (ranked) endpoints that have not been measured recently (e.g. because the snitch
    +     *    has sent them no traffic) get probes with exponential backoff.
    +     *
    +     * The backoff is capped at MAX_PROBE_INTERVAL_MS. Furthermore the probes are stopped after
    +     * MAX_PROBE_INTERVAL_MS of no ranking requests as well.
    +     *
    +     * At the end of this method, any passed AnnotatedMeasurements that need latency probes will have non zero
    +     * probeTimerMillis members set.
    +     */
    +    @VisibleForTesting
    +    static void calculateProbes(Map<InetAddressAndPort, AnnotatedMeasurement> samples, long intervalMillis) {
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry: samples.entrySet())
    +        {
    +            if (entry.getKey().equals(FBUtilities.getBroadcastAddressAndPort()))
    +                continue;
    +
    +            AnnotatedMeasurement measurement = entry.getValue();
    +            long lastMeasure = measurement.millisSinceLastMeasure.getAndAdd(intervalMillis);
    +            long lastRequest = measurement.millisSinceLastRequest.getAndAdd(intervalMillis);
    +
    +            if (lastMeasure >= MIN_PROBE_INTERVAL_MS && lastRequest < MAX_PROBE_INTERVAL_MS)
    +            {
    +                if (measurement.probeTimerMillis == 0)
    +                {
    +                    measurement.probeTimerMillis = intervalMillis;
    +                }
    +                else if (measurement.probeFuture != null && measurement.probeFuture.isDone())
    +                {
    +                    measurement.probeTimerMillis = Math.min(MAX_PROBE_INTERVAL_MS, measurement.probeTimerMillis * 2);
    --- End diff --
    
    Ack, +1


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r226142547
  
    --- Diff: src/java/org/apache/cassandra/net/MessagingService.java ---
    @@ -665,9 +665,9 @@ public void updateBackPressureOnSend(InetAddressAndPort host, IAsyncCallback cal
         /**
          * Updates the back-pressure state on reception from the given host if enabled and the given message callback supports it.
          *
    -     * @param host The replica host the back-pressure state refers to.
    +     * @param host     The replica host the back-pressure state refers to.
    --- End diff --
    
    Ack, all setup thanks :-)


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225350948
  
    --- Diff: src/java/org/apache/cassandra/service/StorageServiceMBean.java ---
    @@ -469,15 +469,22 @@
          * If {@code epSnitchClassName} is specified, it will configure a new snitch instance and make it a
          * 'dynamic snitch' if {@code dynamic} is specified and {@code true}.
          *
    -     * The parameters {@code dynamicUpdateInterval}, {@code dynamicResetInterval} and {@code dynamicBadnessThreshold}
    +     * The parameters {@code dynamicUpdateInterval}, {@code dynamicLatencyProbeInterval} and {@code dynamicBadnessThreshold}
          * can be specified individually to update the parameters of the dynamic snitch during runtime.
          *
    -     * @param epSnitchClassName        the canonical path name for a class implementing IEndpointSnitch
    -     * @param dynamic                  boolean that decides whether dynamicsnitch is used or not - only valid, if {@code epSnitchClassName} is specified
    -     * @param dynamicUpdateInterval    integer, in ms (defaults to the value configured in cassandra.yaml, which defaults to 100)
    -     * @param dynamicResetInterval     integer, in ms (defaults to the value configured in cassandra.yaml, which defaults to 600,000)
    -     * @param dynamicBadnessThreshold  double, (defaults to the value configured in cassandra.yaml, which defaults to 0.0)
    +     * @param epSnitchClassName          the canonical path name for a class implementing IEndpointSnitch
    +     * @param dynamicBadnessThreshold    double, (defaults to the value configured in cassandra.yaml, which defaults to 0.0)
    +     * @param dynamic                    boolean that decides whether dynamicsnitch is used or not - only valid, if {@code epSnitchClassName} is specified
    +     * @param dynamicUpdateInterval      integer, in ms (defaults to the value configured in cassandra.yaml, which defaults to 100)
    +     * @param dynamicLatencyProbeInterval integer, in ms (defaults to the value configured in cassandra.yaml, which defaults to 1,000)
          */
    +    public void updateSnitch(String epSnitchClassName, Double dynamicBadnessThreshold, Boolean dynamic, Integer dynamicUpdateInterval, Integer dynamicLatencyProbeInterval) throws ClassNotFoundException;
    --- End diff --
    
    Hm, like I said before, I didn't want to expose the DES pluggability in the JMX interface since I wasn't sure that we wanted to support it as a user interface (I just want to give a path forward for devs to experiment with different implementations which e.g. use different measurement techniques or load (un)balancing techniques).
    
    Like you say, `IEndpointSnitch`'s are not configurable currently (they have to have an empty constructor that e.g. calls their own `loadConfiguration` method a la `GossipingPropertyFileSnitch`. I think requiring custom DES implementations to do something similar isn't unreasonable (and it reduces complexity).
    



---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r237958792
  
    --- Diff: src/java/org/apache/cassandra/locator/dynamicsnitch/DynamicEndpointSnitchEMA.java ---
    @@ -0,0 +1,133 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.cassandra.locator.dynamicsnitch;
    +
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.Optional;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +
    +import org.apache.cassandra.locator.DynamicEndpointSnitch;
    +import org.apache.cassandra.locator.IEndpointSnitch;
    +import org.apache.cassandra.locator.InetAddressAndPort;
    +import org.apache.cassandra.metrics.ExponentialMovingAverage;
    +
    +
    +/**
    + * A dynamic snitching implementation that uses Exponentially Moving Averages as a low pass filter to prefer
    + * or de-prefer hosts
    + *
    + * This implementation generates a few orders of magnitude less garbage than histograms and is close to 10x faster,
    + * but as it is not a Median LPF (it is an Average LPF), it is more vulnerable to noise. This may be acceptable but
    + * given the significant change in behavior this is not the default in 4.0
    + */
    +public class DynamicEndpointSnitchEMA extends DynamicEndpointSnitch
    +{
    +    // A ~10 sample EMA heavily weighted to the past values to minimize noise
    +    private static final double EMA_ALPHA = 0.10;
    +
    +    protected static class EMASnitchMeasurement implements ISnitchMeasurement
    +    {
    +        public final ExponentialMovingAverage avg;
    +
    +        EMASnitchMeasurement(double initial)
    +        {
    +            avg = new ExponentialMovingAverage(EMA_ALPHA, initial);
    +        }
    +
    +        @Override
    +        public void sample(long value)
    +        {
    +            avg.update(value);
    +        }
    +
    +        @Override
    +        public double measure()
    +        {
    +            return avg.getAvg();
    +        }
    +
    +        @Override
    +        public Iterable<Double> measurements()
    +        {
    +            return Collections.singletonList(avg.getAvg());
    +        }
    +    }
    +
    +    // Called via reflection
    +    public DynamicEndpointSnitchEMA(IEndpointSnitch snitch)
    +    {
    +        this(snitch, "ema");
    +    }
    +
    +    public DynamicEndpointSnitchEMA(IEndpointSnitch snitch, String instance)
    +    {
    +        super(snitch, instance);
    +    }
    +
    +    @Override
    +    protected ISnitchMeasurement measurementImpl(long initialValue)
    +    {
    +        return new EMASnitchMeasurement(initialValue);
    +    }
    +
    +    /**
    +     * Unlike the Histogram implementation, calling this measure method is reasonably cheap (doesn't require a
    +     * Snapshot or anything) so we can skip a round of iterations and just normalize the scores slightly
    +     * differently
    +     */
    +    @Override
    +    public Map<InetAddressAndPort, Double> calculateScores()
    +    {
    +        // We're going to weight the latency for each host against the worst one we see, to
    +        // arrive at sort of a 'badness percentage' for them. First, find the worst for each:
    +        HashMap<InetAddressAndPort, Double> newScores = new HashMap<>(samples.size());
    +        Optional<Double> maxObservedAvgLatency = samples.values().stream()
    --- End diff --
    
    I went ahead and removed the EMA entirely, let's follow up on that in another change.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225347547
  
    --- Diff: src/java/org/apache/cassandra/net/IAsyncCallback.java ---
    @@ -31,11 +31,23 @@
         void response(MessageIn<T> msg);
     
         /**
    +     * @deprecated Use {@link #latencyMeasurementType()}. This is left for backwards
    +     *             compatibility with existing IAsyncCallback implementations but will
    +     *             be removed at a future date.
          * @return true if this callback is on the read path and its latency should be
          * given as input to the dynamic snitch.
          */
    +    @Deprecated
    --- End diff --
    
    Removing it will cause a lot of changes, I can do that but it will make the diff much larger (I see about 20 files that would need changes)


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225347778
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitchEMA.java ---
    @@ -0,0 +1,181 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.cassandra.locator;
    +
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.concurrent.ConcurrentHashMap;
    +
    +import org.apache.cassandra.gms.Gossiper;
    +import org.apache.cassandra.metrics.ExponentialMovingAverage;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +
    +
    +/**
    + * A dynamic snitching implementation that uses Exponentially Decaying Histograms to prefer or
    + * de-prefer hosts
    + *
    + * This was the default implementation prior to Cassandra 4.0 and is being left as the default
    + * in 4.0
    + */
    +public class DynamicEndpointSnitchEMA extends DynamicEndpointSnitch
    +{
    +    // A ~10 sample EMA
    +    private static final double EMA_ALPHA = 0.10;
    +
    +    private final ConcurrentHashMap<InetAddressAndPort, AnnotatedEMA> samples = new ConcurrentHashMap<>();
    +
    +    /**
    +     * Adds two boolean markers to the ExponentialMovingAverage for telling if the data has been
    +     * updated or requested recently.
    +     *
    +     * recentlyMeasured is updated through {@link AnnotatedEMA#update(long, boolean)}
    +     * recentlyRequested is updated through {@link DynamicEndpointSnitch#markRequested}
    +     *
    +     * Both markers are periodically reset via {@link DynamicEndpointSnitch#latencyProbeNeeded(long)}
    +     */
    +    private static class AnnotatedEMA extends ExponentialMovingAverage
    --- End diff --
    
    Ok, makes sense, refactoring :-)


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225298201
  
    --- Diff: doc/source/operating/snitch.rst ---
    @@ -28,16 +28,18 @@ In cassandra, the snitch has two functions:
     
     Dynamic snitching
     ^^^^^^^^^^^^^^^^^
    -
     The dynamic snitch monitor read latencies to avoid reading from hosts that have slowed down. The dynamic snitch is
     configured with the following properties on ``cassandra.yaml``:
     
     - ``dynamic_snitch``: whether the dynamic snitch should be enabled or disabled.
     - ``dynamic_snitch_update_interval_in_ms``: controls how often to perform the more expensive part of host score
       calculation.
    -- ``dynamic_snitch_reset_interval_in_ms``: if set greater than zero, this will allow 'pinning' of replicas to hosts
    -  in order to increase cache capacity.
    -- ``dynamic_snitch_badness_threshold:``: The badness threshold will control how much worse the pinned host has to be
    +- ``dynamic_snitch_latency_probe_interval_in_ms``: controls how often this node
    --- End diff --
    
    Document dynamic snitch class name?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225705215
  
    --- Diff: src/java/org/apache/cassandra/service/StorageServiceMBean.java ---
    @@ -469,15 +469,22 @@
          * If {@code epSnitchClassName} is specified, it will configure a new snitch instance and make it a
          * 'dynamic snitch' if {@code dynamic} is specified and {@code true}.
          *
    -     * The parameters {@code dynamicUpdateInterval}, {@code dynamicResetInterval} and {@code dynamicBadnessThreshold}
    +     * The parameters {@code dynamicUpdateInterval}, {@code dynamicLatencyProbeInterval} and {@code dynamicBadnessThreshold}
          * can be specified individually to update the parameters of the dynamic snitch during runtime.
          *
    -     * @param epSnitchClassName        the canonical path name for a class implementing IEndpointSnitch
    -     * @param dynamic                  boolean that decides whether dynamicsnitch is used or not - only valid, if {@code epSnitchClassName} is specified
    -     * @param dynamicUpdateInterval    integer, in ms (defaults to the value configured in cassandra.yaml, which defaults to 100)
    -     * @param dynamicResetInterval     integer, in ms (defaults to the value configured in cassandra.yaml, which defaults to 600,000)
    -     * @param dynamicBadnessThreshold  double, (defaults to the value configured in cassandra.yaml, which defaults to 0.0)
    +     * @param epSnitchClassName          the canonical path name for a class implementing IEndpointSnitch
    +     * @param dynamicBadnessThreshold    double, (defaults to the value configured in cassandra.yaml, which defaults to 0.0)
    +     * @param dynamic                    boolean that decides whether dynamicsnitch is used or not - only valid, if {@code epSnitchClassName} is specified
    +     * @param dynamicUpdateInterval      integer, in ms (defaults to the value configured in cassandra.yaml, which defaults to 100)
    +     * @param dynamicLatencyProbeInterval integer, in ms (defaults to the value configured in cassandra.yaml, which defaults to 1,000)
          */
    +    public void updateSnitch(String epSnitchClassName, Double dynamicBadnessThreshold, Boolean dynamic, Integer dynamicUpdateInterval, Integer dynamicLatencyProbeInterval) throws ClassNotFoundException;
    --- End diff --
    
    OK that sounds reasonable.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r237956751
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -154,31 +326,203 @@ private void registerMBean()
     
         public void close()
         {
    -        updateSchedular.cancel(false);
    -        resetSchedular.cancel(false);
    +        if (updateScoresScheduler != null)
    +            updateScoresScheduler.cancel(false);
    +        if (updateSamplesScheduler != null)
    +            updateSamplesScheduler.cancel(false);
    +
    +        for (AnnotatedMeasurement measurement : samples.values())
    +        {
    +            if (measurement.probeFuture != null)
    +                measurement.probeFuture.cancel(false);
    +
    +            measurement.millisSinceLastMeasure.set(0);
    +            measurement.millisSinceLastRequest.set(MAX_PROBE_INTERVAL_MS);
    +            measurement.probeTimerMillis = 0;
    +        }
     
             MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
             try
             {
    -            mbs.unregisterMBean(new ObjectName(mbeanName));
    +            if (mbeanRegistered)
    +                mbs.unregisterMBean(new ObjectName(mbeanName));
             }
             catch (Exception e)
             {
                 throw new RuntimeException(e);
             }
         }
     
    +    /**
    +     * Background task running on the samples dictionary. The default implementation sends latency probes (PING)
    +     * messages to explore nodes that we have not received timings for recently but have ranked in
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}.
    +     */
    +    protected void updateSamples()
    +    {
    +        // Split calculation of probe timers from sending probes for testability
    +        calculateProbes(samples, dynamicLatencyProbeInterval);
    +
    +        if (!StorageService.instance.isGossipActive())
    +            return;
    +
    +        schedulePings(samples);
    +    }
    +
    +    /**
    +     * This method mutates the passed AnnotatedMeasurements to implement capped exponential backoff per endpoint.
    +     *
    +     * The algorithm is as follows:
    +     * 1. All samples get their millisSinceLastMeasure and millisSinceLastRequest fields
    +     *    incremented by the passed interval
    +     * 2. Any recently requested (ranked) endpoints that have not been measured recently (e.g. because the snitch
    +     *    has sent them no traffic) get probes with exponential backoff.
    +     *
    +     * The backoff is capped at MAX_PROBE_INTERVAL_MS. Furthermore the probes are stopped after
    +     * MAX_PROBE_INTERVAL_MS of no ranking requests as well.
    +     *
    +     * At the end of this method, any passed AnnotatedMeasurements that need latency probes will have non zero
    +     * probeTimerMillis members set.
    +     */
    +    @VisibleForTesting
    +    static void calculateProbes(Map<InetAddressAndPort, AnnotatedMeasurement> samples, long intervalMillis) {
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry: samples.entrySet())
    +        {
    +            if (entry.getKey().equals(FBUtilities.getBroadcastAddressAndPort()))
    +                continue;
    +
    +            AnnotatedMeasurement measurement = entry.getValue();
    +            long lastMeasure = measurement.millisSinceLastMeasure.getAndAdd(intervalMillis);
    +            long lastRequest = measurement.millisSinceLastRequest.getAndAdd(intervalMillis);
    +
    +            if (lastMeasure >= MIN_PROBE_INTERVAL_MS && lastRequest < MAX_PROBE_INTERVAL_MS)
    +            {
    +                if (measurement.probeTimerMillis == 0)
    +                {
    +                    measurement.probeTimerMillis = intervalMillis;
    +                }
    +                else if (measurement.probeFuture != null && measurement.probeFuture.isDone())
    +                {
    +                    measurement.probeTimerMillis = Math.min(MAX_PROBE_INTERVAL_MS, measurement.probeTimerMillis * 2);
    +                }
    +            }
    +            else
    +            {
    +                measurement.probeTimerMillis = 0;
    +            }
    +        }
    +    }
    +
    +    @VisibleForTesting
    +    void schedulePings(Map<InetAddressAndPort, AnnotatedMeasurement> samples)
    +    {
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry: samples.entrySet())
    +        {
    +            AnnotatedMeasurement measurement = entry.getValue();
    +            long delay = measurement.probeTimerMillis;
    +            long millisSinceLastRequest = measurement.millisSinceLastRequest.get();
    +
    +            if (millisSinceLastRequest > MAX_PROBE_INTERVAL_MS && !Gossiper.instance.isAlive(entry.getKey()))
    +            {
    +                samples.remove(entry.getKey());
    +            }
    +
    +            if (delay > 0 && millisSinceLastRequest < MAX_PROBE_INTERVAL_MS &&
    +                (measurement.probeFuture == null || measurement.probeFuture.isDone()) &&
    +                !updateSamplesScheduler.isCancelled())
    --- End diff --
    
    Ok, I thought that it might be good to check just to be safe but sure I took it out.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225361545
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -154,20 +228,101 @@ private void registerMBean()
     
         public void close()
         {
    -        updateSchedular.cancel(false);
    -        resetSchedular.cancel(false);
    +        if (updateScheduler != null)
    +            updateScheduler.cancel(false);
    +        if (latencyProbeScheduler != null)
    +            latencyProbeScheduler.cancel(false);
     
             MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
             try
             {
    -            mbs.unregisterMBean(new ObjectName(mbeanName));
    +            if (mbeanRegistered)
    +                mbs.unregisterMBean(new ObjectName(mbeanName));
             }
             catch (Exception e)
             {
                 throw new RuntimeException(e);
             }
         }
     
    +    /**
    +     * Determines if latency probes need to be sent, and potentially sends a single latency probe per invocation
    +     */
    +    protected void maybeSendLatencyProbe()
    +    {
    +        long currentTimeNS = System.nanoTime();
    +        markNextAllowedProbeGenerationTime(currentTimeNS);
    +
    +        Optional<InetAddressAndPort> needsProbe = latencyProbeNeeded(currentTimeNS);
    +        needsProbe.ifPresent(this::sendPingMessageToPeer);
    +    }
    +
    +    /**
    +     * Determines which peers need latency at a particular time. Note that this takes currentTimeNS for testability
    +     * of this code path.
    +     * @param currentTimeNS The current time to evaluate. Used mostly for testing.
    +     * @return An Optional that if present contains a host to probe.
    +     */
    +    @VisibleForTesting
    +    Optional<InetAddressAndPort> latencyProbeNeeded(long currentTimeNS) {
    +        if (currentProbePosition >= latencyProbeSequence.size() && (currentTimeNS > nextProbeGenerationTime))
    +        {
    +            nextProbeGenerationTime = nextAllowedProbeGenerationTime;
    +            latencyProbeSequence.clear();
    +
    +            // Delegate to the subclass to actually figure out what the probe sequence should be
    +            updateLatencyProbeSequence(latencyProbeSequence);
    +
    +            if (latencyProbeSequence.size() > 0)
    +                Collections.shuffle(latencyProbeSequence);
    +
    +            currentProbePosition = 0;
    +        }
    +
    +        if (currentProbePosition < latencyProbeSequence.size())
    +        {
    +            try
    +            {
    +                return Optional.of(latencyProbeSequence.get(currentProbePosition++));
    +            }
    +            catch (IndexOutOfBoundsException ignored) {}
    +        }
    +
    +        return Optional.empty();
    +    }
    +
    +    private void sendPingMessageToPeer(InetAddressAndPort to)
    +    {
    +        logger.trace("Sending a small and large PingMessage to {}", to);
    +        IAsyncCallback latencyProbeHandler = new IAsyncCallback()
    --- End diff --
    
    Hm, ok I'm not entirely sure I follow but I will take a whack at it and get back to you!


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r229507783
  
    --- Diff: test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java ---
    @@ -60,57 +130,229 @@ private static EndpointsForRange full(InetAddressAndPort... endpoints)
         }
     
         @Test
    -    public void testSnitch() throws InterruptedException, IOException, ConfigurationException
    +    public void testSortedByProximity() throws InterruptedException, IOException, ConfigurationException
         {
    -        // do this because SS needs to be initialized before DES can work properly.
    -        StorageService.instance.unsafeInitialize();
    -        SimpleSnitch ss = new SimpleSnitch();
    -        DynamicEndpointSnitch dsnitch = new DynamicEndpointSnitch(ss, String.valueOf(ss.hashCode()));
    -        InetAddressAndPort self = FBUtilities.getBroadcastAddressAndPort();
    -        InetAddressAndPort host1 = InetAddressAndPort.getByName("127.0.0.2");
    -        InetAddressAndPort host2 = InetAddressAndPort.getByName("127.0.0.3");
    -        InetAddressAndPort host3 = InetAddressAndPort.getByName("127.0.0.4");
    -        InetAddressAndPort host4 = InetAddressAndPort.getByName("127.0.0.5");
    -        List<InetAddressAndPort> hosts = Arrays.asList(host1, host2, host3);
    +        InetAddressAndPort self = hosts[0];
    +        List<InetAddressAndPort> allHosts = Arrays.asList(hosts[1], hosts[2], hosts[3]);
     
             // first, make all hosts equal
    -        setScores(dsnitch, 1, hosts, 10, 10, 10);
    -        EndpointsForRange order = full(host1, host2, host3);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    -
    -        // make host1 a little worse
    -        setScores(dsnitch, 1, hosts, 20, 10, 10);
    -        order = full(host2, host3, host1);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    -
    -        // make host2 as bad as host1
    -        setScores(dsnitch, 2, hosts, 15, 20, 10);
    -        order = full(host3, host1, host2);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    -
    -        // make host3 the worst
    -        setScores(dsnitch, 3, hosts, 10, 10, 30);
    -        order = full(host1, host2, host3);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    -
    -        // make host3 equal to the others
    -        setScores(dsnitch, 5, hosts, 10, 10, 10);
    -        order = full(host1, host2, host3);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    +        setScores(dsnitch, 1, allHosts, 10, 10, 10);
    +        EndpointsForRange order = full(hosts[1], hosts[2], hosts[3]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        // make hosts[1] a little worse
    +        setScores(dsnitch, 2, allHosts, 20, 10, 10);
    +        order = full(hosts[2], hosts[3], hosts[1]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        // make hosts[2] as bad as hosts[1]
    +        setScores(dsnitch, 4, allHosts, 15, 20, 10);
    +        order = full(hosts[3], hosts[1], hosts[2]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        // make hosts[3] the worst
    +        setScores(dsnitch, 10, allHosts, 10, 10, 30);
    +        order = full(hosts[1], hosts[2], hosts[3]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        // make hosts[3] equal to the others
    +        setScores(dsnitch, 15, allHosts, 10, 10, 10);
    +        order = full(hosts[1], hosts[2], hosts[3]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
     
             /// Tests CASSANDRA-6683 improvements
             // make the scores differ enough from the ideal order that we sort by score; under the old
             // dynamic snitch behavior (where we only compared neighbors), these wouldn't get sorted
    -        setScores(dsnitch, 20, hosts, 10, 70, 20);
    -        order = full(host1, host3, host2);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    +        dsnitch.reset();
    +        setScores(dsnitch, 20, allHosts, 10, 70, 20);
    +        order = full(hosts[1], hosts[3], hosts[2]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        order = full(hosts[4], hosts[1], hosts[3], hosts[2]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3], hosts[4])));
    +
    +        setScores(dsnitch, 40, allHosts, 10, 10, 11);
    +        order = full(hosts[4], hosts[1], hosts[2], hosts[3]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3], hosts[4])));
    +    }
    +
    +    // CASSANDRA-14459 improvements to add latency probes instead of resets
    +    @Test
    +    public void testLatencyProbeNeeded()
    +    {
    +        InetAddressAndPort self = hosts[0];
    +
    +        // Four hosts, two are very latent, but all are requested for ranking
    +        dsnitch.receiveTiming(hosts[1], 20, LatencyMeasurementType.READ);
    +        dsnitch.receiveTiming(hosts[2], 10, LatencyMeasurementType.READ);
    +        dsnitch.receiveTiming(hosts[3], 1000, LatencyMeasurementType.READ);
    +        dsnitch.receiveTiming(hosts[4], 1000, LatencyMeasurementType.READ);
    +        dsnitch.updateScores();
    +
    +        EndpointsForRange orderBefore = full(hosts[2], hosts[1], hosts[3], hosts[4]);
    +
    +        Map<InetAddressAndPort, DynamicEndpointSnitch.AnnotatedMeasurement> measurements = dsnitch.getMeasurementsWithPort();
    +        // At this point we haven't ranked any of hosts so we should need no probes even
    +        DynamicEndpointSnitch.calculateProbes(measurements, dsnitch.dynamicLatencyProbeInterval);
    +        assertFalse(measurements.values().stream().anyMatch(m -> m.probeTimerMillis > 0));
    +        assertTrue(measurements.values().stream().allMatch(m -> m.probeFuture == null));
    +
    +        // Two hosts continue receiving traffic but the last two are always ranked by the snitch
    +        for (int i = 0; i < 10; i++)
    +        {
    +            dsnitch.receiveTiming(hosts[1], 20, LatencyMeasurementType.READ);
    +            dsnitch.receiveTiming(hosts[2], 10, LatencyMeasurementType.READ);
    +            Util.assertRCEquals(orderBefore, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3], hosts[4])));
    +        }
    +        dsnitch.updateScores();
    +
    +        int probesBefore = (int) ((DynamicEndpointSnitch.MIN_PROBE_INTERVAL_MS / dsnitch.dynamicLatencyProbeInterval) + 1);
    +
    +        for (int i = 0; i < probesBefore; i++)
    +            DynamicEndpointSnitch.calculateProbes(measurements, dsnitch.dynamicLatencyProbeInterval);
    +        assertEquals(2, measurements.values().stream().filter(m -> m.probeTimerMillis > 0).count());
    +        assertEquals(0, measurements.values().stream().filter(m -> m.probeFuture != null).count());
    +
    +        // Both requested but non measured hosts should have single interval timers set
    +        assertEquals(dsnitch.dynamicLatencyProbeInterval, measurements.get(hosts[3]).probeTimerMillis);
    +        assertEquals(dsnitch.dynamicLatencyProbeInterval, measurements.get(hosts[4]).probeTimerMillis);
    +
    +        dsnitch.schedulePings(measurements);
    +        // We should have two scheduled futures now
    +        assertEquals(2, measurements.values().stream().filter(m -> m.probeFuture != null).count());
    +
    +        int numProbes = 0;
    +        for (int i = 1; (1 << i) * dsnitch.dynamicLatencyProbeInterval < DynamicEndpointSnitch.MAX_PROBE_INTERVAL_MS; i++)
    +        {
    +            DynamicEndpointSnitch.AnnotatedMeasurement m3 = measurements.get(hosts[3]);
    +            DynamicEndpointSnitch.AnnotatedMeasurement m4 = measurements.get(hosts[4]);
    +            // Emulate the probe actually running
    +            m3.probeFuture.cancel(false);
    +            m4.probeFuture.cancel(false);
    +            DynamicEndpointSnitch.calculateProbes(measurements, dsnitch.dynamicLatencyProbeInterval);
    +            dsnitch.schedulePings(measurements);
    +            assertEquals((1 << i) * dsnitch.dynamicLatencyProbeInterval, m3.probeTimerMillis);
    +            assertEquals((1 << i) * dsnitch.dynamicLatencyProbeInterval, m4.probeTimerMillis);
    +            numProbes++;
    +        }
    +
    +        // If we did exponential backoff right ... we should have the right number of probes
    +        assertTrue(numProbes > 0);
    +        assertEquals(numProbes,
    +                     Math.floor(Math.log(DynamicEndpointSnitch.MAX_PROBE_INTERVAL_MS / dsnitch.dynamicLatencyProbeInterval) / Math.log(2.0)),
    +                     0.01);
    +
    +        measurements.get(hosts[3]).probeFuture.cancel(false);
    +        measurements.get(hosts[4]).probeFuture.cancel(false);
    +
    +        // Now we should do ten minute probes until we exceed MAX_PROBE_INTERVAL_MS since we last requested ranking
    +        int interval = dsnitch.dynamicLatencyProbeInterval;
    +        for (int i = (probesBefore + numProbes) * interval; i < DynamicEndpointSnitch.MAX_PROBE_INTERVAL_MS; i += interval)
    +        {
    +            DynamicEndpointSnitch.calculateProbes(measurements, dsnitch.dynamicLatencyProbeInterval);
    +            assertEquals(DynamicEndpointSnitch.MAX_PROBE_INTERVAL_MS, measurements.get(hosts[3]).probeTimerMillis);
    +            assertEquals(DynamicEndpointSnitch.MAX_PROBE_INTERVAL_MS, measurements.get(hosts[4]).probeTimerMillis);
    +            dsnitch.schedulePings(measurements);
    +            measurements.get(hosts[3]).probeFuture.cancel(false);
    +            measurements.get(hosts[4]).probeFuture.cancel(false);
    +        }
    +
    +        // Since we haven't measured, we should stop probing
    --- End diff --
    
    Haven't requested right? It's true we haven't measured it, but it's not having measured it that makes not probed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225700578
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitchEMA.java ---
    @@ -0,0 +1,181 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.cassandra.locator;
    +
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.concurrent.ConcurrentHashMap;
    +
    +import org.apache.cassandra.gms.Gossiper;
    +import org.apache.cassandra.metrics.ExponentialMovingAverage;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +
    +
    +/**
    + * A dynamic snitching implementation that uses Exponentially Decaying Histograms to prefer or
    + * de-prefer hosts
    + *
    + * This was the default implementation prior to Cassandra 4.0 and is being left as the default
    + * in 4.0
    + */
    +public class DynamicEndpointSnitchEMA extends DynamicEndpointSnitch
    +{
    +    // A ~10 sample EMA
    +    private static final double EMA_ALPHA = 0.10;
    +
    +    private final ConcurrentHashMap<InetAddressAndPort, AnnotatedEMA> samples = new ConcurrentHashMap<>();
    +
    +    /**
    +     * Adds two boolean markers to the ExponentialMovingAverage for telling if the data has been
    +     * updated or requested recently.
    +     *
    +     * recentlyMeasured is updated through {@link AnnotatedEMA#update(long, boolean)}
    +     * recentlyRequested is updated through {@link DynamicEndpointSnitch#markRequested}
    +     *
    +     * Both markers are periodically reset via {@link DynamicEndpointSnitch#latencyProbeNeeded(long)}
    +     */
    +    private static class AnnotatedEMA extends ExponentialMovingAverage
    +    {
    +        volatile boolean recentlyRequested = false;
    +        volatile boolean recentlyMeasured = false;
    +
    +        AnnotatedEMA(double alpha, double initialValue)
    +        {
    +            super(alpha, initialValue);
    +        }
    +
    +        public void update(long value, boolean isRealRead)
    +        {
    +            recentlyMeasured = recentlyMeasured || isRealRead;
    +            super.update(value);
    +        }
    +    }
    +
    +    // Called via reflection
    +    public DynamicEndpointSnitchEMA(IEndpointSnitch snitch)
    +    {
    +        this(snitch, "ema");
    +    }
    +
    +    protected DynamicEndpointSnitchEMA(IEndpointSnitch snitch, String instance)
    +    {
    +        super(snitch, instance);
    +    }
    +
    +    private void receiveTiming(InetAddressAndPort host, long latency, boolean isRealRead) // this is cheap
    +    {
    +        AnnotatedEMA sample = samples.get(host);
    +        if (sample == null)
    +        {
    +            AnnotatedEMA maybeNewSample = new AnnotatedEMA(EMA_ALPHA, latency);
    +            sample = samples.putIfAbsent(host, maybeNewSample);
    +            if (sample == null)
    +                sample = maybeNewSample;
    +        }
    +        sample.update(latency, isRealRead);
    +    }
    +
    +    public void receiveTiming(InetAddressAndPort address, long latency, LatencyMeasurementType measurementType)
    +    {
    +        if (measurementType == LatencyMeasurementType.READ)
    +            receiveTiming(address, latency, true);
    +        else if (measurementType == LatencyMeasurementType.PROBE)
    +            receiveTiming(address, latency, false);
    +    }
    +
    +    @Override
    +    protected void reset() {
    +        this.samples.clear();
    +        super.reset();
    +    }
    +
    +    @Override
    +    public Map<InetAddressAndPort, Double> calculateScores()
    +    {
    +        // We're going to weight the latency for each host against the worst one we see, to
    +        // arrive at sort of a 'badness percentage' for them. First, find the worst for each:
    +        HashMap<InetAddressAndPort, Double> newScores = new HashMap<>(samples.size());
    +        Optional<Double> maxObservedAvgLatency = samples.values().stream()
    +                                                        .map(ExponentialMovingAverage::getAvg)
    +                                                        .max(Double::compare);
    +
    +        final double maxAvgLatency = maxObservedAvgLatency.isPresent() ? maxObservedAvgLatency.get() : 1;
    +
    +        // now make another pass to do the weighting based on the maximums we found before
    +        for (Map.Entry<InetAddressAndPort, AnnotatedEMA> entry : samples.entrySet())
    +        {
    +            // Samples may have changed but rather than creating garbage by copying samples we just ensure
    +            // that all scores are less than 1.0
    +            double addrAvg = entry.getValue().getAvg();
    +            double score = addrAvg / Math.max(addrAvg, maxAvgLatency);
    +            // finally, add the severity without any weighting, since hosts scale this relative to their own load
    +            // and the size of the task causing the severity.
    +            // "Severity" is basically a measure of compaction activity (CASSANDRA-3722).
    +            if (USE_SEVERITY)
    +                score += getSeverity(entry.getKey());
    +            // lowest score (least amount of badness) wins.
    +            newScores.put(entry.getKey(), score);
    +        }
    +        return newScores;
    +    }
    +
    +    @Override
    +    public List<Double> dumpTimings(String hostname) throws UnknownHostException
    +    {
    +        InetAddressAndPort host = InetAddressAndPort.getByName(hostname);
    +        ArrayList<Double> timings = new ArrayList<>();
    +        ExponentialMovingAverage avg = samples.get(host);
    +        if (avg != null)
    +        {
    +            timings.add(avg.getAvg());
    +        }
    +        return timings;
    +    }
    +
    +    @Override
    +    protected void updateLatencyProbeSequence(List<InetAddressAndPort> probeSequence)
    +    {
    +        samples.keySet().retainAll(Gossiper.instance.getLiveMembers());
    +
    +        for (Map.Entry<InetAddressAndPort, AnnotatedEMA> entry: samples.entrySet())
    +        {
    +            // We only send latency probes to nodes that we may plausibly talk to (requested is true)
    +            // but we have not talked to since the last reset of this information (recentlyMeasured is false)
    +            if (entry.getValue().recentlyRequested && !entry.getValue().recentlyMeasured)
    +            {
    +                probeSequence.add(entry.getKey());
    +            }
    +
    +            entry.getValue().recentlyMeasured = false;
    +            entry.getValue().recentlyRequested = false;
    +        }
    +    }
    +
    +    @Override
    +    protected void markRequested(InetAddressAndPort address)
    +    {
    +        AnnotatedEMA ema = samples.get(address);
    +        if (ema != null)
    +            ema.recentlyRequested = true;
    --- End diff --
    
    It's about scalability more than performance. Every time you write to the cache line it has to be acquired in an exclusive state to write and it's invalidated in caches everywhere else. For volatile fields that are set to the same value repeatedly at high frequency it can be better to not do that.
    
    It's more about avoiding death by a thousand cuts WRT to performance. If it were really really not clear I would say don't do it, but it's a simple enough pattern for setting a boolean.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r226462964
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -210,8 +368,10 @@ public String getDatacenter(InetAddressAndPort endpoint)
     
             // TODO: avoid copy
             replicas = subsnitch.sortedByProximity(address, replicas);
    -        HashMap<InetAddressAndPort, Double> scores = this.scores; // Make sure the score don't change in the middle of the loop below
    -                                                           // (which wouldn't really matter here but its cleaner that way).
    +        // Make sure the score don't change in the middle of the loop below
    +        // (which wouldn't really matter here but its cleaner that way).
    +        final Map<InetAddressAndPort, Double> scores = this.scores;
    --- End diff --
    
    Cool, I made them consistent the other way.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r240308095
  
    --- Diff: src/java/org/apache/cassandra/net/ResponseVerbHandler.java ---
    @@ -49,7 +49,7 @@ public void doVerb(MessageIn message, int id)
             else
             {
                 //TODO: Should we add latency only in success cases?
    --- End diff --
    
    Do you think we should remove this TODO while are here?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225298684
  
    --- Diff: src/java/org/apache/cassandra/config/Config.java ---
    @@ -224,8 +224,11 @@
     
         public String endpoint_snitch;
         public boolean dynamic_snitch = true;
    +    public String dynamic_snitch_class_name = "org.apache.cassandra.locator.DynamicEndpointSnitchHistogram";
         public int dynamic_snitch_update_interval_in_ms = 100;
    +    @Deprecated
    --- End diff --
    
    If it's unused you can change it to an Integer and set it to null and then if it's non-null log a warning. You will also need to update NEWS.txt to document what happened with the snitch configuration and active probing. I would wait until it's closer to done to add it.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225704650
  
    --- Diff: src/java/org/apache/cassandra/net/MessagingService.java ---
    @@ -665,9 +665,9 @@ public void updateBackPressureOnSend(InetAddressAndPort host, IAsyncCallback cal
         /**
          * Updates the back-pressure state on reception from the given host if enabled and the given message callback supports it.
          *
    -     * @param host The replica host the back-pressure state refers to.
    +     * @param host     The replica host the back-pressure state refers to.
    --- End diff --
    
    If you go to IntelliJ preferences, search for saved, under Editor -> General -> Strip trailing spaces you can select modified lines from the drop down. Not sure if that impacts this particular reformatting here.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r240308899
  
    --- Diff: src/java/org/apache/cassandra/service/StorageServiceMBean.java ---
    @@ -463,21 +463,33 @@
         public Map<String, String> getViewBuildStatusesWithPort(String keyspace, String view);
     
         /**
    -     * Change endpointsnitch class and dynamic-ness (and dynamic attributes) at runtime.
    +     * Change endpointsnitch class and dynamicsnitch class at runtime.
          *
          * This method is used to change the snitch implementation and/or dynamic snitch parameters.
          * If {@code epSnitchClassName} is specified, it will configure a new snitch instance and make it a
    -     * 'dynamic snitch' if {@code dynamic} is specified and {@code true}.
    +     * 'dynamic snitch' if {@code dynamicSnitchClassName} is not null.
          *
    -     * The parameters {@code dynamicUpdateInterval}, {@code dynamicResetInterval} and {@code dynamicBadnessThreshold}
    +     * The parameters {@code dynamicUpdateInterval}, {@code dynamicSampleUpdateInterval} and {@code dynamicBadnessThreshold}
          * can be specified individually to update the parameters of the dynamic snitch during runtime.
          *
    -     * @param epSnitchClassName        the canonical path name for a class implementing IEndpointSnitch
    -     * @param dynamic                  boolean that decides whether dynamicsnitch is used or not - only valid, if {@code epSnitchClassName} is specified
    -     * @param dynamicUpdateInterval    integer, in ms (defaults to the value configured in cassandra.yaml, which defaults to 100)
    -     * @param dynamicResetInterval     integer, in ms (defaults to the value configured in cassandra.yaml, which defaults to 600,000)
    -     * @param dynamicBadnessThreshold  double, (defaults to the value configured in cassandra.yaml, which defaults to 0.0)
    +     * @param epSnitchClassName            the canonical path name for a class implementing IEndpointSnitch or null.
    +     *                                     If null then no snitch change is made. If an empty string the existing
    +     *                                     Snitch class is used.
    +     * @param dynamicSnitchClassName       the canonical path name for a class extending DynamicEndpointSnitch. If
    +     *                                     null while epSnitchClassName is not null, this turns off dynamic snitching;
    +     *                                     otherwise just settings are updated. If an empty string is passed then
    +     *                                     dynamic snitching is kept with the default implementation.
    +     * @param dynamicUpdateInterval        integer, in ms (defaults to the value configured in cassandra.yaml, which defaults to 100)
    +     * @param dynamicSampleUpdateInterval  integer, in ms (defaults to the value configured in cassandra.yaml, which defaults to 1,000)
    --- End diff --
    
    Documenting defaults in too many locations means the docs are likely to be obsoleted. Such as when you fix the bug with the probe rate.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r229468946
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -154,31 +326,203 @@ private void registerMBean()
     
         public void close()
         {
    -        updateSchedular.cancel(false);
    -        resetSchedular.cancel(false);
    +        if (updateScoresScheduler != null)
    +            updateScoresScheduler.cancel(false);
    +        if (updateSamplesScheduler != null)
    +            updateSamplesScheduler.cancel(false);
    +
    +        for (AnnotatedMeasurement measurement : samples.values())
    +        {
    +            if (measurement.probeFuture != null)
    +                measurement.probeFuture.cancel(false);
    +
    +            measurement.millisSinceLastMeasure.set(0);
    +            measurement.millisSinceLastRequest.set(MAX_PROBE_INTERVAL_MS);
    +            measurement.probeTimerMillis = 0;
    +        }
     
             MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
             try
             {
    -            mbs.unregisterMBean(new ObjectName(mbeanName));
    +            if (mbeanRegistered)
    +                mbs.unregisterMBean(new ObjectName(mbeanName));
             }
             catch (Exception e)
             {
                 throw new RuntimeException(e);
             }
         }
     
    +    /**
    +     * Background task running on the samples dictionary. The default implementation sends latency probes (PING)
    +     * messages to explore nodes that we have not received timings for recently but have ranked in
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}.
    +     */
    +    protected void updateSamples()
    +    {
    +        // Split calculation of probe timers from sending probes for testability
    +        calculateProbes(samples, dynamicLatencyProbeInterval);
    +
    +        if (!StorageService.instance.isGossipActive())
    +            return;
    +
    +        schedulePings(samples);
    --- End diff --
    
    Can we stick to just probes? So name it scheduleProbes?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225347698
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -22,121 +22,194 @@
     import java.net.InetAddress;
     import java.net.UnknownHostException;
     import java.util.*;
    -import java.util.concurrent.ConcurrentHashMap;
     import java.util.concurrent.ScheduledFuture;
     import java.util.concurrent.TimeUnit;
     import java.util.stream.Collectors;
     
    -import com.codahale.metrics.ExponentiallyDecayingReservoir;
     import javax.management.MBeanServer;
     import javax.management.ObjectName;
     
    -import com.codahale.metrics.Snapshot;
    +import com.google.common.annotations.VisibleForTesting;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
     import org.apache.cassandra.concurrent.ScheduledExecutors;
     import org.apache.cassandra.config.DatabaseDescriptor;
     import org.apache.cassandra.gms.ApplicationState;
     import org.apache.cassandra.gms.EndpointState;
     import org.apache.cassandra.gms.Gossiper;
     import org.apache.cassandra.gms.VersionedValue;
    +import org.apache.cassandra.net.IAsyncCallback;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +import org.apache.cassandra.net.MessageIn;
    +import org.apache.cassandra.net.MessageOut;
     import org.apache.cassandra.net.MessagingService;
    +import org.apache.cassandra.net.PingMessage;
     import org.apache.cassandra.service.StorageService;
     import org.apache.cassandra.utils.FBUtilities;
     
    +import static org.apache.cassandra.net.MessagingService.Verb.PING;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.LARGE_MESSAGE;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.SMALL_MESSAGE;
    +
    +
     /**
      * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector
    + * Note that the subclasses (e.g. {@link DynamicEndpointSnitchHistogram}) are responsible for actually measuring
    + * latency and populating the {@link #scores} map.
      */
    -public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
    +public abstract class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
     {
    -    private static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    -
    -    private static final double ALPHA = 0.75; // set to 0.75 to make EDS more biased to towards the newer values
    -    private static final int WINDOW_SIZE = 100;
    -
    -    private volatile int dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    -    private volatile int dynamicResetInterval = DatabaseDescriptor.getDynamicResetInterval();
    -    private volatile double dynamicBadnessThreshold = DatabaseDescriptor.getDynamicBadnessThreshold();
    +    private static final Logger logger = LoggerFactory.getLogger(DynamicEndpointSnitch.class);
    +
    +    // Subclass specific functionality
    +    protected static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    protected boolean registered = false;
    +    // The scores map is updated via copy in updateScores
    +    // We keep it in the base class for performance reasons (so it can be easily aliased)
    +    protected volatile Map<InetAddressAndPort, Double> scores = new HashMap<>();
    +
    +    // Rate limit how often we generate latency probes
    +    protected long nextAllowedProbeGenerationTime;
    --- End diff --
    
    I dig backoff, implementing.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225347277
  
    --- Diff: src/java/org/apache/cassandra/config/Config.java ---
    @@ -224,8 +224,11 @@
     
         public String endpoint_snitch;
         public boolean dynamic_snitch = true;
    +    public String dynamic_snitch_class_name = "org.apache.cassandra.locator.DynamicEndpointSnitchHistogram";
         public int dynamic_snitch_update_interval_in_ms = 100;
    +    @Deprecated
    --- End diff --
    
    ack, will do


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r237978698
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -154,31 +326,203 @@ private void registerMBean()
     
         public void close()
         {
    -        updateSchedular.cancel(false);
    -        resetSchedular.cancel(false);
    +        if (updateScoresScheduler != null)
    +            updateScoresScheduler.cancel(false);
    +        if (updateSamplesScheduler != null)
    +            updateSamplesScheduler.cancel(false);
    +
    +        for (AnnotatedMeasurement measurement : samples.values())
    +        {
    +            if (measurement.probeFuture != null)
    +                measurement.probeFuture.cancel(false);
    +
    +            measurement.millisSinceLastMeasure.set(0);
    +            measurement.millisSinceLastRequest.set(MAX_PROBE_INTERVAL_MS);
    +            measurement.probeTimerMillis = 0;
    +        }
     
             MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
             try
             {
    -            mbs.unregisterMBean(new ObjectName(mbeanName));
    +            if (mbeanRegistered)
    +                mbs.unregisterMBean(new ObjectName(mbeanName));
             }
             catch (Exception e)
             {
                 throw new RuntimeException(e);
             }
         }
     
    +    /**
    +     * Background task running on the samples dictionary. The default implementation sends latency probes (PING)
    +     * messages to explore nodes that we have not received timings for recently but have ranked in
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}.
    +     */
    +    protected void updateSamples()
    +    {
    +        // Split calculation of probe timers from sending probes for testability
    +        calculateProbes(samples, dynamicLatencyProbeInterval);
    +
    +        if (!StorageService.instance.isGossipActive())
    +            return;
    +
    +        schedulePings(samples);
    +    }
    +
    +    /**
    +     * This method mutates the passed AnnotatedMeasurements to implement capped exponential backoff per endpoint.
    +     *
    +     * The algorithm is as follows:
    +     * 1. All samples get their millisSinceLastMeasure and millisSinceLastRequest fields
    +     *    incremented by the passed interval
    +     * 2. Any recently requested (ranked) endpoints that have not been measured recently (e.g. because the snitch
    +     *    has sent them no traffic) get probes with exponential backoff.
    +     *
    +     * The backoff is capped at MAX_PROBE_INTERVAL_MS. Furthermore the probes are stopped after
    +     * MAX_PROBE_INTERVAL_MS of no ranking requests as well.
    +     *
    +     * At the end of this method, any passed AnnotatedMeasurements that need latency probes will have non zero
    +     * probeTimerMillis members set.
    +     */
    +    @VisibleForTesting
    +    static void calculateProbes(Map<InetAddressAndPort, AnnotatedMeasurement> samples, long intervalMillis) {
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry: samples.entrySet())
    +        {
    +            if (entry.getKey().equals(FBUtilities.getBroadcastAddressAndPort()))
    +                continue;
    +
    +            AnnotatedMeasurement measurement = entry.getValue();
    +            long lastMeasure = measurement.millisSinceLastMeasure.getAndAdd(intervalMillis);
    +            long lastRequest = measurement.millisSinceLastRequest.getAndAdd(intervalMillis);
    +
    +            if (lastMeasure >= MIN_PROBE_INTERVAL_MS && lastRequest < MAX_PROBE_INTERVAL_MS)
    +            {
    +                if (measurement.probeTimerMillis == 0)
    +                {
    +                    measurement.probeTimerMillis = intervalMillis;
    +                }
    +                else if (measurement.probeFuture != null && measurement.probeFuture.isDone())
    +                {
    +                    measurement.probeTimerMillis = Math.min(MAX_PROBE_INTERVAL_MS, measurement.probeTimerMillis * 2);
    +                }
    +            }
    +            else
    +            {
    +                measurement.probeTimerMillis = 0;
    +            }
    +        }
    +    }
    +
    +    @VisibleForTesting
    +    void schedulePings(Map<InetAddressAndPort, AnnotatedMeasurement> samples)
    +    {
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry: samples.entrySet())
    +        {
    +            AnnotatedMeasurement measurement = entry.getValue();
    +            long delay = measurement.probeTimerMillis;
    +            long millisSinceLastRequest = measurement.millisSinceLastRequest.get();
    +
    +            if (millisSinceLastRequest > MAX_PROBE_INTERVAL_MS && !Gossiper.instance.isAlive(entry.getKey()))
    +            {
    +                samples.remove(entry.getKey());
    +            }
    +
    +            if (delay > 0 && millisSinceLastRequest < MAX_PROBE_INTERVAL_MS &&
    --- End diff --
    
    Yea I don't think this check is needed anymore since I separated out the two pieces, removed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r226745092
  
    --- Diff: src/java/org/apache/cassandra/locator/dynamicsnitch/DynamicEndpointSnitchEMA.java ---
    @@ -0,0 +1,133 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.cassandra.locator.dynamicsnitch;
    +
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.Optional;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +
    +import org.apache.cassandra.locator.DynamicEndpointSnitch;
    +import org.apache.cassandra.locator.IEndpointSnitch;
    +import org.apache.cassandra.locator.InetAddressAndPort;
    +import org.apache.cassandra.metrics.ExponentialMovingAverage;
    +
    +
    +/**
    + * A dynamic snitching implementation that uses Exponentially Moving Averages as a low pass filter to prefer
    + * or de-prefer hosts
    + *
    + * This implementation generates a few orders of magnitude less garbage than histograms and is close to 10x faster,
    + * but as it is not a Median LPF (it is an Average LPF), it is more vulnerable to noise. This may be acceptable but
    + * given the significant change in behavior this is not the default in 4.0
    + */
    +public class DynamicEndpointSnitchEMA extends DynamicEndpointSnitch
    +{
    +    // A ~10 sample EMA heavily weighted to the past values to minimize noise
    +    private static final double EMA_ALPHA = 0.10;
    +
    +    protected static class EMASnitchMeasurement implements ISnitchMeasurement
    +    {
    +        public final ExponentialMovingAverage avg;
    +
    +        EMASnitchMeasurement(double initial)
    +        {
    +            avg = new ExponentialMovingAverage(EMA_ALPHA, initial);
    +        }
    +
    +        @Override
    +        public void sample(long value)
    +        {
    +            avg.update(value);
    +        }
    +
    +        @Override
    +        public double measure()
    +        {
    +            return avg.getAvg();
    +        }
    +
    +        @Override
    +        public Iterable<Double> measurements()
    +        {
    +            return Collections.singletonList(avg.getAvg());
    +        }
    +    }
    +
    +    // Called via reflection
    +    public DynamicEndpointSnitchEMA(IEndpointSnitch snitch)
    +    {
    +        this(snitch, "ema");
    +    }
    +
    +    public DynamicEndpointSnitchEMA(IEndpointSnitch snitch, String instance)
    +    {
    +        super(snitch, instance);
    +    }
    +
    +    @Override
    +    protected ISnitchMeasurement measurementImpl(long initialValue)
    +    {
    +        return new EMASnitchMeasurement(initialValue);
    +    }
    +
    +    /**
    +     * Unlike the Histogram implementation, calling this measure method is reasonably cheap (doesn't require a
    +     * Snapshot or anything) so we can skip a round of iterations and just normalize the scores slightly
    +     * differently
    +     */
    +    @Override
    +    public Map<InetAddressAndPort, Double> calculateScores()
    +    {
    +        // We're going to weight the latency for each host against the worst one we see, to
    +        // arrive at sort of a 'badness percentage' for them. First, find the worst for each:
    +        HashMap<InetAddressAndPort, Double> newScores = new HashMap<>(samples.size());
    +        Optional<Double> maxObservedAvgLatency = samples.values().stream()
    --- End diff --
    
    This iterates the measurements twice while they are changing and that can result in the maximum no longer being correct so they won't be scaled to 1.0. This means that severity is no longer relative to 1.0.
    
    Not a huge deal but it's also a lot of code duplication to avoid allocating the extra map.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r236764154
  
    --- Diff: test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java ---
    @@ -60,57 +130,229 @@ private static EndpointsForRange full(InetAddressAndPort... endpoints)
         }
     
         @Test
    -    public void testSnitch() throws InterruptedException, IOException, ConfigurationException
    +    public void testSortedByProximity() throws InterruptedException, IOException, ConfigurationException
         {
    -        // do this because SS needs to be initialized before DES can work properly.
    -        StorageService.instance.unsafeInitialize();
    -        SimpleSnitch ss = new SimpleSnitch();
    -        DynamicEndpointSnitch dsnitch = new DynamicEndpointSnitch(ss, String.valueOf(ss.hashCode()));
    -        InetAddressAndPort self = FBUtilities.getBroadcastAddressAndPort();
    -        InetAddressAndPort host1 = InetAddressAndPort.getByName("127.0.0.2");
    -        InetAddressAndPort host2 = InetAddressAndPort.getByName("127.0.0.3");
    -        InetAddressAndPort host3 = InetAddressAndPort.getByName("127.0.0.4");
    -        InetAddressAndPort host4 = InetAddressAndPort.getByName("127.0.0.5");
    -        List<InetAddressAndPort> hosts = Arrays.asList(host1, host2, host3);
    +        InetAddressAndPort self = hosts[0];
    +        List<InetAddressAndPort> allHosts = Arrays.asList(hosts[1], hosts[2], hosts[3]);
     
             // first, make all hosts equal
    -        setScores(dsnitch, 1, hosts, 10, 10, 10);
    -        EndpointsForRange order = full(host1, host2, host3);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    -
    -        // make host1 a little worse
    -        setScores(dsnitch, 1, hosts, 20, 10, 10);
    -        order = full(host2, host3, host1);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    -
    -        // make host2 as bad as host1
    -        setScores(dsnitch, 2, hosts, 15, 20, 10);
    -        order = full(host3, host1, host2);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    -
    -        // make host3 the worst
    -        setScores(dsnitch, 3, hosts, 10, 10, 30);
    -        order = full(host1, host2, host3);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    -
    -        // make host3 equal to the others
    -        setScores(dsnitch, 5, hosts, 10, 10, 10);
    -        order = full(host1, host2, host3);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    +        setScores(dsnitch, 1, allHosts, 10, 10, 10);
    +        EndpointsForRange order = full(hosts[1], hosts[2], hosts[3]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        // make hosts[1] a little worse
    +        setScores(dsnitch, 2, allHosts, 20, 10, 10);
    +        order = full(hosts[2], hosts[3], hosts[1]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        // make hosts[2] as bad as hosts[1]
    +        setScores(dsnitch, 4, allHosts, 15, 20, 10);
    +        order = full(hosts[3], hosts[1], hosts[2]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        // make hosts[3] the worst
    +        setScores(dsnitch, 10, allHosts, 10, 10, 30);
    +        order = full(hosts[1], hosts[2], hosts[3]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        // make hosts[3] equal to the others
    +        setScores(dsnitch, 15, allHosts, 10, 10, 10);
    +        order = full(hosts[1], hosts[2], hosts[3]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
     
             /// Tests CASSANDRA-6683 improvements
             // make the scores differ enough from the ideal order that we sort by score; under the old
             // dynamic snitch behavior (where we only compared neighbors), these wouldn't get sorted
    -        setScores(dsnitch, 20, hosts, 10, 70, 20);
    -        order = full(host1, host3, host2);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    +        dsnitch.reset();
    +        setScores(dsnitch, 20, allHosts, 10, 70, 20);
    +        order = full(hosts[1], hosts[3], hosts[2]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        order = full(hosts[4], hosts[1], hosts[3], hosts[2]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3], hosts[4])));
    +
    +        setScores(dsnitch, 40, allHosts, 10, 10, 11);
    +        order = full(hosts[4], hosts[1], hosts[2], hosts[3]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3], hosts[4])));
    +    }
    +
    +    // CASSANDRA-14459 improvements to add latency probes instead of resets
    +    @Test
    +    public void testLatencyProbeNeeded()
    +    {
    +        InetAddressAndPort self = hosts[0];
    +
    +        // Four hosts, two are very latent, but all are requested for ranking
    +        dsnitch.receiveTiming(hosts[1], 20, LatencyMeasurementType.READ);
    +        dsnitch.receiveTiming(hosts[2], 10, LatencyMeasurementType.READ);
    +        dsnitch.receiveTiming(hosts[3], 1000, LatencyMeasurementType.READ);
    +        dsnitch.receiveTiming(hosts[4], 1000, LatencyMeasurementType.READ);
    +        dsnitch.updateScores();
    +
    +        EndpointsForRange orderBefore = full(hosts[2], hosts[1], hosts[3], hosts[4]);
    +
    +        Map<InetAddressAndPort, DynamicEndpointSnitch.AnnotatedMeasurement> measurements = dsnitch.getMeasurementsWithPort();
    +        // At this point we haven't ranked any of hosts so we should need no probes even
    --- End diff --
    
    re-worded to make it clearer.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r226462782
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -188,6 +343,9 @@ public String getDatacenter(InetAddressAndPort endpoint)
         public <C extends ReplicaCollection<? extends C>> C sortedByProximity(final InetAddressAndPort address, C unsortedAddresses)
         {
             assert address.equals(FBUtilities.getBroadcastAddressAndPort()); // we only know about ourself
    +        for (Replica unsortedAddress : unsortedAddresses)
    --- End diff --
    
    I don't think that combining the two maps is a good idea and this loop seems pretty cheap to me whether we do it here first or add a statement to a later loop that reads the score map. Due to the way that we alias scores to and do full copy on write I'm also worried about correctness given the number of comments explaining why that scores aliasing is so important I don't want to risk changing that in this patch.
    
    I do agree a through examination of the scores datastructure and way that we update it may be warranted in the future (e.g. we could create some kind of immutable view just for the endpoints we're ranking, although I don't know a way to do that without adding lots of extra garbage).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r237979844
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -154,31 +326,203 @@ private void registerMBean()
     
         public void close()
         {
    -        updateSchedular.cancel(false);
    -        resetSchedular.cancel(false);
    +        if (updateScoresScheduler != null)
    +            updateScoresScheduler.cancel(false);
    +        if (updateSamplesScheduler != null)
    +            updateSamplesScheduler.cancel(false);
    +
    +        for (AnnotatedMeasurement measurement : samples.values())
    +        {
    +            if (measurement.probeFuture != null)
    +                measurement.probeFuture.cancel(false);
    +
    +            measurement.millisSinceLastMeasure.set(0);
    +            measurement.millisSinceLastRequest.set(MAX_PROBE_INTERVAL_MS);
    +            measurement.probeTimerMillis = 0;
    +        }
     
             MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
             try
             {
    -            mbs.unregisterMBean(new ObjectName(mbeanName));
    +            if (mbeanRegistered)
    +                mbs.unregisterMBean(new ObjectName(mbeanName));
             }
             catch (Exception e)
             {
                 throw new RuntimeException(e);
             }
         }
     
    +    /**
    +     * Background task running on the samples dictionary. The default implementation sends latency probes (PING)
    +     * messages to explore nodes that we have not received timings for recently but have ranked in
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}.
    +     */
    +    protected void updateSamples()
    +    {
    +        // Split calculation of probe timers from sending probes for testability
    +        calculateProbes(samples, dynamicLatencyProbeInterval);
    +
    +        if (!StorageService.instance.isGossipActive())
    +            return;
    +
    +        schedulePings(samples);
    +    }
    +
    +    /**
    +     * This method mutates the passed AnnotatedMeasurements to implement capped exponential backoff per endpoint.
    +     *
    +     * The algorithm is as follows:
    +     * 1. All samples get their millisSinceLastMeasure and millisSinceLastRequest fields
    +     *    incremented by the passed interval
    +     * 2. Any recently requested (ranked) endpoints that have not been measured recently (e.g. because the snitch
    +     *    has sent them no traffic) get probes with exponential backoff.
    +     *
    +     * The backoff is capped at MAX_PROBE_INTERVAL_MS. Furthermore the probes are stopped after
    +     * MAX_PROBE_INTERVAL_MS of no ranking requests as well.
    +     *
    +     * At the end of this method, any passed AnnotatedMeasurements that need latency probes will have non zero
    +     * probeTimerMillis members set.
    +     */
    +    @VisibleForTesting
    +    static void calculateProbes(Map<InetAddressAndPort, AnnotatedMeasurement> samples, long intervalMillis) {
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry: samples.entrySet())
    +        {
    +            if (entry.getKey().equals(FBUtilities.getBroadcastAddressAndPort()))
    +                continue;
    +
    +            AnnotatedMeasurement measurement = entry.getValue();
    +            long lastMeasure = measurement.millisSinceLastMeasure.getAndAdd(intervalMillis);
    +            long lastRequest = measurement.millisSinceLastRequest.getAndAdd(intervalMillis);
    +
    +            if (lastMeasure >= MIN_PROBE_INTERVAL_MS && lastRequest < MAX_PROBE_INTERVAL_MS)
    +            {
    +                if (measurement.probeTimerMillis == 0)
    +                {
    +                    measurement.probeTimerMillis = intervalMillis;
    --- End diff --
    
    It might not be null if we had scheduled a 10 minute probe and then got a real reading.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r237961057
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -154,31 +326,203 @@ private void registerMBean()
     
         public void close()
         {
    -        updateSchedular.cancel(false);
    -        resetSchedular.cancel(false);
    +        if (updateScoresScheduler != null)
    +            updateScoresScheduler.cancel(false);
    +        if (updateSamplesScheduler != null)
    +            updateSamplesScheduler.cancel(false);
    +
    +        for (AnnotatedMeasurement measurement : samples.values())
    +        {
    +            if (measurement.probeFuture != null)
    +                measurement.probeFuture.cancel(false);
    +
    +            measurement.millisSinceLastMeasure.set(0);
    +            measurement.millisSinceLastRequest.set(MAX_PROBE_INTERVAL_MS);
    +            measurement.probeTimerMillis = 0;
    +        }
     
             MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
             try
             {
    -            mbs.unregisterMBean(new ObjectName(mbeanName));
    +            if (mbeanRegistered)
    +                mbs.unregisterMBean(new ObjectName(mbeanName));
             }
             catch (Exception e)
             {
                 throw new RuntimeException(e);
             }
         }
     
    +    /**
    +     * Background task running on the samples dictionary. The default implementation sends latency probes (PING)
    +     * messages to explore nodes that we have not received timings for recently but have ranked in
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}.
    +     */
    +    protected void updateSamples()
    +    {
    +        // Split calculation of probe timers from sending probes for testability
    +        calculateProbes(samples, dynamicLatencyProbeInterval);
    +
    +        if (!StorageService.instance.isGossipActive())
    +            return;
    +
    +        schedulePings(samples);
    +    }
    +
    +    /**
    +     * This method mutates the passed AnnotatedMeasurements to implement capped exponential backoff per endpoint.
    +     *
    +     * The algorithm is as follows:
    +     * 1. All samples get their millisSinceLastMeasure and millisSinceLastRequest fields
    +     *    incremented by the passed interval
    +     * 2. Any recently requested (ranked) endpoints that have not been measured recently (e.g. because the snitch
    +     *    has sent them no traffic) get probes with exponential backoff.
    +     *
    +     * The backoff is capped at MAX_PROBE_INTERVAL_MS. Furthermore the probes are stopped after
    +     * MAX_PROBE_INTERVAL_MS of no ranking requests as well.
    +     *
    +     * At the end of this method, any passed AnnotatedMeasurements that need latency probes will have non zero
    +     * probeTimerMillis members set.
    +     */
    +    @VisibleForTesting
    +    static void calculateProbes(Map<InetAddressAndPort, AnnotatedMeasurement> samples, long intervalMillis) {
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry: samples.entrySet())
    +        {
    +            if (entry.getKey().equals(FBUtilities.getBroadcastAddressAndPort()))
    +                continue;
    +
    +            AnnotatedMeasurement measurement = entry.getValue();
    +            long lastMeasure = measurement.millisSinceLastMeasure.getAndAdd(intervalMillis);
    +            long lastRequest = measurement.millisSinceLastRequest.getAndAdd(intervalMillis);
    +
    +            if (lastMeasure >= MIN_PROBE_INTERVAL_MS && lastRequest < MAX_PROBE_INTERVAL_MS)
    +            {
    +                if (measurement.probeTimerMillis == 0)
    +                {
    +                    measurement.probeTimerMillis = intervalMillis;
    +                }
    +                else if (measurement.probeFuture != null && measurement.probeFuture.isDone())
    +                {
    +                    measurement.probeTimerMillis = Math.min(MAX_PROBE_INTERVAL_MS, measurement.probeTimerMillis * 2);
    +                }
    +            }
    +            else
    +            {
    +                measurement.probeTimerMillis = 0;
    +            }
    +        }
    +    }
    +
    +    @VisibleForTesting
    +    void schedulePings(Map<InetAddressAndPort, AnnotatedMeasurement> samples)
    +    {
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry: samples.entrySet())
    +        {
    +            AnnotatedMeasurement measurement = entry.getValue();
    +            long delay = measurement.probeTimerMillis;
    +            long millisSinceLastRequest = measurement.millisSinceLastRequest.get();
    +
    +            if (millisSinceLastRequest > MAX_PROBE_INTERVAL_MS && !Gossiper.instance.isAlive(entry.getKey()))
    +            {
    +                samples.remove(entry.getKey());
    +            }
    +
    +            if (delay > 0 && millisSinceLastRequest < MAX_PROBE_INTERVAL_MS &&
    +                (measurement.probeFuture == null || measurement.probeFuture.isDone()) &&
    +                !updateSamplesScheduler.isCancelled())
    +            {
    +                logger.trace("Scheduled latency probe against {} in {}ms", entry.getKey(), delay);
    +                measurement.probeFuture = latencyProbeExecutor.schedule(() -> sendPingMessageToPeer(entry.getKey()),
    +                                                                        delay, TimeUnit.MILLISECONDS);
    +            }
    +        }
    +    }
    +
    +    /**
    +     * Method that actually sends latency probes as PING messages. This is the only function in this class
    +     * that operates on the latencyProbeExecutor thread and it records the maximum latency between a small and large
    +     * message channel ping.
    +     */
    +    private void sendPingMessageToPeer(InetAddressAndPort to)
    +    {
    +        probeRateLimiter.acquire(dynamicLatencyProbeInterval);
    +
    +        if (!StorageService.instance.isGossipActive())
    --- End diff --
    
    Because this thread can run up to 10 minutes after it was scheduled, we maybe shouldn't be talking to other nodes at this junction (i.e. gossip is turned off). I know it's racey in the sense that we can check, then gossip can be turned off, then we can send a ping; but realistically users are going to be having issues where they're trying to drain the node and then still get messages 10 minutes later out of this.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r227617598
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -22,121 +22,194 @@
     import java.net.InetAddress;
     import java.net.UnknownHostException;
     import java.util.*;
    -import java.util.concurrent.ConcurrentHashMap;
     import java.util.concurrent.ScheduledFuture;
     import java.util.concurrent.TimeUnit;
     import java.util.stream.Collectors;
     
    -import com.codahale.metrics.ExponentiallyDecayingReservoir;
     import javax.management.MBeanServer;
     import javax.management.ObjectName;
     
    -import com.codahale.metrics.Snapshot;
    +import com.google.common.annotations.VisibleForTesting;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
     import org.apache.cassandra.concurrent.ScheduledExecutors;
     import org.apache.cassandra.config.DatabaseDescriptor;
     import org.apache.cassandra.gms.ApplicationState;
     import org.apache.cassandra.gms.EndpointState;
     import org.apache.cassandra.gms.Gossiper;
     import org.apache.cassandra.gms.VersionedValue;
    +import org.apache.cassandra.net.IAsyncCallback;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +import org.apache.cassandra.net.MessageIn;
    +import org.apache.cassandra.net.MessageOut;
     import org.apache.cassandra.net.MessagingService;
    +import org.apache.cassandra.net.PingMessage;
     import org.apache.cassandra.service.StorageService;
     import org.apache.cassandra.utils.FBUtilities;
     
    +import static org.apache.cassandra.net.MessagingService.Verb.PING;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.LARGE_MESSAGE;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.SMALL_MESSAGE;
    +
    +
     /**
      * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector
    + * Note that the subclasses (e.g. {@link DynamicEndpointSnitchHistogram}) are responsible for actually measuring
    + * latency and populating the {@link #scores} map.
      */
    -public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
    +public abstract class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
     {
    -    private static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    -
    -    private static final double ALPHA = 0.75; // set to 0.75 to make EDS more biased to towards the newer values
    -    private static final int WINDOW_SIZE = 100;
    -
    -    private volatile int dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    -    private volatile int dynamicResetInterval = DatabaseDescriptor.getDynamicResetInterval();
    -    private volatile double dynamicBadnessThreshold = DatabaseDescriptor.getDynamicBadnessThreshold();
    +    private static final Logger logger = LoggerFactory.getLogger(DynamicEndpointSnitch.class);
    +
    +    // Subclass specific functionality
    +    protected static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    protected boolean registered = false;
    +    // The scores map is updated via copy in updateScores
    +    // We keep it in the base class for performance reasons (so it can be easily aliased)
    +    protected volatile Map<InetAddressAndPort, Double> scores = new HashMap<>();
    +
    +    // Rate limit how often we generate latency probes
    +    protected long nextAllowedProbeGenerationTime;
    --- End diff --
    
    Alright, exponential backoff implemented and indeed it reduces the messaging significantly. The current patch only starts probing nodes that have been requested in the last ten minutes, but have not had latency measurements in the last one minute. These probes exponentially backoff until ten minutes and then they either keep going at that pace if the host is still being requested, or stop if the host is no longer being requested.
    
    I've left a bunch of comments in the code to explain the algo if it's unclear.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r229457395
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -23,128 +23,300 @@
     import java.net.UnknownHostException;
     import java.util.*;
     import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ExecutionException;
     import java.util.concurrent.ScheduledFuture;
     import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
     import java.util.stream.Collectors;
     
    -import com.codahale.metrics.ExponentiallyDecayingReservoir;
     import javax.management.MBeanServer;
     import javax.management.ObjectName;
     
    -import com.codahale.metrics.Snapshot;
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.util.concurrent.Futures;
    +import com.google.common.util.concurrent.MoreExecutors;
    +import com.google.common.util.concurrent.RateLimiter;
    +import com.google.common.util.concurrent.SettableFuture;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
     import org.apache.cassandra.concurrent.ScheduledExecutors;
     import org.apache.cassandra.config.DatabaseDescriptor;
    +import org.apache.cassandra.exceptions.RequestFailureReason;
     import org.apache.cassandra.gms.ApplicationState;
     import org.apache.cassandra.gms.EndpointState;
     import org.apache.cassandra.gms.Gossiper;
     import org.apache.cassandra.gms.VersionedValue;
    +import org.apache.cassandra.locator.dynamicsnitch.DynamicEndpointSnitchHistogram;
    +import org.apache.cassandra.net.IAsyncCallbackWithFailure;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +import org.apache.cassandra.net.MessageIn;
    +import org.apache.cassandra.net.MessageOut;
     import org.apache.cassandra.net.MessagingService;
    +import org.apache.cassandra.net.PingMessage;
     import org.apache.cassandra.service.StorageService;
     import org.apache.cassandra.utils.FBUtilities;
     
    +import static org.apache.cassandra.net.MessagingService.Verb.PING;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.LARGE_MESSAGE;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.SMALL_MESSAGE;
    +
    +
     /**
      * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector
    + * Note that the subclasses (e.g. {@link DynamicEndpointSnitchHistogram}) are responsible for actually measuring
    + * latency and providing an ISnitchMeasurement implementation back to this class.
      */
    -public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
    +public abstract class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
     {
    -    private static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    -
    -    private static final double ALPHA = 0.75; // set to 0.75 to make EDS more biased to towards the newer values
    -    private static final int WINDOW_SIZE = 100;
    -
    -    private volatile int dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    -    private volatile int dynamicResetInterval = DatabaseDescriptor.getDynamicResetInterval();
    -    private volatile double dynamicBadnessThreshold = DatabaseDescriptor.getDynamicBadnessThreshold();
    +    private static final Logger logger = LoggerFactory.getLogger(DynamicEndpointSnitch.class);
    +
    +    // Latency measurement and ranking. The samples contain latency measurements and the scores are used for ranking
    +    protected boolean registered = false;
    +    protected static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    protected volatile Map<InetAddressAndPort, Double> scores = new HashMap<>();
    +    protected final Map<InetAddressAndPort, AnnotatedMeasurement> samples = new ConcurrentHashMap<>();
    +
    +    // Latency probe functionality for actively probing endpoints that we haven't measured recently but are ranking
    +    public static final long MAX_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_max_probe_interval_ms", 60 * 10 * 1000L);
    +    public static final long MIN_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_min_probe_interval_ms", 60 * 1000L) ;
    +    // The probe rate is set later when configuration is read
    +    protected static final RateLimiter probeRateLimiter = RateLimiter.create(1);
    +    protected static final DebuggableScheduledThreadPoolExecutor latencyProbeExecutor = new DebuggableScheduledThreadPoolExecutor(1, "LatencyProbes", Thread.MIN_PRIORITY);
    +
    +    // User configuration of the snitch tunables
    +    protected volatile int dynamicUpdateInterval = -1;
    +    protected volatile int dynamicLatencyProbeInterval = -1;
    +    protected volatile double dynamicBadnessThreshold = 0;
     
         // the score for a merged set of endpoints must be this much worse than the score for separate endpoints to
         // warrant not merging two ranges into a single range
         private static final double RANGE_MERGING_PREFERENCE = 1.5;
     
         private String mbeanName;
    -    private boolean registered = false;
    -
    -    private volatile HashMap<InetAddressAndPort, Double> scores = new HashMap<>();
    -    private final ConcurrentHashMap<InetAddressAndPort, ExponentiallyDecayingReservoir> samples = new ConcurrentHashMap<>();
    +    private boolean mbeanRegistered = false;
     
         public final IEndpointSnitch subsnitch;
     
    -    private volatile ScheduledFuture<?> updateSchedular;
    -    private volatile ScheduledFuture<?> resetSchedular;
    -
    -    private final Runnable update;
    -    private final Runnable reset;
    +    private volatile ScheduledFuture<?> updateScoresScheduler;
    +    private volatile ScheduledFuture<?> updateSamplesScheduler;
     
         public DynamicEndpointSnitch(IEndpointSnitch snitch)
         {
             this(snitch, null);
         }
     
    -    public DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
    +    protected DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
         {
             mbeanName = "org.apache.cassandra.db:type=DynamicEndpointSnitch";
             if (instance != null)
                 mbeanName += ",instance=" + instance;
             subsnitch = snitch;
    -        update = new Runnable()
    +
    +        if (DatabaseDescriptor.isDaemonInitialized())
             {
    -            public void run()
    -            {
    -                updateScores();
    -            }
    -        };
    -        reset = new Runnable()
    +            applyConfigChanges(DatabaseDescriptor.getDynamicUpdateInterval(),
    +                               DatabaseDescriptor.getDynamicSampleUpdateInterval(),
    --- End diff --
    
    This config option doesn't look like it hooks up to the rate limiter the way you intended?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225324418
  
    --- Diff: src/java/org/apache/cassandra/gms/Gossiper.java ---
    @@ -38,6 +38,7 @@
     import com.google.common.util.concurrent.Uninterruptibles;
     
     import org.apache.cassandra.locator.InetAddressAndPort;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    --- End diff --
    
    Unused import


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225338651
  
    --- Diff: src/java/org/apache/cassandra/service/StorageServiceMBean.java ---
    @@ -469,15 +469,22 @@
          * If {@code epSnitchClassName} is specified, it will configure a new snitch instance and make it a
          * 'dynamic snitch' if {@code dynamic} is specified and {@code true}.
          *
    -     * The parameters {@code dynamicUpdateInterval}, {@code dynamicResetInterval} and {@code dynamicBadnessThreshold}
    +     * The parameters {@code dynamicUpdateInterval}, {@code dynamicLatencyProbeInterval} and {@code dynamicBadnessThreshold}
          * can be specified individually to update the parameters of the dynamic snitch during runtime.
          *
    -     * @param epSnitchClassName        the canonical path name for a class implementing IEndpointSnitch
    -     * @param dynamic                  boolean that decides whether dynamicsnitch is used or not - only valid, if {@code epSnitchClassName} is specified
    -     * @param dynamicUpdateInterval    integer, in ms (defaults to the value configured in cassandra.yaml, which defaults to 100)
    -     * @param dynamicResetInterval     integer, in ms (defaults to the value configured in cassandra.yaml, which defaults to 600,000)
    -     * @param dynamicBadnessThreshold  double, (defaults to the value configured in cassandra.yaml, which defaults to 0.0)
    +     * @param epSnitchClassName          the canonical path name for a class implementing IEndpointSnitch
    +     * @param dynamicBadnessThreshold    double, (defaults to the value configured in cassandra.yaml, which defaults to 0.0)
    +     * @param dynamic                    boolean that decides whether dynamicsnitch is used or not - only valid, if {@code epSnitchClassName} is specified
    +     * @param dynamicUpdateInterval      integer, in ms (defaults to the value configured in cassandra.yaml, which defaults to 100)
    +     * @param dynamicLatencyProbeInterval integer, in ms (defaults to the value configured in cassandra.yaml, which defaults to 1,000)
          */
    +    public void updateSnitch(String epSnitchClassName, Double dynamicBadnessThreshold, Boolean dynamic, Integer dynamicUpdateInterval, Integer dynamicLatencyProbeInterval) throws ClassNotFoundException;
    --- End diff --
    
    They can specify a snitch class name, but not a dynamic snitch class name?
    In general they can't specify much in terms of arbitrary config for the DES they custom specified earlier. Seems like pluggability without configurability seems to be only half way there. Seems like they should be able to provide a map containing the config.
    
    I mean maybe custom snitches are configured some other way, but that turns updating the snitch via JMX into a two step process. Once to satisfy C* and again to satisfy your custom snitch implementation.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225361447
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -22,121 +22,194 @@
     import java.net.InetAddress;
     import java.net.UnknownHostException;
     import java.util.*;
    -import java.util.concurrent.ConcurrentHashMap;
     import java.util.concurrent.ScheduledFuture;
     import java.util.concurrent.TimeUnit;
     import java.util.stream.Collectors;
     
    -import com.codahale.metrics.ExponentiallyDecayingReservoir;
     import javax.management.MBeanServer;
     import javax.management.ObjectName;
     
    -import com.codahale.metrics.Snapshot;
    +import com.google.common.annotations.VisibleForTesting;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
     import org.apache.cassandra.concurrent.ScheduledExecutors;
     import org.apache.cassandra.config.DatabaseDescriptor;
     import org.apache.cassandra.gms.ApplicationState;
     import org.apache.cassandra.gms.EndpointState;
     import org.apache.cassandra.gms.Gossiper;
     import org.apache.cassandra.gms.VersionedValue;
    +import org.apache.cassandra.net.IAsyncCallback;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +import org.apache.cassandra.net.MessageIn;
    +import org.apache.cassandra.net.MessageOut;
     import org.apache.cassandra.net.MessagingService;
    +import org.apache.cassandra.net.PingMessage;
     import org.apache.cassandra.service.StorageService;
     import org.apache.cassandra.utils.FBUtilities;
     
    +import static org.apache.cassandra.net.MessagingService.Verb.PING;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.LARGE_MESSAGE;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.SMALL_MESSAGE;
    +
    +
     /**
      * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector
    + * Note that the subclasses (e.g. {@link DynamicEndpointSnitchHistogram}) are responsible for actually measuring
    + * latency and populating the {@link #scores} map.
      */
    -public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
    +public abstract class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
     {
    -    private static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    -
    -    private static final double ALPHA = 0.75; // set to 0.75 to make EDS more biased to towards the newer values
    -    private static final int WINDOW_SIZE = 100;
    -
    -    private volatile int dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    -    private volatile int dynamicResetInterval = DatabaseDescriptor.getDynamicResetInterval();
    -    private volatile double dynamicBadnessThreshold = DatabaseDescriptor.getDynamicBadnessThreshold();
    +    private static final Logger logger = LoggerFactory.getLogger(DynamicEndpointSnitch.class);
    +
    +    // Subclass specific functionality
    +    protected static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    protected boolean registered = false;
    +    // The scores map is updated via copy in updateScores
    +    // We keep it in the base class for performance reasons (so it can be easily aliased)
    +    protected volatile Map<InetAddressAndPort, Double> scores = new HashMap<>();
    +
    +    // Rate limit how often we generate latency probes
    +    protected long nextAllowedProbeGenerationTime;
    +    protected long nextProbeGenerationTime;
    +    protected int currentProbePosition = 0;
    +    protected final List<InetAddressAndPort> latencyProbeSequence = new ArrayList<>();
    +
    +    // DES general functionality
    +    protected volatile int dynamicUpdateInterval = -1;
    +    protected volatile int dynamicLatencyProbeInterval = -1;
    +    protected volatile double dynamicBadnessThreshold = 0;
     
         // the score for a merged set of endpoints must be this much worse than the score for separate endpoints to
         // warrant not merging two ranges into a single range
         private static final double RANGE_MERGING_PREFERENCE = 1.5;
     
         private String mbeanName;
    -    private boolean registered = false;
    -
    -    private volatile HashMap<InetAddressAndPort, Double> scores = new HashMap<>();
    -    private final ConcurrentHashMap<InetAddressAndPort, ExponentiallyDecayingReservoir> samples = new ConcurrentHashMap<>();
    +    private boolean mbeanRegistered = false;
     
         public final IEndpointSnitch subsnitch;
     
    -    private volatile ScheduledFuture<?> updateSchedular;
    -    private volatile ScheduledFuture<?> resetSchedular;
    +    private volatile ScheduledFuture<?> updateScheduler;
    +    private volatile ScheduledFuture<?> latencyProbeScheduler;
     
    -    private final Runnable update;
    -    private final Runnable reset;
    +    private final Runnable update = this::updateScores;
    +    private final Runnable latencyProbe = this::maybeSendLatencyProbe;
     
         public DynamicEndpointSnitch(IEndpointSnitch snitch)
         {
             this(snitch, null);
         }
     
    -    public DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
    +    protected DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
         {
             mbeanName = "org.apache.cassandra.db:type=DynamicEndpointSnitch";
             if (instance != null)
                 mbeanName += ",instance=" + instance;
             subsnitch = snitch;
    -        update = new Runnable()
    -        {
    -            public void run()
    -            {
    -                updateScores();
    -            }
    -        };
    -        reset = new Runnable()
    -        {
    -            public void run()
    -            {
    -                // we do this so that a host considered bad has a chance to recover, otherwise would we never try
    -                // to read from it, which would cause its score to never change
    -                reset();
    -            }
    -        };
     
             if (DatabaseDescriptor.isDaemonInitialized())
             {
    -            updateSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(update, dynamicUpdateInterval, dynamicUpdateInterval, TimeUnit.MILLISECONDS);
    -            resetSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(reset, dynamicResetInterval, dynamicResetInterval, TimeUnit.MILLISECONDS);
    +            applyConfigChanges();
                 registerMBean();
             }
    +        nextProbeGenerationTime = System.nanoTime();
    +    }
    +
    +    /**
    +     * Records a latency. This MUST be cheap as it is called in the fast path
    +     */
    +    abstract public void receiveTiming(InetAddressAndPort address, long latency, LatencyMeasurementType measurementType);
    +
    +    /**
    +     * Called periodically from {@link DynamicEndpointSnitch#updateScores()}. This method returns a new score map
    +     * that will be assigned to the {@link DynamicEndpointSnitch#scores} map.
    +     * This is generally more expensive than {@link DynamicEndpointSnitch#receiveTiming}
    +     */
    +    abstract public Map<InetAddressAndPort, Double> calculateScores();
    +
    +    /**
    +     * Signals that we actually tried to rank vs this host. This way any latency probing can just focus
    +     * on hosts that this peer would potentially talk to.
    +     * @param address
    +     */
    +    abstract protected void markRequested(InetAddressAndPort address);
    +
    +    /**
    +     * Dump the underlying metrics backing the DES's decisions for a given host
    +     */
    +    abstract public List<Double> dumpTimings(String hostname) throws UnknownHostException;
    +
    +    /**
    +     * Populates the provided probe sequence using the underlying metrics
    +     */
    +    abstract void updateLatencyProbeSequence(List<InetAddressAndPort> sequence);
    --- End diff --
    
    Agreed assuming I can pull the samples back to the superclass, working on it


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r229468795
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -23,128 +23,300 @@
     import java.net.UnknownHostException;
     import java.util.*;
     import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ExecutionException;
     import java.util.concurrent.ScheduledFuture;
     import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
     import java.util.stream.Collectors;
     
    -import com.codahale.metrics.ExponentiallyDecayingReservoir;
     import javax.management.MBeanServer;
     import javax.management.ObjectName;
     
    -import com.codahale.metrics.Snapshot;
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.util.concurrent.Futures;
    +import com.google.common.util.concurrent.MoreExecutors;
    +import com.google.common.util.concurrent.RateLimiter;
    +import com.google.common.util.concurrent.SettableFuture;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
     import org.apache.cassandra.concurrent.ScheduledExecutors;
     import org.apache.cassandra.config.DatabaseDescriptor;
    +import org.apache.cassandra.exceptions.RequestFailureReason;
     import org.apache.cassandra.gms.ApplicationState;
     import org.apache.cassandra.gms.EndpointState;
     import org.apache.cassandra.gms.Gossiper;
     import org.apache.cassandra.gms.VersionedValue;
    +import org.apache.cassandra.locator.dynamicsnitch.DynamicEndpointSnitchHistogram;
    +import org.apache.cassandra.net.IAsyncCallbackWithFailure;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +import org.apache.cassandra.net.MessageIn;
    +import org.apache.cassandra.net.MessageOut;
     import org.apache.cassandra.net.MessagingService;
    +import org.apache.cassandra.net.PingMessage;
     import org.apache.cassandra.service.StorageService;
     import org.apache.cassandra.utils.FBUtilities;
     
    +import static org.apache.cassandra.net.MessagingService.Verb.PING;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.LARGE_MESSAGE;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.SMALL_MESSAGE;
    +
    +
     /**
      * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector
    + * Note that the subclasses (e.g. {@link DynamicEndpointSnitchHistogram}) are responsible for actually measuring
    + * latency and providing an ISnitchMeasurement implementation back to this class.
      */
    -public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
    +public abstract class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
     {
    -    private static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    -
    -    private static final double ALPHA = 0.75; // set to 0.75 to make EDS more biased to towards the newer values
    -    private static final int WINDOW_SIZE = 100;
    -
    -    private volatile int dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    -    private volatile int dynamicResetInterval = DatabaseDescriptor.getDynamicResetInterval();
    -    private volatile double dynamicBadnessThreshold = DatabaseDescriptor.getDynamicBadnessThreshold();
    +    private static final Logger logger = LoggerFactory.getLogger(DynamicEndpointSnitch.class);
    +
    +    // Latency measurement and ranking. The samples contain latency measurements and the scores are used for ranking
    +    protected boolean registered = false;
    +    protected static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    protected volatile Map<InetAddressAndPort, Double> scores = new HashMap<>();
    +    protected final Map<InetAddressAndPort, AnnotatedMeasurement> samples = new ConcurrentHashMap<>();
    +
    +    // Latency probe functionality for actively probing endpoints that we haven't measured recently but are ranking
    +    public static final long MAX_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_max_probe_interval_ms", 60 * 10 * 1000L);
    +    public static final long MIN_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_min_probe_interval_ms", 60 * 1000L) ;
    +    // The probe rate is set later when configuration is read
    +    protected static final RateLimiter probeRateLimiter = RateLimiter.create(1);
    +    protected static final DebuggableScheduledThreadPoolExecutor latencyProbeExecutor = new DebuggableScheduledThreadPoolExecutor(1, "LatencyProbes", Thread.MIN_PRIORITY);
    +
    +    // User configuration of the snitch tunables
    +    protected volatile int dynamicUpdateInterval = -1;
    +    protected volatile int dynamicLatencyProbeInterval = -1;
    +    protected volatile double dynamicBadnessThreshold = 0;
     
         // the score for a merged set of endpoints must be this much worse than the score for separate endpoints to
         // warrant not merging two ranges into a single range
         private static final double RANGE_MERGING_PREFERENCE = 1.5;
     
         private String mbeanName;
    -    private boolean registered = false;
    -
    -    private volatile HashMap<InetAddressAndPort, Double> scores = new HashMap<>();
    -    private final ConcurrentHashMap<InetAddressAndPort, ExponentiallyDecayingReservoir> samples = new ConcurrentHashMap<>();
    +    private boolean mbeanRegistered = false;
     
         public final IEndpointSnitch subsnitch;
     
    -    private volatile ScheduledFuture<?> updateSchedular;
    -    private volatile ScheduledFuture<?> resetSchedular;
    -
    -    private final Runnable update;
    -    private final Runnable reset;
    +    private volatile ScheduledFuture<?> updateScoresScheduler;
    +    private volatile ScheduledFuture<?> updateSamplesScheduler;
     
         public DynamicEndpointSnitch(IEndpointSnitch snitch)
         {
             this(snitch, null);
         }
     
    -    public DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
    +    protected DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
         {
             mbeanName = "org.apache.cassandra.db:type=DynamicEndpointSnitch";
             if (instance != null)
                 mbeanName += ",instance=" + instance;
             subsnitch = snitch;
    -        update = new Runnable()
    +
    +        if (DatabaseDescriptor.isDaemonInitialized())
             {
    -            public void run()
    -            {
    -                updateScores();
    -            }
    -        };
    -        reset = new Runnable()
    +            applyConfigChanges(DatabaseDescriptor.getDynamicUpdateInterval(),
    +                               DatabaseDescriptor.getDynamicSampleUpdateInterval(),
    +                               DatabaseDescriptor.getDynamicBadnessThreshold());
    +            open();
    +        }
    +    }
    +
    +    /**
    +     * Allows subclasses to inject new ways of measuring latency back to this abstract base class.
    +     */
    +    protected interface ISnitchMeasurement
    +    {
    +        void sample(long value);
    +        double measure();
    +        Iterable<Double> measurements();
    +    }
    +
    +    /**
    +     * Adds some boookeeping that the DES uses over top of the various metrics techniques used by the
    +     * implementations. This is used to allow CASSANDRA-14459 latency probes as well as further experimentation
    +     * on new latency measurement techniques in CASSANDRA-14817
    +     *
    +     * {@link AnnotatedMeasurement#millisSinceLastRequest} is set to zero through
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}. It defaults to
    +     * the maximum interval so that we only start probing once it has been ranked at least once
    +     * {@link AnnotatedMeasurement#millisSinceLastMeasure} is set to zero from
    +     * {@link DynamicEndpointSnitch#receiveTiming(InetAddressAndPort, long, LatencyMeasurementType)}.
    +     *
    +     * {@link AnnotatedMeasurement#millisSinceLastMeasure and {@link AnnotatedMeasurement#probeTimerMillis}
    +     * are incremented via {@link DynamicEndpointSnitch#updateSamples()}
    +     */
    +    protected static class AnnotatedMeasurement
    +    {
    +        // Used to optimally target latency probes only on nodes that are both requested for ranking
    +        // and are not being measured. For example with token aware clients a large portion of the cluster will never
    +        // be ranked at all and therefore we won't probe them.
    +        public AtomicLong millisSinceLastRequest = new AtomicLong(MAX_PROBE_INTERVAL_MS);
    +        public AtomicLong millisSinceLastMeasure = new AtomicLong(0);
    +        public long probeTimerMillis = 0;
    +        public ScheduledFuture<?> probeFuture = null;
    +
    +        // The underlying measurement technique. E.g. a median filter (histogram) or an EMA filter, or ...
    +        public final ISnitchMeasurement measurement;
    +
    +        public AnnotatedMeasurement(ISnitchMeasurement measurement)
    +        {
    +            this.measurement = measurement;
    +        }
    +
    +        @Override
    +        public String toString()
    +        {
    +            return "AnnotatedMeasurement{" +
    +                   "millisSinceLastRequest=" + millisSinceLastRequest +
    +                   ", millisSinceLastMeasure=" + millisSinceLastMeasure +
    +                   ", probeTimerMillis=" + probeTimerMillis +
    +                   ", probeFuturePending=" + (probeFuture != null && !probeFuture.isDone()) +
    +                   ", measurementClass=" + measurement.getClass().getSimpleName() +
    +                   '}';
    +        }
    +    }
    +
    +    /**
    +     * Allows the subclasses to inject ISnitchMeasurement instances back into this common class so that common
    +     * functionality can be handled here
    +     * @param initialValue The initial value of the measurement, some implementations may use this, others may not
    +     * @return a constructed instance of an ISnitchMeasurement interface
    +     */
    +    abstract protected ISnitchMeasurement measurementImpl(long initialValue);
    +
    +    /**
    +     * Records a latency. This MUST be cheap as it is called in the fast path
    +     */
    +    public void receiveTiming(InetAddressAndPort address, long latency, LatencyMeasurementType measurementType)
    +    {
    +        if (measurementType == LatencyMeasurementType.IGNORE)
    +           return;
    +
    +        AnnotatedMeasurement sample = samples.get(address);
    +
    +        if (sample == null)
    +        {
    +            AnnotatedMeasurement maybeNewSample = new AnnotatedMeasurement(measurementImpl(latency));
    +            sample = samples.putIfAbsent(address, maybeNewSample);
    +            if (sample == null)
    +                sample = maybeNewSample;
    +        }
    +
    +        if (measurementType == LatencyMeasurementType.READ && sample.millisSinceLastMeasure.get() > 0)
    +            sample.millisSinceLastMeasure.lazySet(0);
    +
    +        sample.measurement.sample(latency);
    +    }
    +
    +    @VisibleForTesting
    +    protected void reset()
    +    {
    +        scores.clear();
    +        samples.clear();
    +    }
    +
    +    @VisibleForTesting
    +    void updateScores()
    +    {
    +        if (!StorageService.instance.isGossipActive())
    +            return;
    +
    +        if (!registered)
             {
    -            public void run()
    +            if (MessagingService.instance() != null)
                 {
    -                // we do this so that a host considered bad has a chance to recover, otherwise would we never try
    -                // to read from it, which would cause its score to never change
    -                reset();
    +                MessagingService.instance().register(this);
    +                registered = true;
                 }
    -        };
    +        }
     
    -        if (DatabaseDescriptor.isDaemonInitialized())
    +        this.scores = calculateScores();
    +    }
    +
    +    /**
    +     * This is generally expensive and is called periodically (semi-frequently) not on the fast path.
    +     * @return a freshly constructed scores map.
    +     */
    +    public Map<InetAddressAndPort, Double> calculateScores()
    +    {
    +        double maxLatency = 1;
    +
    +        Map<InetAddressAndPort, Double> measurements = new HashMap<>(samples.size());
    +
    +        // We're going to weight the latency for each host against the worst one we see, to
    +        // arrive at sort of a 'badness percentage' for them. First, find the worst for each:
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry : samples.entrySet())
             {
    -            updateSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(update, dynamicUpdateInterval, dynamicUpdateInterval, TimeUnit.MILLISECONDS);
    -            resetSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(reset, dynamicResetInterval, dynamicResetInterval, TimeUnit.MILLISECONDS);
    -            registerMBean();
    +            // This is expensive for e.g. the Histogram, so do it once and cache the result
    +            double measure = entry.getValue().measurement.measure();
    +            if (measure > maxLatency)
    +                maxLatency = measure;
    +            measurements.put(entry.getKey(), measure);
             }
    +
    +        HashMap<InetAddressAndPort, Double> newScores = new HashMap<>(measurements.size());
    +
    +        // now make another pass to do the weighting based on the maximums we found before
    +        for (Map.Entry<InetAddressAndPort, Double> entry : measurements.entrySet())
    +        {
    +            double score = entry.getValue() / maxLatency;
    +            // finally, add the severity without any weighting, since hosts scale this relative to their own load and the size of the task causing the severity.
    +            // "Severity" is basically a measure of compaction activity (CASSANDRA-3722).
    +            if (USE_SEVERITY)
    +                score += getSeverity(entry.getKey());
    +            // lowest score (least amount of badness) wins.
    +            newScores.put(entry.getKey(), score);
    +        }
    +
    +        return newScores;
         }
     
         /**
    -     * Update configuration from {@link DatabaseDescriptor} and estart the update-scheduler and reset-scheduler tasks
    +     * Update configuration from {@link DatabaseDescriptor} and restart the various scheduler tasks
          * if the configured rates for these tasks have changed.
          */
    -    public void applyConfigChanges()
    +    public void applyConfigChanges(int newDynamicUpdateInternal, int newDynamicLatencyProbeInterval, double newDynamicBadnessThreshold)
         {
    -        if (dynamicUpdateInterval != DatabaseDescriptor.getDynamicUpdateInterval())
    +        if (dynamicUpdateInterval != newDynamicUpdateInternal)
             {
    -            dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    +            dynamicUpdateInterval = newDynamicUpdateInternal;
                 if (DatabaseDescriptor.isDaemonInitialized())
                 {
    -                updateSchedular.cancel(false);
    -                updateSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(update, dynamicUpdateInterval, dynamicUpdateInterval, TimeUnit.MILLISECONDS);
    +                if (updateScoresScheduler != null)
    +                    updateScoresScheduler.cancel(false);
    +                updateScoresScheduler = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(this::updateScores, dynamicUpdateInterval, dynamicUpdateInterval, TimeUnit.MILLISECONDS);
    --- End diff --
    
    Should this be more "maybeScheduleProbes"


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225348415
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitchEMA.java ---
    @@ -0,0 +1,181 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.cassandra.locator;
    +
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.concurrent.ConcurrentHashMap;
    +
    +import org.apache.cassandra.gms.Gossiper;
    +import org.apache.cassandra.metrics.ExponentialMovingAverage;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +
    +
    +/**
    + * A dynamic snitching implementation that uses Exponentially Decaying Histograms to prefer or
    + * de-prefer hosts
    + *
    + * This was the default implementation prior to Cassandra 4.0 and is being left as the default
    + * in 4.0
    + */
    +public class DynamicEndpointSnitchEMA extends DynamicEndpointSnitch
    +{
    +    // A ~10 sample EMA
    +    private static final double EMA_ALPHA = 0.10;
    +
    +    private final ConcurrentHashMap<InetAddressAndPort, AnnotatedEMA> samples = new ConcurrentHashMap<>();
    +
    +    /**
    +     * Adds two boolean markers to the ExponentialMovingAverage for telling if the data has been
    +     * updated or requested recently.
    +     *
    +     * recentlyMeasured is updated through {@link AnnotatedEMA#update(long, boolean)}
    +     * recentlyRequested is updated through {@link DynamicEndpointSnitch#markRequested}
    +     *
    +     * Both markers are periodically reset via {@link DynamicEndpointSnitch#latencyProbeNeeded(long)}
    +     */
    +    private static class AnnotatedEMA extends ExponentialMovingAverage
    +    {
    +        volatile boolean recentlyRequested = false;
    +        volatile boolean recentlyMeasured = false;
    +
    +        AnnotatedEMA(double alpha, double initialValue)
    +        {
    +            super(alpha, initialValue);
    +        }
    +
    +        public void update(long value, boolean isRealRead)
    +        {
    +            recentlyMeasured = recentlyMeasured || isRealRead;
    --- End diff --
    
    Ah, I interpreted "store extra boolean" flags as meaning you wanted booleans, but I guess what you mean is C style enum flags? I dig it.
    
    Changing the style to Tequals as now we're doing bitwise flags.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225349554
  
    --- Diff: src/java/org/apache/cassandra/net/MessagingService.java ---
    @@ -665,9 +665,9 @@ public void updateBackPressureOnSend(InetAddressAndPort host, IAsyncCallback cal
         /**
          * Updates the back-pressure state on reception from the given host if enabled and the given message callback supports it.
          *
    -     * @param host The replica host the back-pressure state refers to.
    +     * @param host     The replica host the back-pressure state refers to.
    --- End diff --
    
    Interesting, I'm just using the default style guide for idea that came with Cassandra (`ant generate-idea-files`). If it's just that review is tough you can tell github not to show you whitespace changes.
    
    That's good feedback though that you don't want unrelated whitespace changes, I'll take it out :-)


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r229483741
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -154,31 +326,203 @@ private void registerMBean()
     
         public void close()
         {
    -        updateSchedular.cancel(false);
    -        resetSchedular.cancel(false);
    +        if (updateScoresScheduler != null)
    +            updateScoresScheduler.cancel(false);
    +        if (updateSamplesScheduler != null)
    +            updateSamplesScheduler.cancel(false);
    +
    +        for (AnnotatedMeasurement measurement : samples.values())
    +        {
    +            if (measurement.probeFuture != null)
    +                measurement.probeFuture.cancel(false);
    +
    +            measurement.millisSinceLastMeasure.set(0);
    +            measurement.millisSinceLastRequest.set(MAX_PROBE_INTERVAL_MS);
    +            measurement.probeTimerMillis = 0;
    +        }
     
             MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
             try
             {
    -            mbs.unregisterMBean(new ObjectName(mbeanName));
    +            if (mbeanRegistered)
    +                mbs.unregisterMBean(new ObjectName(mbeanName));
             }
             catch (Exception e)
             {
                 throw new RuntimeException(e);
             }
         }
     
    +    /**
    +     * Background task running on the samples dictionary. The default implementation sends latency probes (PING)
    +     * messages to explore nodes that we have not received timings for recently but have ranked in
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}.
    +     */
    +    protected void updateSamples()
    +    {
    +        // Split calculation of probe timers from sending probes for testability
    +        calculateProbes(samples, dynamicLatencyProbeInterval);
    +
    +        if (!StorageService.instance.isGossipActive())
    +            return;
    +
    +        schedulePings(samples);
    +    }
    +
    +    /**
    +     * This method mutates the passed AnnotatedMeasurements to implement capped exponential backoff per endpoint.
    +     *
    +     * The algorithm is as follows:
    +     * 1. All samples get their millisSinceLastMeasure and millisSinceLastRequest fields
    +     *    incremented by the passed interval
    +     * 2. Any recently requested (ranked) endpoints that have not been measured recently (e.g. because the snitch
    +     *    has sent them no traffic) get probes with exponential backoff.
    +     *
    +     * The backoff is capped at MAX_PROBE_INTERVAL_MS. Furthermore the probes are stopped after
    +     * MAX_PROBE_INTERVAL_MS of no ranking requests as well.
    +     *
    +     * At the end of this method, any passed AnnotatedMeasurements that need latency probes will have non zero
    +     * probeTimerMillis members set.
    +     */
    +    @VisibleForTesting
    +    static void calculateProbes(Map<InetAddressAndPort, AnnotatedMeasurement> samples, long intervalMillis) {
    --- End diff --
    
    Yea if we end up moving the ping scheduling to their own thread this will basically be mandatory imo.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r240295987
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -21,133 +21,505 @@
     import java.net.InetAddress;
     import java.net.UnknownHostException;
     import java.util.*;
    +import java.util.concurrent.CancellationException;
     import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ExecutionException;
     import java.util.concurrent.ScheduledFuture;
     import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
    +import java.util.concurrent.atomic.AtomicLong;
     import java.util.stream.Collectors;
     
    -import com.codahale.metrics.ExponentiallyDecayingReservoir;
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.util.concurrent.Futures;
    +import com.google.common.util.concurrent.MoreExecutors;
    +import com.google.common.util.concurrent.RateLimiter;
    +import com.google.common.util.concurrent.SettableFuture;
     
    -import com.codahale.metrics.Snapshot;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
     import org.apache.cassandra.concurrent.ScheduledExecutors;
     import org.apache.cassandra.config.DatabaseDescriptor;
    +import org.apache.cassandra.exceptions.RequestFailureReason;
     import org.apache.cassandra.gms.ApplicationState;
     import org.apache.cassandra.gms.EndpointState;
     import org.apache.cassandra.gms.Gossiper;
     import org.apache.cassandra.gms.VersionedValue;
    +import org.apache.cassandra.locator.dynamicsnitch.DynamicEndpointSnitchHistogram;
    +import org.apache.cassandra.net.IAsyncCallbackWithFailure;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +import org.apache.cassandra.net.MessageIn;
    +import org.apache.cassandra.net.MessageOut;
     import org.apache.cassandra.net.MessagingService;
    +import org.apache.cassandra.net.PingMessage;
     import org.apache.cassandra.service.StorageService;
     import org.apache.cassandra.utils.FBUtilities;
     import org.apache.cassandra.utils.MBeanWrapper;
     
    +import static org.apache.cassandra.net.MessagingService.Verb.PING;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.LARGE_MESSAGE;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.SMALL_MESSAGE;
    +
    +
     /**
      * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector
    + * Note that the subclasses (e.g. {@link DynamicEndpointSnitchHistogram}) are responsible for actually measuring
    + * latency and providing an ISnitchMeasurement implementation back to this class.
      */
    -public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
    +public abstract class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
     {
    -    private static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    private static final Logger logger = LoggerFactory.getLogger(DynamicEndpointSnitch.class);
    +
    +    // Latency measurement and ranking. The samples contain latency measurements and the scores are used for ranking
    +    protected boolean registered = false;
    +    protected static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    protected volatile Map<InetAddressAndPort, Double> scores = new HashMap<>();
     
    -    private static final double ALPHA = 0.75; // set to 0.75 to make EDS more biased to towards the newer values
    -    private static final int WINDOW_SIZE = 100;
    +    protected final Map<InetAddressAndPort, AnnotatedMeasurement> samples = new ConcurrentHashMap<>();
     
    -    private volatile int dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    -    private volatile int dynamicResetInterval = DatabaseDescriptor.getDynamicResetInterval();
    -    private volatile double dynamicBadnessThreshold = DatabaseDescriptor.getDynamicBadnessThreshold();
    +    // Latency probe functionality for actively probing endpoints that we haven't measured recently but are ranking
    +    public static final long MAX_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_max_probe_interval_ms", 60 * 10 * 1000L);
    +    public static final long MIN_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_min_probe_interval_ms", 60 * 1000L) ;
    +    // The probe rate is set later when configuration is read in applyConfigChanges
    +    protected static final RateLimiter probeRateLimiter = RateLimiter.create(1);
    +    protected static final DebuggableScheduledThreadPoolExecutor latencyProbeExecutor = new DebuggableScheduledThreadPoolExecutor("LatencyProbes");
    +    private long lastUpdateSamplesNanos = System.nanoTime();
    +
    +    // User configuration of the snitch tunables
    +    protected volatile int dynamicUpdateInterval = -1;
    +    protected volatile int dynamicSampleUpdateInterval = -1;
    +    protected volatile double dynamicBadnessThreshold = 0;
     
         // the score for a merged set of endpoints must be this much worse than the score for separate endpoints to
         // warrant not merging two ranges into a single range
         private static final double RANGE_MERGING_PREFERENCE = 1.5;
     
         private String mbeanName;
    -    private boolean registered = false;
    -
    -    private volatile HashMap<InetAddressAndPort, Double> scores = new HashMap<>();
    -    private final ConcurrentHashMap<InetAddressAndPort, ExponentiallyDecayingReservoir> samples = new ConcurrentHashMap<>();
    +    private boolean mbeanRegistered = false;
     
         public final IEndpointSnitch subsnitch;
     
    -    private volatile ScheduledFuture<?> updateSchedular;
    -    private volatile ScheduledFuture<?> resetSchedular;
    -
    -    private final Runnable update;
    -    private final Runnable reset;
    +    private volatile ScheduledFuture<?> updateScoresScheduler = null;
    +    private volatile ScheduledFuture<?> updateSamplesScheduler = null;
     
         public DynamicEndpointSnitch(IEndpointSnitch snitch)
         {
             this(snitch, null);
         }
     
    -    public DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
    +    protected DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
         {
             mbeanName = "org.apache.cassandra.db:type=DynamicEndpointSnitch";
             if (instance != null)
                 mbeanName += ",instance=" + instance;
             subsnitch = snitch;
    -        update = new Runnable()
    +
    +        if (DatabaseDescriptor.isDaemonInitialized())
             {
    -            public void run()
    +            open();
    +        }
    +    }
    +
    +    /**
    +     * Update configuration of the background tasks and restart the various scheduler tasks
    +     * if the configured rates for these tasks have changed.
    +     */
    +    public void applyConfigChanges(int newDynamicUpdateInternal, int newDynamicSampleUpdateInterval, double newDynamicBadnessThreshold)
    +    {
    +        if (DatabaseDescriptor.isDaemonInitialized())
    +        {
    +            if (dynamicUpdateInterval != newDynamicUpdateInternal || updateScoresScheduler == null)
                 {
    -                updateScores();
    +                cancelAndWait(updateScoresScheduler, Math.max(1, dynamicUpdateInterval), TimeUnit.MILLISECONDS);
    +                updateScoresScheduler = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(this::updateScores, newDynamicUpdateInternal, newDynamicUpdateInternal, TimeUnit.MILLISECONDS);
                 }
    -        };
    -        reset = new Runnable()
    +
    +            if (dynamicSampleUpdateInterval != newDynamicSampleUpdateInterval || updateSamplesScheduler == null)
    +            {
    +                cancelAndWait(updateSamplesScheduler, Math.max(1, dynamicSampleUpdateInterval), TimeUnit.MILLISECONDS);
    +                if (newDynamicSampleUpdateInterval > 0)
    +                    updateSamplesScheduler = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(this::updateSamples, newDynamicSampleUpdateInterval, newDynamicSampleUpdateInterval, TimeUnit.MILLISECONDS);
    +            }
    +        }
    +
    +        dynamicUpdateInterval = newDynamicUpdateInternal;
    +        dynamicSampleUpdateInterval = newDynamicSampleUpdateInterval;
    +        dynamicBadnessThreshold = newDynamicBadnessThreshold;
    +
    +        if (dynamicSampleUpdateInterval > 0)
    +            probeRateLimiter.setRate(dynamicSampleUpdateInterval);
    +    }
    +
    +    public synchronized void open()
    +    {
    +        applyConfigChanges(DatabaseDescriptor.getDynamicUpdateInterval(),
    +                           DatabaseDescriptor.getDynamicSampleUpdateInterval(),
    +                           DatabaseDescriptor.getDynamicBadnessThreshold());
    +
    +        MBeanWrapper.instance.registerMBean(this, mbeanName);
    +        mbeanRegistered = true;
    +    }
    +
    +    public synchronized void close()
    +    {
    +        cancelAndWait(updateScoresScheduler, Math.max(1, dynamicUpdateInterval), TimeUnit.MILLISECONDS);
    +        cancelAndWait(updateSamplesScheduler, Math.max(1, dynamicSampleUpdateInterval), TimeUnit.MILLISECONDS);
    +        updateScoresScheduler = null;
    +        updateSamplesScheduler = null;
    +
    +        for (AnnotatedMeasurement measurement : samples.values())
             {
    -            public void run()
    +            cancelAndWait(measurement.probeFuture, PING.getTimeout(), TimeUnit.MILLISECONDS);
    +
    +            measurement.millisSinceLastMeasure.set(0);
    +            measurement.millisSinceLastRequest.set(MAX_PROBE_INTERVAL_MS);
    +            measurement.nextProbeDelayMillis = 0;
    +        }
    +
    +        if (mbeanRegistered)
    +            MBeanWrapper.instance.unregisterMBean(mbeanName);
    +
    +        mbeanRegistered = false;
    +    }
    +
    +    private static void cancelAndWait(ScheduledFuture future, long timeout, TimeUnit unit)
    +    {
    +        if (future != null)
    +        {
    +            future.cancel(false);
    +            try
                 {
    -                // we do this so that a host considered bad has a chance to recover, otherwise would we never try
    -                // to read from it, which would cause its score to never change
    -                reset();
    +                future.get(timeout, unit);
                 }
    -        };
    +            catch (CancellationException | InterruptedException | ExecutionException | TimeoutException ignored)
    +            {
    +                // Exception is expected to happen eventually due to the cancel -> get
    +            }
    +        }
    +    }
     
    -        if (DatabaseDescriptor.isDaemonInitialized())
    +    /**
    +     * Allows subclasses to inject new ways of measuring latency back to this abstract base class.
    +     */
    +    protected interface ISnitchMeasurement
    +    {
    +        void sample(long value);
    +        double measure();
    +        Iterable<Double> measurements();
    +    }
    +
    +    /**
    +     * Adds some boookeeping that the DES uses over top of the various metrics techniques used by the
    +     * implementations. This is used to allow CASSANDRA-14459 latency probes as well as further safe
    +     * experimentation on new latency measurement techniques in CASSANDRA-14817
    +     *
    +     * {@link AnnotatedMeasurement#millisSinceLastRequest} is set to zero through
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}. It defaults to
    +     * the maximum interval so that we only start probing once it has been ranked at least once
    +     * {@link AnnotatedMeasurement#millisSinceLastMeasure} is set to zero from
    +     * {@link DynamicEndpointSnitch#receiveTiming(InetAddressAndPort, long, LatencyMeasurementType)}.
    +     *
    +     * {@link AnnotatedMeasurement#millisSinceLastMeasure and {@link AnnotatedMeasurement#nextProbeDelayMillis }
    +     * are incremented via {@link DynamicEndpointSnitch#updateSamples()}
    +     */
    +    protected static class AnnotatedMeasurement
    +    {
    +        // Used to optimally target latency probes only on nodes that are both requested for ranking
    +        // and are not being measured. For example with token aware clients a large portion of the cluster will never
    +        // be ranked at all and therefore we won't probe them.
    +        public AtomicLong millisSinceLastRequest = new AtomicLong(MAX_PROBE_INTERVAL_MS);
    +        public AtomicLong millisSinceLastMeasure = new AtomicLong(0);
    +        public volatile long nextProbeDelayMillis = 0;
    +        public volatile ScheduledFuture<?> probeFuture = null;
    +
    +        // The underlying measurement technique. E.g. a median filter (histogram) or an EMA filter, or ...
    +        public final ISnitchMeasurement measurement;
    +        public volatile double cachedMeasurement;
    +
    +        public AnnotatedMeasurement(ISnitchMeasurement measurement)
             {
    -            updateSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(update, dynamicUpdateInterval, dynamicUpdateInterval, TimeUnit.MILLISECONDS);
    -            resetSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(reset, dynamicResetInterval, dynamicResetInterval, TimeUnit.MILLISECONDS);
    -            registerMBean();
    +            this.measurement = measurement;
    +            this.cachedMeasurement = measurement.measure();
    +        }
    +
    +        @Override
    +        public String toString()
    +        {
    +            return "AnnotatedMeasurement{" +
    +                   "millisSinceLastRequest=" + millisSinceLastRequest +
    +                   ", millisSinceLastMeasure=" + millisSinceLastMeasure +
    +                   ", nextProbeDelayMillis=" + nextProbeDelayMillis +
    +                   ", probeFuturePending=" + (probeFuture != null && !probeFuture.isDone()) +
    +                   ", measurementClass=" + measurement.getClass().getSimpleName() +
    +                   ", cachedMeasurement=" + cachedMeasurement +
    +                   '}';
             }
         }
     
         /**
    -     * Update configuration from {@link DatabaseDescriptor} and estart the update-scheduler and reset-scheduler tasks
    -     * if the configured rates for these tasks have changed.
    +     * Allows the subclasses to inject ISnitchMeasurement instances back into this common class so that common
    +     * functionality can be handled here
    +     * @param initialValue The initial value of the measurement, some implementations may use this, others may not
    +     * @return a constructed instance of an ISnitchMeasurement interface
          */
    -    public void applyConfigChanges()
    +    abstract protected ISnitchMeasurement measurementImpl(long initialValue);
    +
    +    /**
    +     * Records a latency. This MUST be cheap as it is called in the fast path
    +     */
    +    public void receiveTiming(InetAddressAndPort address, long latencyMicros, LatencyMeasurementType measurementType)
    +    {
    +        if (measurementType == LatencyMeasurementType.IGNORE)
    +           return;
    +
    +        AnnotatedMeasurement sample = samples.get(address);
    +
    +        if (sample == null)
    +        {
    +            AnnotatedMeasurement maybeNewSample = new AnnotatedMeasurement(measurementImpl(latencyMicros));
    +            sample = samples.putIfAbsent(address, maybeNewSample);
    +            if (sample == null)
    +                sample = maybeNewSample;
    +        }
    +
    +        if (measurementType == LatencyMeasurementType.READ && sample.millisSinceLastMeasure.get() > 0)
    +            sample.millisSinceLastMeasure.lazySet(0);
    +
    +        sample.measurement.sample(latencyMicros);
    +    }
    +
    +    @VisibleForTesting
    +    protected void reset()
    +    {
    +        scores.clear();
    +        samples.clear();
    +    }
    +
    +    @VisibleForTesting
    +    protected void updateScores()
         {
    -        if (dynamicUpdateInterval != DatabaseDescriptor.getDynamicUpdateInterval())
    +        if (!StorageService.instance.isGossipActive())
    +            return;
    +
    +        if (!registered)
             {
    -            dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    -            if (DatabaseDescriptor.isDaemonInitialized())
    +            if (MessagingService.instance() != null)
                 {
    -                updateSchedular.cancel(false);
    -                updateSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(update, dynamicUpdateInterval, dynamicUpdateInterval, TimeUnit.MILLISECONDS);
    +                MessagingService.instance().register(this);
    +                registered = true;
                 }
             }
     
    -        if (dynamicResetInterval != DatabaseDescriptor.getDynamicResetInterval())
    +        this.scores = calculateScores(samples);
    +    }
    +
    +    /**
    +     * This is generally expensive and is called periodically (semi-frequently) not on the fast path.
    +     * The main concern here is generating garbage from the measurements (e.g. histograms in particular)
    +     */
    +    @VisibleForTesting
    +    protected static Map<InetAddressAndPort, Double> calculateScores(Map<InetAddressAndPort, AnnotatedMeasurement> samples)
    +    {
    +        double maxLatency = 1;
    +        HashMap<InetAddressAndPort, Double> newScores = new HashMap<>(samples.size());
    +
    +        // We're going to weight the latency for each host against the worst one we see, to
    +        // arrive at sort of a 'badness percentage' for them. First, find the worst latency for each:
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry : samples.entrySet())
             {
    -            dynamicResetInterval = DatabaseDescriptor.getDynamicResetInterval();
    -            if (DatabaseDescriptor.isDaemonInitialized())
    +            AnnotatedMeasurement annotatedMeasurement = entry.getValue();
    +
    +            // only compute the measurement, which probably generates the most garbage (e.g. for this Histogram),
    +            // for endpoints that have been recently updated (millisSinceLastRequest) or somewhat recently requested
    +            // for ranking.
    +            if (annotatedMeasurement.millisSinceLastMeasure.get() < MIN_PROBE_INTERVAL_MS ||
    +                annotatedMeasurement.millisSinceLastRequest.get() <= MAX_PROBE_INTERVAL_MS)
                 {
    -                resetSchedular.cancel(false);
    -                resetSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(reset, dynamicResetInterval, dynamicResetInterval, TimeUnit.MILLISECONDS);
    +                // This is expensive for e.g. the Histogram, so do it once and cache the result
    +                annotatedMeasurement.cachedMeasurement = annotatedMeasurement.measurement.measure();
                 }
    +
    +            newScores.put(entry.getKey(), annotatedMeasurement.cachedMeasurement);
    +            maxLatency = Math.max(annotatedMeasurement.cachedMeasurement, maxLatency);
             }
     
    -        dynamicBadnessThreshold = DatabaseDescriptor.getDynamicBadnessThreshold();
    +        // now make another pass to normalize the latency scores based on the maximums we found before
    +        for (Map.Entry<InetAddressAndPort, Double> entry : newScores.entrySet())
    +        {
    +            double score = entry.getValue() / maxLatency;
    +            // finally, add the severity without any weighting, since hosts scale this relative to their own load and the size of the task causing the severity.
    +            // "Severity" is basically a measure of compaction activity (CASSANDRA-3722).
    +            if (USE_SEVERITY)
    +                score += getSeverity(entry.getKey());
    +            // lowest score (least amount of badness) wins.
    +            newScores.put(entry.getKey(), score);
    +        }
    +
    +        return newScores;
         }
     
    -    private void registerMBean()
    +    /**
    +     * Background task running on the samples dictionary. The default implementation sends latency probes (PING)
    +     * messages to explore nodes that we have not received timings for recently but have ranked in
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}.
    +     */
    +    protected void updateSamples()
         {
    -        MBeanWrapper.instance.registerMBean(this, mbeanName);
    +        long updateIntervalMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - lastUpdateSamplesNanos);
    +        lastUpdateSamplesNanos = System.nanoTime();
    +
    +        // Split calculation of probe timers from sending probes for testability
    +        calculateProbes(samples, updateIntervalMillis);
    +
    +        // We do this after the calculations so that the progression of the logical clocks continues regardless
    +        // if gossip is enabled or not. However if Gossip is not active we don't _send_ the probes
    +        if (!StorageService.instance.isGossipActive())
    +            return;
    +
    +        scheduleProbes(samples);
    +    }
    +
    +    /**
    +     * This method mutates the passed AnnotatedMeasurements to implement capped exponential backoff per endpoint.
    +     *
    +     * The algorithm is as follows:
    +     * 1. All samples get their millisSinceLastMeasure and millisSinceLastRequest fields
    +     *    incremented by the passed interval
    +     * 2. Any recently requested (ranked) endpoints that have not been measured recently (e.g. because the snitch
    +     *    has sent them no traffic) get probes with exponential backoff.
    +     *
    +     * The backoff is capped at MAX_PROBE_INTERVAL_MS. Furthermore the probes are stopped after
    +     * MAX_PROBE_INTERVAL_MS of no ranking requests as well.
    +     *
    +     * At the end of this method, any passed AnnotatedMeasurements that need latency probes will have non zero
    +     * nextProbeDelayMillis members set.
    +     */
    +    @VisibleForTesting
    +    protected static void calculateProbes(Map<InetAddressAndPort, AnnotatedMeasurement> samples, long intervalMillis) {
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry: samples.entrySet())
    +        {
    +            if (entry.getKey().equals(FBUtilities.getBroadcastAddressAndPort()))
    +                continue;
    +
    +            AnnotatedMeasurement measurement = entry.getValue();
    +            long lastMeasure = measurement.millisSinceLastMeasure.addAndGet(intervalMillis);
    +            long lastRequest = measurement.millisSinceLastRequest.addAndGet(intervalMillis);
    +
    +            if (lastMeasure >= MIN_PROBE_INTERVAL_MS && lastRequest < MAX_PROBE_INTERVAL_MS)
    +            {
    +                if (measurement.nextProbeDelayMillis == 0)
    +                {
    +                    measurement.nextProbeDelayMillis = intervalMillis;
    +                }
    +                else if (measurement.probeFuture != null && measurement.probeFuture.isDone())
    +                {
    +                    measurement.nextProbeDelayMillis = Math.min(MAX_PROBE_INTERVAL_MS, measurement.nextProbeDelayMillis * 2);
    +                }
    +            }
    +            else
    +            {
    +                measurement.nextProbeDelayMillis = 0;
    +            }
    +        }
         }
     
    -    public void close()
    +    @VisibleForTesting
    +    void scheduleProbes(Map<InetAddressAndPort, AnnotatedMeasurement> samples)
         {
    -        updateSchedular.cancel(false);
    -        resetSchedular.cancel(false);
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry: samples.entrySet())
    +        {
    +            AnnotatedMeasurement measurement = entry.getValue();
    +            if (measurement.millisSinceLastRequest.get() > MAX_PROBE_INTERVAL_MS &&
    +                !Gossiper.instance.isAlive(entry.getKey()))
    +            {
    +                samples.remove(entry.getKey());
    +                continue;
    +            }
     
    -        MBeanWrapper.instance.unregisterMBean(mbeanName);
    +            long delay = measurement.nextProbeDelayMillis;
    +            if (delay > 0 && (measurement.probeFuture == null || measurement.probeFuture.isDone()))
    +            {
    +                logger.trace("Scheduled latency probe against {} in {}ms", entry.getKey(), delay);
    +                measurement.probeFuture = latencyProbeExecutor.schedule(() -> sendPingMessageToPeer(entry.getKey()),
    +                                                                        delay, TimeUnit.MILLISECONDS);
    +            }
    +        }
    +    }
    +
    +    /**
    +     * Method that actually sends latency probes as PING messages. This is the only function in this class
    +     * that operates on the latencyProbeExecutor thread and it records the maximum latency between a small and large
    +     * message channel ping.
    +     */
    +    private void sendPingMessageToPeer(InetAddressAndPort to)
    +    {
    +        // This method may have been scheduled (a long time) before it executes, so have to do
    +        // some quick sanity checks before sending a message to this host
    +        if (!StorageService.instance.isGossipActive() || !Gossiper.instance.isAlive(to))
    +            return;
    +
    +        probeRateLimiter.acquire(dynamicSampleUpdateInterval);
    --- End diff --
    
    Reminder to fix the bug where this always does one a second.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r229479751
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -154,31 +326,203 @@ private void registerMBean()
     
         public void close()
         {
    -        updateSchedular.cancel(false);
    -        resetSchedular.cancel(false);
    +        if (updateScoresScheduler != null)
    +            updateScoresScheduler.cancel(false);
    +        if (updateSamplesScheduler != null)
    +            updateSamplesScheduler.cancel(false);
    +
    +        for (AnnotatedMeasurement measurement : samples.values())
    +        {
    +            if (measurement.probeFuture != null)
    +                measurement.probeFuture.cancel(false);
    +
    +            measurement.millisSinceLastMeasure.set(0);
    +            measurement.millisSinceLastRequest.set(MAX_PROBE_INTERVAL_MS);
    +            measurement.probeTimerMillis = 0;
    +        }
     
             MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
             try
             {
    -            mbs.unregisterMBean(new ObjectName(mbeanName));
    +            if (mbeanRegistered)
    +                mbs.unregisterMBean(new ObjectName(mbeanName));
             }
             catch (Exception e)
             {
                 throw new RuntimeException(e);
             }
         }
     
    +    /**
    +     * Background task running on the samples dictionary. The default implementation sends latency probes (PING)
    +     * messages to explore nodes that we have not received timings for recently but have ranked in
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}.
    +     */
    +    protected void updateSamples()
    +    {
    +        // Split calculation of probe timers from sending probes for testability
    +        calculateProbes(samples, dynamicLatencyProbeInterval);
    +
    +        if (!StorageService.instance.isGossipActive())
    +            return;
    +
    +        schedulePings(samples);
    +    }
    +
    +    /**
    +     * This method mutates the passed AnnotatedMeasurements to implement capped exponential backoff per endpoint.
    +     *
    +     * The algorithm is as follows:
    +     * 1. All samples get their millisSinceLastMeasure and millisSinceLastRequest fields
    +     *    incremented by the passed interval
    +     * 2. Any recently requested (ranked) endpoints that have not been measured recently (e.g. because the snitch
    +     *    has sent them no traffic) get probes with exponential backoff.
    +     *
    +     * The backoff is capped at MAX_PROBE_INTERVAL_MS. Furthermore the probes are stopped after
    +     * MAX_PROBE_INTERVAL_MS of no ranking requests as well.
    +     *
    +     * At the end of this method, any passed AnnotatedMeasurements that need latency probes will have non zero
    +     * probeTimerMillis members set.
    +     */
    +    @VisibleForTesting
    +    static void calculateProbes(Map<InetAddressAndPort, AnnotatedMeasurement> samples, long intervalMillis) {
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry: samples.entrySet())
    +        {
    +            if (entry.getKey().equals(FBUtilities.getBroadcastAddressAndPort()))
    +                continue;
    +
    +            AnnotatedMeasurement measurement = entry.getValue();
    +            long lastMeasure = measurement.millisSinceLastMeasure.getAndAdd(intervalMillis);
    +            long lastRequest = measurement.millisSinceLastRequest.getAndAdd(intervalMillis);
    +
    +            if (lastMeasure >= MIN_PROBE_INTERVAL_MS && lastRequest < MAX_PROBE_INTERVAL_MS)
    +            {
    +                if (measurement.probeTimerMillis == 0)
    +                {
    +                    measurement.probeTimerMillis = intervalMillis;
    +                }
    +                else if (measurement.probeFuture != null && measurement.probeFuture.isDone())
    +                {
    +                    measurement.probeTimerMillis = Math.min(MAX_PROBE_INTERVAL_MS, measurement.probeTimerMillis * 2);
    +                }
    +            }
    +            else
    +            {
    +                measurement.probeTimerMillis = 0;
    +            }
    +        }
    +    }
    +
    +    @VisibleForTesting
    +    void schedulePings(Map<InetAddressAndPort, AnnotatedMeasurement> samples)
    +    {
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry: samples.entrySet())
    +        {
    +            AnnotatedMeasurement measurement = entry.getValue();
    +            long delay = measurement.probeTimerMillis;
    +            long millisSinceLastRequest = measurement.millisSinceLastRequest.get();
    +
    +            if (millisSinceLastRequest > MAX_PROBE_INTERVAL_MS && !Gossiper.instance.isAlive(entry.getKey()))
    --- End diff --
    
    This way all of the global state (`gossiper`) and threading state (`latencyProbeExecutor`) are in this method instead of `calculateProbes`, which means that we can call `calculateProbes` without checking if the gossiper is active (to maintain the logical clock moving forward).
    
    If we want to calculate the elapse time as you suggest instead of relying on the scheduler to schedule us ~1s then this becomes less of a problem.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r226142857
  
    --- Diff: src/java/org/apache/cassandra/net/IAsyncCallback.java ---
    @@ -31,11 +31,23 @@
         void response(MessageIn<T> msg);
     
         /**
    +     * @deprecated Use {@link #latencyMeasurementType()}. This is left for backwards
    +     *             compatibility with existing IAsyncCallback implementations but will
    +     *             be removed at a future date.
          * @return true if this callback is on the read path and its latency should be
          * given as input to the dynamic snitch.
          */
    +    @Deprecated
    --- End diff --
    
    Ok, I've done the refactor. I'd like to keep it as a separate commit though so that backporting is easier if that's ok?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r240288561
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -21,133 +21,505 @@
     import java.net.InetAddress;
     import java.net.UnknownHostException;
     import java.util.*;
    +import java.util.concurrent.CancellationException;
     import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ExecutionException;
     import java.util.concurrent.ScheduledFuture;
     import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
    +import java.util.concurrent.atomic.AtomicLong;
     import java.util.stream.Collectors;
     
    -import com.codahale.metrics.ExponentiallyDecayingReservoir;
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.util.concurrent.Futures;
    +import com.google.common.util.concurrent.MoreExecutors;
    +import com.google.common.util.concurrent.RateLimiter;
    +import com.google.common.util.concurrent.SettableFuture;
     
    -import com.codahale.metrics.Snapshot;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
     import org.apache.cassandra.concurrent.ScheduledExecutors;
     import org.apache.cassandra.config.DatabaseDescriptor;
    +import org.apache.cassandra.exceptions.RequestFailureReason;
     import org.apache.cassandra.gms.ApplicationState;
     import org.apache.cassandra.gms.EndpointState;
     import org.apache.cassandra.gms.Gossiper;
     import org.apache.cassandra.gms.VersionedValue;
    +import org.apache.cassandra.locator.dynamicsnitch.DynamicEndpointSnitchHistogram;
    +import org.apache.cassandra.net.IAsyncCallbackWithFailure;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +import org.apache.cassandra.net.MessageIn;
    +import org.apache.cassandra.net.MessageOut;
     import org.apache.cassandra.net.MessagingService;
    +import org.apache.cassandra.net.PingMessage;
     import org.apache.cassandra.service.StorageService;
     import org.apache.cassandra.utils.FBUtilities;
     import org.apache.cassandra.utils.MBeanWrapper;
     
    +import static org.apache.cassandra.net.MessagingService.Verb.PING;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.LARGE_MESSAGE;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.SMALL_MESSAGE;
    +
    +
     /**
      * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector
    + * Note that the subclasses (e.g. {@link DynamicEndpointSnitchHistogram}) are responsible for actually measuring
    + * latency and providing an ISnitchMeasurement implementation back to this class.
      */
    -public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
    +public abstract class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
     {
    -    private static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    private static final Logger logger = LoggerFactory.getLogger(DynamicEndpointSnitch.class);
    +
    +    // Latency measurement and ranking. The samples contain latency measurements and the scores are used for ranking
    +    protected boolean registered = false;
    +    protected static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    protected volatile Map<InetAddressAndPort, Double> scores = new HashMap<>();
     
    -    private static final double ALPHA = 0.75; // set to 0.75 to make EDS more biased to towards the newer values
    -    private static final int WINDOW_SIZE = 100;
    +    protected final Map<InetAddressAndPort, AnnotatedMeasurement> samples = new ConcurrentHashMap<>();
     
    -    private volatile int dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    -    private volatile int dynamicResetInterval = DatabaseDescriptor.getDynamicResetInterval();
    -    private volatile double dynamicBadnessThreshold = DatabaseDescriptor.getDynamicBadnessThreshold();
    +    // Latency probe functionality for actively probing endpoints that we haven't measured recently but are ranking
    +    public static final long MAX_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_max_probe_interval_ms", 60 * 10 * 1000L);
    +    public static final long MIN_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_min_probe_interval_ms", 60 * 1000L) ;
    +    // The probe rate is set later when configuration is read in applyConfigChanges
    +    protected static final RateLimiter probeRateLimiter = RateLimiter.create(1);
    +    protected static final DebuggableScheduledThreadPoolExecutor latencyProbeExecutor = new DebuggableScheduledThreadPoolExecutor("LatencyProbes");
    +    private long lastUpdateSamplesNanos = System.nanoTime();
    +
    +    // User configuration of the snitch tunables
    +    protected volatile int dynamicUpdateInterval = -1;
    +    protected volatile int dynamicSampleUpdateInterval = -1;
    +    protected volatile double dynamicBadnessThreshold = 0;
     
         // the score for a merged set of endpoints must be this much worse than the score for separate endpoints to
         // warrant not merging two ranges into a single range
         private static final double RANGE_MERGING_PREFERENCE = 1.5;
     
         private String mbeanName;
    -    private boolean registered = false;
    -
    -    private volatile HashMap<InetAddressAndPort, Double> scores = new HashMap<>();
    -    private final ConcurrentHashMap<InetAddressAndPort, ExponentiallyDecayingReservoir> samples = new ConcurrentHashMap<>();
    +    private boolean mbeanRegistered = false;
     
         public final IEndpointSnitch subsnitch;
     
    -    private volatile ScheduledFuture<?> updateSchedular;
    -    private volatile ScheduledFuture<?> resetSchedular;
    -
    -    private final Runnable update;
    -    private final Runnable reset;
    +    private volatile ScheduledFuture<?> updateScoresScheduler = null;
    +    private volatile ScheduledFuture<?> updateSamplesScheduler = null;
     
         public DynamicEndpointSnitch(IEndpointSnitch snitch)
         {
             this(snitch, null);
         }
     
    -    public DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
    +    protected DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
         {
             mbeanName = "org.apache.cassandra.db:type=DynamicEndpointSnitch";
             if (instance != null)
                 mbeanName += ",instance=" + instance;
             subsnitch = snitch;
    -        update = new Runnable()
    +
    +        if (DatabaseDescriptor.isDaemonInitialized())
             {
    -            public void run()
    +            open();
    +        }
    +    }
    +
    +    /**
    +     * Update configuration of the background tasks and restart the various scheduler tasks
    +     * if the configured rates for these tasks have changed.
    +     */
    +    public void applyConfigChanges(int newDynamicUpdateInternal, int newDynamicSampleUpdateInterval, double newDynamicBadnessThreshold)
    +    {
    +        if (DatabaseDescriptor.isDaemonInitialized())
    +        {
    +            if (dynamicUpdateInterval != newDynamicUpdateInternal || updateScoresScheduler == null)
                 {
    -                updateScores();
    +                cancelAndWait(updateScoresScheduler, Math.max(1, dynamicUpdateInterval), TimeUnit.MILLISECONDS);
    +                updateScoresScheduler = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(this::updateScores, newDynamicUpdateInternal, newDynamicUpdateInternal, TimeUnit.MILLISECONDS);
                 }
    -        };
    -        reset = new Runnable()
    +
    +            if (dynamicSampleUpdateInterval != newDynamicSampleUpdateInterval || updateSamplesScheduler == null)
    +            {
    +                cancelAndWait(updateSamplesScheduler, Math.max(1, dynamicSampleUpdateInterval), TimeUnit.MILLISECONDS);
    +                if (newDynamicSampleUpdateInterval > 0)
    +                    updateSamplesScheduler = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(this::updateSamples, newDynamicSampleUpdateInterval, newDynamicSampleUpdateInterval, TimeUnit.MILLISECONDS);
    +            }
    +        }
    +
    +        dynamicUpdateInterval = newDynamicUpdateInternal;
    +        dynamicSampleUpdateInterval = newDynamicSampleUpdateInterval;
    +        dynamicBadnessThreshold = newDynamicBadnessThreshold;
    +
    +        if (dynamicSampleUpdateInterval > 0)
    +            probeRateLimiter.setRate(dynamicSampleUpdateInterval);
    +    }
    +
    +    public synchronized void open()
    +    {
    +        applyConfigChanges(DatabaseDescriptor.getDynamicUpdateInterval(),
    +                           DatabaseDescriptor.getDynamicSampleUpdateInterval(),
    +                           DatabaseDescriptor.getDynamicBadnessThreshold());
    +
    +        MBeanWrapper.instance.registerMBean(this, mbeanName);
    +        mbeanRegistered = true;
    +    }
    +
    +    public synchronized void close()
    +    {
    +        cancelAndWait(updateScoresScheduler, Math.max(1, dynamicUpdateInterval), TimeUnit.MILLISECONDS);
    +        cancelAndWait(updateSamplesScheduler, Math.max(1, dynamicSampleUpdateInterval), TimeUnit.MILLISECONDS);
    +        updateScoresScheduler = null;
    +        updateSamplesScheduler = null;
    +
    +        for (AnnotatedMeasurement measurement : samples.values())
             {
    -            public void run()
    +            cancelAndWait(measurement.probeFuture, PING.getTimeout(), TimeUnit.MILLISECONDS);
    +
    +            measurement.millisSinceLastMeasure.set(0);
    +            measurement.millisSinceLastRequest.set(MAX_PROBE_INTERVAL_MS);
    +            measurement.nextProbeDelayMillis = 0;
    +        }
    +
    +        if (mbeanRegistered)
    +            MBeanWrapper.instance.unregisterMBean(mbeanName);
    +
    +        mbeanRegistered = false;
    +    }
    +
    +    private static void cancelAndWait(ScheduledFuture future, long timeout, TimeUnit unit)
    +    {
    +        if (future != null)
    +        {
    +            future.cancel(false);
    +            try
                 {
    -                // we do this so that a host considered bad has a chance to recover, otherwise would we never try
    -                // to read from it, which would cause its score to never change
    -                reset();
    +                future.get(timeout, unit);
                 }
    -        };
    +            catch (CancellationException | InterruptedException | ExecutionException | TimeoutException ignored)
    +            {
    +                // Exception is expected to happen eventually due to the cancel -> get
    +            }
    +        }
    +    }
     
    -        if (DatabaseDescriptor.isDaemonInitialized())
    +    /**
    +     * Allows subclasses to inject new ways of measuring latency back to this abstract base class.
    +     */
    +    protected interface ISnitchMeasurement
    +    {
    +        void sample(long value);
    +        double measure();
    +        Iterable<Double> measurements();
    +    }
    +
    +    /**
    +     * Adds some boookeeping that the DES uses over top of the various metrics techniques used by the
    +     * implementations. This is used to allow CASSANDRA-14459 latency probes as well as further safe
    +     * experimentation on new latency measurement techniques in CASSANDRA-14817
    +     *
    +     * {@link AnnotatedMeasurement#millisSinceLastRequest} is set to zero through
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}. It defaults to
    +     * the maximum interval so that we only start probing once it has been ranked at least once
    +     * {@link AnnotatedMeasurement#millisSinceLastMeasure} is set to zero from
    +     * {@link DynamicEndpointSnitch#receiveTiming(InetAddressAndPort, long, LatencyMeasurementType)}.
    +     *
    +     * {@link AnnotatedMeasurement#millisSinceLastMeasure and {@link AnnotatedMeasurement#nextProbeDelayMillis }
    +     * are incremented via {@link DynamicEndpointSnitch#updateSamples()}
    +     */
    +    protected static class AnnotatedMeasurement
    +    {
    +        // Used to optimally target latency probes only on nodes that are both requested for ranking
    +        // and are not being measured. For example with token aware clients a large portion of the cluster will never
    +        // be ranked at all and therefore we won't probe them.
    +        public AtomicLong millisSinceLastRequest = new AtomicLong(MAX_PROBE_INTERVAL_MS);
    +        public AtomicLong millisSinceLastMeasure = new AtomicLong(0);
    +        public volatile long nextProbeDelayMillis = 0;
    +        public volatile ScheduledFuture<?> probeFuture = null;
    +
    +        // The underlying measurement technique. E.g. a median filter (histogram) or an EMA filter, or ...
    +        public final ISnitchMeasurement measurement;
    +        public volatile double cachedMeasurement;
    +
    +        public AnnotatedMeasurement(ISnitchMeasurement measurement)
             {
    -            updateSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(update, dynamicUpdateInterval, dynamicUpdateInterval, TimeUnit.MILLISECONDS);
    -            resetSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(reset, dynamicResetInterval, dynamicResetInterval, TimeUnit.MILLISECONDS);
    -            registerMBean();
    +            this.measurement = measurement;
    +            this.cachedMeasurement = measurement.measure();
    +        }
    +
    +        @Override
    +        public String toString()
    +        {
    +            return "AnnotatedMeasurement{" +
    +                   "millisSinceLastRequest=" + millisSinceLastRequest +
    +                   ", millisSinceLastMeasure=" + millisSinceLastMeasure +
    +                   ", nextProbeDelayMillis=" + nextProbeDelayMillis +
    +                   ", probeFuturePending=" + (probeFuture != null && !probeFuture.isDone()) +
    +                   ", measurementClass=" + measurement.getClass().getSimpleName() +
    +                   ", cachedMeasurement=" + cachedMeasurement +
    +                   '}';
             }
         }
     
         /**
    -     * Update configuration from {@link DatabaseDescriptor} and estart the update-scheduler and reset-scheduler tasks
    -     * if the configured rates for these tasks have changed.
    +     * Allows the subclasses to inject ISnitchMeasurement instances back into this common class so that common
    +     * functionality can be handled here
    +     * @param initialValue The initial value of the measurement, some implementations may use this, others may not
    +     * @return a constructed instance of an ISnitchMeasurement interface
          */
    -    public void applyConfigChanges()
    +    abstract protected ISnitchMeasurement measurementImpl(long initialValue);
    +
    +    /**
    +     * Records a latency. This MUST be cheap as it is called in the fast path
    +     */
    +    public void receiveTiming(InetAddressAndPort address, long latencyMicros, LatencyMeasurementType measurementType)
    +    {
    +        if (measurementType == LatencyMeasurementType.IGNORE)
    +           return;
    +
    +        AnnotatedMeasurement sample = samples.get(address);
    +
    +        if (sample == null)
    +        {
    +            AnnotatedMeasurement maybeNewSample = new AnnotatedMeasurement(measurementImpl(latencyMicros));
    +            sample = samples.putIfAbsent(address, maybeNewSample);
    +            if (sample == null)
    +                sample = maybeNewSample;
    +        }
    +
    +        if (measurementType == LatencyMeasurementType.READ && sample.millisSinceLastMeasure.get() > 0)
    +            sample.millisSinceLastMeasure.lazySet(0);
    +
    +        sample.measurement.sample(latencyMicros);
    +    }
    +
    +    @VisibleForTesting
    +    protected void reset()
    +    {
    +        scores.clear();
    +        samples.clear();
    +    }
    +
    +    @VisibleForTesting
    +    protected void updateScores()
         {
    -        if (dynamicUpdateInterval != DatabaseDescriptor.getDynamicUpdateInterval())
    +        if (!StorageService.instance.isGossipActive())
    +            return;
    +
    +        if (!registered)
             {
    -            dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    -            if (DatabaseDescriptor.isDaemonInitialized())
    +            if (MessagingService.instance() != null)
                 {
    -                updateSchedular.cancel(false);
    -                updateSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(update, dynamicUpdateInterval, dynamicUpdateInterval, TimeUnit.MILLISECONDS);
    +                MessagingService.instance().register(this);
    +                registered = true;
                 }
             }
     
    -        if (dynamicResetInterval != DatabaseDescriptor.getDynamicResetInterval())
    +        this.scores = calculateScores(samples);
    +    }
    +
    +    /**
    +     * This is generally expensive and is called periodically (semi-frequently) not on the fast path.
    +     * The main concern here is generating garbage from the measurements (e.g. histograms in particular)
    +     */
    +    @VisibleForTesting
    +    protected static Map<InetAddressAndPort, Double> calculateScores(Map<InetAddressAndPort, AnnotatedMeasurement> samples)
    +    {
    +        double maxLatency = 1;
    +        HashMap<InetAddressAndPort, Double> newScores = new HashMap<>(samples.size());
    +
    +        // We're going to weight the latency for each host against the worst one we see, to
    +        // arrive at sort of a 'badness percentage' for them. First, find the worst latency for each:
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry : samples.entrySet())
             {
    -            dynamicResetInterval = DatabaseDescriptor.getDynamicResetInterval();
    -            if (DatabaseDescriptor.isDaemonInitialized())
    +            AnnotatedMeasurement annotatedMeasurement = entry.getValue();
    +
    +            // only compute the measurement, which probably generates the most garbage (e.g. for this Histogram),
    +            // for endpoints that have been recently updated (millisSinceLastRequest) or somewhat recently requested
    +            // for ranking.
    +            if (annotatedMeasurement.millisSinceLastMeasure.get() < MIN_PROBE_INTERVAL_MS ||
    --- End diff --
    
    Do we need to check millisSinceLast measure here? We could only check since last request. If we have requested the value then we know independently it will be either probed or getting real measurements so it is worth calculating the scores.
    
    I think a comment to that effect and a simpler condition would make this easier to reason about.



---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225334814
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -210,8 +368,10 @@ public String getDatacenter(InetAddressAndPort endpoint)
     
             // TODO: avoid copy
             replicas = subsnitch.sortedByProximity(address, replicas);
    -        HashMap<InetAddressAndPort, Double> scores = this.scores; // Make sure the score don't change in the middle of the loop below
    -                                                           // (which wouldn't really matter here but its cleaner that way).
    +        // Make sure the score don't change in the middle of the loop below
    +        // (which wouldn't really matter here but its cleaner that way).
    +        final Map<InetAddressAndPort, Double> scores = this.scores;
    --- End diff --
    
    I don't think we generally do final on the stack or for parameters to functions. I can't find anything in the style guide that says you can't do it, but I recall being told that at one point. I'll ask around.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r236752030
  
    --- Diff: test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java ---
    @@ -60,57 +130,229 @@ private static EndpointsForRange full(InetAddressAndPort... endpoints)
         }
     
         @Test
    -    public void testSnitch() throws InterruptedException, IOException, ConfigurationException
    +    public void testSortedByProximity() throws InterruptedException, IOException, ConfigurationException
         {
    -        // do this because SS needs to be initialized before DES can work properly.
    -        StorageService.instance.unsafeInitialize();
    -        SimpleSnitch ss = new SimpleSnitch();
    -        DynamicEndpointSnitch dsnitch = new DynamicEndpointSnitch(ss, String.valueOf(ss.hashCode()));
    -        InetAddressAndPort self = FBUtilities.getBroadcastAddressAndPort();
    -        InetAddressAndPort host1 = InetAddressAndPort.getByName("127.0.0.2");
    -        InetAddressAndPort host2 = InetAddressAndPort.getByName("127.0.0.3");
    -        InetAddressAndPort host3 = InetAddressAndPort.getByName("127.0.0.4");
    -        InetAddressAndPort host4 = InetAddressAndPort.getByName("127.0.0.5");
    -        List<InetAddressAndPort> hosts = Arrays.asList(host1, host2, host3);
    +        InetAddressAndPort self = hosts[0];
    +        List<InetAddressAndPort> allHosts = Arrays.asList(hosts[1], hosts[2], hosts[3]);
     
             // first, make all hosts equal
    -        setScores(dsnitch, 1, hosts, 10, 10, 10);
    -        EndpointsForRange order = full(host1, host2, host3);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    -
    -        // make host1 a little worse
    -        setScores(dsnitch, 1, hosts, 20, 10, 10);
    -        order = full(host2, host3, host1);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    -
    -        // make host2 as bad as host1
    -        setScores(dsnitch, 2, hosts, 15, 20, 10);
    -        order = full(host3, host1, host2);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    -
    -        // make host3 the worst
    -        setScores(dsnitch, 3, hosts, 10, 10, 30);
    -        order = full(host1, host2, host3);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    -
    -        // make host3 equal to the others
    -        setScores(dsnitch, 5, hosts, 10, 10, 10);
    -        order = full(host1, host2, host3);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    +        setScores(dsnitch, 1, allHosts, 10, 10, 10);
    +        EndpointsForRange order = full(hosts[1], hosts[2], hosts[3]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        // make hosts[1] a little worse
    +        setScores(dsnitch, 2, allHosts, 20, 10, 10);
    +        order = full(hosts[2], hosts[3], hosts[1]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        // make hosts[2] as bad as hosts[1]
    +        setScores(dsnitch, 4, allHosts, 15, 20, 10);
    +        order = full(hosts[3], hosts[1], hosts[2]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        // make hosts[3] the worst
    +        setScores(dsnitch, 10, allHosts, 10, 10, 30);
    +        order = full(hosts[1], hosts[2], hosts[3]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        // make hosts[3] equal to the others
    +        setScores(dsnitch, 15, allHosts, 10, 10, 10);
    +        order = full(hosts[1], hosts[2], hosts[3]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
     
             /// Tests CASSANDRA-6683 improvements
             // make the scores differ enough from the ideal order that we sort by score; under the old
             // dynamic snitch behavior (where we only compared neighbors), these wouldn't get sorted
    -        setScores(dsnitch, 20, hosts, 10, 70, 20);
    -        order = full(host1, host3, host2);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    +        dsnitch.reset();
    +        setScores(dsnitch, 20, allHosts, 10, 70, 20);
    +        order = full(hosts[1], hosts[3], hosts[2]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        order = full(hosts[4], hosts[1], hosts[3], hosts[2]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3], hosts[4])));
    +
    +        setScores(dsnitch, 40, allHosts, 10, 10, 11);
    +        order = full(hosts[4], hosts[1], hosts[2], hosts[3]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3], hosts[4])));
    +    }
    +
    +    // CASSANDRA-14459 improvements to add latency probes instead of resets
    +    @Test
    +    public void testLatencyProbeNeeded()
    +    {
    +        InetAddressAndPort self = hosts[0];
    +
    +        // Four hosts, two are very latent, but all are requested for ranking
    +        dsnitch.receiveTiming(hosts[1], 20, LatencyMeasurementType.READ);
    +        dsnitch.receiveTiming(hosts[2], 10, LatencyMeasurementType.READ);
    +        dsnitch.receiveTiming(hosts[3], 1000, LatencyMeasurementType.READ);
    +        dsnitch.receiveTiming(hosts[4], 1000, LatencyMeasurementType.READ);
    +        dsnitch.updateScores();
    +
    +        EndpointsForRange orderBefore = full(hosts[2], hosts[1], hosts[3], hosts[4]);
    +
    +        Map<InetAddressAndPort, DynamicEndpointSnitch.AnnotatedMeasurement> measurements = dsnitch.getMeasurementsWithPort();
    +        // At this point we haven't ranked any of hosts so we should need no probes even
    +        DynamicEndpointSnitch.calculateProbes(measurements, dsnitch.dynamicLatencyProbeInterval);
    +        assertFalse(measurements.values().stream().anyMatch(m -> m.probeTimerMillis > 0));
    +        assertTrue(measurements.values().stream().allMatch(m -> m.probeFuture == null));
    +
    +        // Two hosts continue receiving traffic but the last two are always ranked by the snitch
    +        for (int i = 0; i < 10; i++)
    +        {
    +            dsnitch.receiveTiming(hosts[1], 20, LatencyMeasurementType.READ);
    +            dsnitch.receiveTiming(hosts[2], 10, LatencyMeasurementType.READ);
    +            Util.assertRCEquals(orderBefore, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3], hosts[4])));
    +        }
    +        dsnitch.updateScores();
    +
    +        int probesBefore = (int) ((DynamicEndpointSnitch.MIN_PROBE_INTERVAL_MS / dsnitch.dynamicLatencyProbeInterval) + 1);
    +
    +        for (int i = 0; i < probesBefore; i++)
    +            DynamicEndpointSnitch.calculateProbes(measurements, dsnitch.dynamicLatencyProbeInterval);
    +        assertEquals(2, measurements.values().stream().filter(m -> m.probeTimerMillis > 0).count());
    +        assertEquals(0, measurements.values().stream().filter(m -> m.probeFuture != null).count());
    +
    +        // Both requested but non measured hosts should have single interval timers set
    +        assertEquals(dsnitch.dynamicLatencyProbeInterval, measurements.get(hosts[3]).probeTimerMillis);
    +        assertEquals(dsnitch.dynamicLatencyProbeInterval, measurements.get(hosts[4]).probeTimerMillis);
    +
    +        dsnitch.schedulePings(measurements);
    +        // We should have two scheduled futures now
    +        assertEquals(2, measurements.values().stream().filter(m -> m.probeFuture != null).count());
    +
    +        int numProbes = 0;
    +        for (int i = 1; (1 << i) * dsnitch.dynamicLatencyProbeInterval < DynamicEndpointSnitch.MAX_PROBE_INTERVAL_MS; i++)
    +        {
    +            DynamicEndpointSnitch.AnnotatedMeasurement m3 = measurements.get(hosts[3]);
    +            DynamicEndpointSnitch.AnnotatedMeasurement m4 = measurements.get(hosts[4]);
    +            // Emulate the probe actually running
    +            m3.probeFuture.cancel(false);
    +            m4.probeFuture.cancel(false);
    +            DynamicEndpointSnitch.calculateProbes(measurements, dsnitch.dynamicLatencyProbeInterval);
    +            dsnitch.schedulePings(measurements);
    +            assertEquals((1 << i) * dsnitch.dynamicLatencyProbeInterval, m3.probeTimerMillis);
    +            assertEquals((1 << i) * dsnitch.dynamicLatencyProbeInterval, m4.probeTimerMillis);
    +            numProbes++;
    +        }
    +
    +        // If we did exponential backoff right ... we should have the right number of probes
    +        assertTrue(numProbes > 0);
    +        assertEquals(numProbes,
    +                     Math.floor(Math.log(DynamicEndpointSnitch.MAX_PROBE_INTERVAL_MS / dsnitch.dynamicLatencyProbeInterval) / Math.log(2.0)),
    +                     0.01);
    +
    +        measurements.get(hosts[3]).probeFuture.cancel(false);
    +        measurements.get(hosts[4]).probeFuture.cancel(false);
    +
    +        // Now we should do ten minute probes until we exceed MAX_PROBE_INTERVAL_MS since we last requested ranking
    +        int interval = dsnitch.dynamicLatencyProbeInterval;
    +        for (int i = (probesBefore + numProbes) * interval; i < DynamicEndpointSnitch.MAX_PROBE_INTERVAL_MS; i += interval)
    +        {
    +            DynamicEndpointSnitch.calculateProbes(measurements, dsnitch.dynamicLatencyProbeInterval);
    +            assertEquals(DynamicEndpointSnitch.MAX_PROBE_INTERVAL_MS, measurements.get(hosts[3]).probeTimerMillis);
    +            assertEquals(DynamicEndpointSnitch.MAX_PROBE_INTERVAL_MS, measurements.get(hosts[4]).probeTimerMillis);
    +            dsnitch.schedulePings(measurements);
    +            measurements.get(hosts[3]).probeFuture.cancel(false);
    +            measurements.get(hosts[4]).probeFuture.cancel(false);
    +        }
    +
    +        // Since we haven't measured, we should stop probing
    --- End diff --
    
    Correct, typo on my part. Fixed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r229459070
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -154,31 +326,203 @@ private void registerMBean()
     
         public void close()
         {
    -        updateSchedular.cancel(false);
    -        resetSchedular.cancel(false);
    +        if (updateScoresScheduler != null)
    +            updateScoresScheduler.cancel(false);
    +        if (updateSamplesScheduler != null)
    +            updateSamplesScheduler.cancel(false);
    +
    +        for (AnnotatedMeasurement measurement : samples.values())
    +        {
    +            if (measurement.probeFuture != null)
    +                measurement.probeFuture.cancel(false);
    +
    +            measurement.millisSinceLastMeasure.set(0);
    +            measurement.millisSinceLastRequest.set(MAX_PROBE_INTERVAL_MS);
    +            measurement.probeTimerMillis = 0;
    +        }
     
             MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
             try
             {
    -            mbs.unregisterMBean(new ObjectName(mbeanName));
    +            if (mbeanRegistered)
    +                mbs.unregisterMBean(new ObjectName(mbeanName));
             }
             catch (Exception e)
             {
                 throw new RuntimeException(e);
             }
         }
     
    +    /**
    +     * Background task running on the samples dictionary. The default implementation sends latency probes (PING)
    +     * messages to explore nodes that we have not received timings for recently but have ranked in
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}.
    +     */
    +    protected void updateSamples()
    +    {
    +        // Split calculation of probe timers from sending probes for testability
    +        calculateProbes(samples, dynamicLatencyProbeInterval);
    +
    +        if (!StorageService.instance.isGossipActive())
    +            return;
    +
    +        schedulePings(samples);
    +    }
    +
    +    /**
    +     * This method mutates the passed AnnotatedMeasurements to implement capped exponential backoff per endpoint.
    +     *
    +     * The algorithm is as follows:
    +     * 1. All samples get their millisSinceLastMeasure and millisSinceLastRequest fields
    +     *    incremented by the passed interval
    +     * 2. Any recently requested (ranked) endpoints that have not been measured recently (e.g. because the snitch
    +     *    has sent them no traffic) get probes with exponential backoff.
    +     *
    +     * The backoff is capped at MAX_PROBE_INTERVAL_MS. Furthermore the probes are stopped after
    +     * MAX_PROBE_INTERVAL_MS of no ranking requests as well.
    +     *
    +     * At the end of this method, any passed AnnotatedMeasurements that need latency probes will have non zero
    +     * probeTimerMillis members set.
    +     */
    +    @VisibleForTesting
    +    static void calculateProbes(Map<InetAddressAndPort, AnnotatedMeasurement> samples, long intervalMillis) {
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry: samples.entrySet())
    +        {
    +            if (entry.getKey().equals(FBUtilities.getBroadcastAddressAndPort()))
    +                continue;
    +
    +            AnnotatedMeasurement measurement = entry.getValue();
    +            long lastMeasure = measurement.millisSinceLastMeasure.getAndAdd(intervalMillis);
    +            long lastRequest = measurement.millisSinceLastRequest.getAndAdd(intervalMillis);
    +
    +            if (lastMeasure >= MIN_PROBE_INTERVAL_MS && lastRequest < MAX_PROBE_INTERVAL_MS)
    +            {
    +                if (measurement.probeTimerMillis == 0)
    +                {
    +                    measurement.probeTimerMillis = intervalMillis;
    +                }
    +                else if (measurement.probeFuture != null && measurement.probeFuture.isDone())
    +                {
    +                    measurement.probeTimerMillis = Math.min(MAX_PROBE_INTERVAL_MS, measurement.probeTimerMillis * 2);
    +                }
    +            }
    +            else
    +            {
    +                measurement.probeTimerMillis = 0;
    +            }
    +        }
    +    }
    +
    +    @VisibleForTesting
    +    void schedulePings(Map<InetAddressAndPort, AnnotatedMeasurement> samples)
    +    {
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry: samples.entrySet())
    +        {
    +            AnnotatedMeasurement measurement = entry.getValue();
    +            long delay = measurement.probeTimerMillis;
    +            long millisSinceLastRequest = measurement.millisSinceLastRequest.get();
    +
    +            if (millisSinceLastRequest > MAX_PROBE_INTERVAL_MS && !Gossiper.instance.isAlive(entry.getKey()))
    --- End diff --
    
    Nit but this feels like it should be in calculateProbes


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r229459897
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -154,31 +326,203 @@ private void registerMBean()
     
         public void close()
         {
    -        updateSchedular.cancel(false);
    -        resetSchedular.cancel(false);
    +        if (updateScoresScheduler != null)
    +            updateScoresScheduler.cancel(false);
    +        if (updateSamplesScheduler != null)
    +            updateSamplesScheduler.cancel(false);
    +
    +        for (AnnotatedMeasurement measurement : samples.values())
    +        {
    +            if (measurement.probeFuture != null)
    +                measurement.probeFuture.cancel(false);
    +
    +            measurement.millisSinceLastMeasure.set(0);
    +            measurement.millisSinceLastRequest.set(MAX_PROBE_INTERVAL_MS);
    +            measurement.probeTimerMillis = 0;
    +        }
     
             MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
             try
             {
    -            mbs.unregisterMBean(new ObjectName(mbeanName));
    +            if (mbeanRegistered)
    +                mbs.unregisterMBean(new ObjectName(mbeanName));
             }
             catch (Exception e)
             {
                 throw new RuntimeException(e);
             }
         }
     
    +    /**
    +     * Background task running on the samples dictionary. The default implementation sends latency probes (PING)
    +     * messages to explore nodes that we have not received timings for recently but have ranked in
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}.
    +     */
    +    protected void updateSamples()
    +    {
    +        // Split calculation of probe timers from sending probes for testability
    +        calculateProbes(samples, dynamicLatencyProbeInterval);
    +
    +        if (!StorageService.instance.isGossipActive())
    +            return;
    +
    +        schedulePings(samples);
    +    }
    +
    +    /**
    +     * This method mutates the passed AnnotatedMeasurements to implement capped exponential backoff per endpoint.
    +     *
    +     * The algorithm is as follows:
    +     * 1. All samples get their millisSinceLastMeasure and millisSinceLastRequest fields
    +     *    incremented by the passed interval
    +     * 2. Any recently requested (ranked) endpoints that have not been measured recently (e.g. because the snitch
    +     *    has sent them no traffic) get probes with exponential backoff.
    +     *
    +     * The backoff is capped at MAX_PROBE_INTERVAL_MS. Furthermore the probes are stopped after
    +     * MAX_PROBE_INTERVAL_MS of no ranking requests as well.
    +     *
    +     * At the end of this method, any passed AnnotatedMeasurements that need latency probes will have non zero
    +     * probeTimerMillis members set.
    +     */
    +    @VisibleForTesting
    +    static void calculateProbes(Map<InetAddressAndPort, AnnotatedMeasurement> samples, long intervalMillis) {
    --- End diff --
    
    Paranoia would be to calculate the actual elapsed interval. 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r239906662
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -21,133 +21,505 @@
     import java.net.InetAddress;
     import java.net.UnknownHostException;
     import java.util.*;
    +import java.util.concurrent.CancellationException;
     import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ExecutionException;
     import java.util.concurrent.ScheduledFuture;
     import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
    +import java.util.concurrent.atomic.AtomicLong;
     import java.util.stream.Collectors;
     
    -import com.codahale.metrics.ExponentiallyDecayingReservoir;
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.util.concurrent.Futures;
    +import com.google.common.util.concurrent.MoreExecutors;
    +import com.google.common.util.concurrent.RateLimiter;
    +import com.google.common.util.concurrent.SettableFuture;
     
    -import com.codahale.metrics.Snapshot;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
     import org.apache.cassandra.concurrent.ScheduledExecutors;
     import org.apache.cassandra.config.DatabaseDescriptor;
    +import org.apache.cassandra.exceptions.RequestFailureReason;
     import org.apache.cassandra.gms.ApplicationState;
     import org.apache.cassandra.gms.EndpointState;
     import org.apache.cassandra.gms.Gossiper;
     import org.apache.cassandra.gms.VersionedValue;
    +import org.apache.cassandra.locator.dynamicsnitch.DynamicEndpointSnitchHistogram;
    +import org.apache.cassandra.net.IAsyncCallbackWithFailure;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +import org.apache.cassandra.net.MessageIn;
    +import org.apache.cassandra.net.MessageOut;
     import org.apache.cassandra.net.MessagingService;
    +import org.apache.cassandra.net.PingMessage;
     import org.apache.cassandra.service.StorageService;
     import org.apache.cassandra.utils.FBUtilities;
     import org.apache.cassandra.utils.MBeanWrapper;
     
    +import static org.apache.cassandra.net.MessagingService.Verb.PING;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.LARGE_MESSAGE;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.SMALL_MESSAGE;
    +
    +
     /**
      * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector
    + * Note that the subclasses (e.g. {@link DynamicEndpointSnitchHistogram}) are responsible for actually measuring
    + * latency and providing an ISnitchMeasurement implementation back to this class.
      */
    -public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
    +public abstract class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
     {
    -    private static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    private static final Logger logger = LoggerFactory.getLogger(DynamicEndpointSnitch.class);
    +
    +    // Latency measurement and ranking. The samples contain latency measurements and the scores are used for ranking
    +    protected boolean registered = false;
    +    protected static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    protected volatile Map<InetAddressAndPort, Double> scores = new HashMap<>();
     
    -    private static final double ALPHA = 0.75; // set to 0.75 to make EDS more biased to towards the newer values
    -    private static final int WINDOW_SIZE = 100;
    +    protected final Map<InetAddressAndPort, AnnotatedMeasurement> samples = new ConcurrentHashMap<>();
     
    -    private volatile int dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    -    private volatile int dynamicResetInterval = DatabaseDescriptor.getDynamicResetInterval();
    -    private volatile double dynamicBadnessThreshold = DatabaseDescriptor.getDynamicBadnessThreshold();
    +    // Latency probe functionality for actively probing endpoints that we haven't measured recently but are ranking
    +    public static final long MAX_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_max_probe_interval_ms", 60 * 10 * 1000L);
    +    public static final long MIN_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_min_probe_interval_ms", 60 * 1000L) ;
    +    // The probe rate is set later when configuration is read in applyConfigChanges
    +    protected static final RateLimiter probeRateLimiter = RateLimiter.create(1);
    +    protected static final DebuggableScheduledThreadPoolExecutor latencyProbeExecutor = new DebuggableScheduledThreadPoolExecutor("LatencyProbes");
    +    private long lastUpdateSamplesNanos = System.nanoTime();
    +
    +    // User configuration of the snitch tunables
    +    protected volatile int dynamicUpdateInterval = -1;
    +    protected volatile int dynamicSampleUpdateInterval = -1;
    +    protected volatile double dynamicBadnessThreshold = 0;
     
         // the score for a merged set of endpoints must be this much worse than the score for separate endpoints to
         // warrant not merging two ranges into a single range
         private static final double RANGE_MERGING_PREFERENCE = 1.5;
     
         private String mbeanName;
    -    private boolean registered = false;
    -
    -    private volatile HashMap<InetAddressAndPort, Double> scores = new HashMap<>();
    -    private final ConcurrentHashMap<InetAddressAndPort, ExponentiallyDecayingReservoir> samples = new ConcurrentHashMap<>();
    +    private boolean mbeanRegistered = false;
     
         public final IEndpointSnitch subsnitch;
     
    -    private volatile ScheduledFuture<?> updateSchedular;
    -    private volatile ScheduledFuture<?> resetSchedular;
    -
    -    private final Runnable update;
    -    private final Runnable reset;
    +    private volatile ScheduledFuture<?> updateScoresScheduler = null;
    +    private volatile ScheduledFuture<?> updateSamplesScheduler = null;
     
         public DynamicEndpointSnitch(IEndpointSnitch snitch)
         {
             this(snitch, null);
         }
     
    -    public DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
    +    protected DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
         {
             mbeanName = "org.apache.cassandra.db:type=DynamicEndpointSnitch";
             if (instance != null)
                 mbeanName += ",instance=" + instance;
             subsnitch = snitch;
    -        update = new Runnable()
    +
    +        if (DatabaseDescriptor.isDaemonInitialized())
             {
    -            public void run()
    +            open();
    +        }
    +    }
    +
    +    /**
    +     * Update configuration of the background tasks and restart the various scheduler tasks
    +     * if the configured rates for these tasks have changed.
    +     */
    +    public void applyConfigChanges(int newDynamicUpdateInternal, int newDynamicSampleUpdateInterval, double newDynamicBadnessThreshold)
    +    {
    +        if (DatabaseDescriptor.isDaemonInitialized())
    +        {
    +            if (dynamicUpdateInterval != newDynamicUpdateInternal || updateScoresScheduler == null)
                 {
    -                updateScores();
    +                cancelAndWait(updateScoresScheduler, Math.max(1, dynamicUpdateInterval), TimeUnit.MILLISECONDS);
    --- End diff --
    
    When I am doing termination I prefer to block for 365 days. Either you need to block and wait for it to complete (or it's a bug) or you don't in which case you can cancel and move on. In between doesn't make much sense to me.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r229461114
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -154,31 +326,203 @@ private void registerMBean()
     
         public void close()
         {
    -        updateSchedular.cancel(false);
    -        resetSchedular.cancel(false);
    +        if (updateScoresScheduler != null)
    +            updateScoresScheduler.cancel(false);
    +        if (updateSamplesScheduler != null)
    +            updateSamplesScheduler.cancel(false);
    +
    +        for (AnnotatedMeasurement measurement : samples.values())
    +        {
    +            if (measurement.probeFuture != null)
    +                measurement.probeFuture.cancel(false);
    +
    +            measurement.millisSinceLastMeasure.set(0);
    +            measurement.millisSinceLastRequest.set(MAX_PROBE_INTERVAL_MS);
    +            measurement.probeTimerMillis = 0;
    +        }
     
             MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
             try
             {
    -            mbs.unregisterMBean(new ObjectName(mbeanName));
    +            if (mbeanRegistered)
    +                mbs.unregisterMBean(new ObjectName(mbeanName));
             }
             catch (Exception e)
             {
                 throw new RuntimeException(e);
             }
         }
     
    +    /**
    +     * Background task running on the samples dictionary. The default implementation sends latency probes (PING)
    +     * messages to explore nodes that we have not received timings for recently but have ranked in
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}.
    +     */
    +    protected void updateSamples()
    +    {
    +        // Split calculation of probe timers from sending probes for testability
    +        calculateProbes(samples, dynamicLatencyProbeInterval);
    +
    +        if (!StorageService.instance.isGossipActive())
    +            return;
    +
    +        schedulePings(samples);
    +    }
    +
    +    /**
    +     * This method mutates the passed AnnotatedMeasurements to implement capped exponential backoff per endpoint.
    +     *
    +     * The algorithm is as follows:
    +     * 1. All samples get their millisSinceLastMeasure and millisSinceLastRequest fields
    +     *    incremented by the passed interval
    +     * 2. Any recently requested (ranked) endpoints that have not been measured recently (e.g. because the snitch
    +     *    has sent them no traffic) get probes with exponential backoff.
    +     *
    +     * The backoff is capped at MAX_PROBE_INTERVAL_MS. Furthermore the probes are stopped after
    +     * MAX_PROBE_INTERVAL_MS of no ranking requests as well.
    +     *
    +     * At the end of this method, any passed AnnotatedMeasurements that need latency probes will have non zero
    +     * probeTimerMillis members set.
    +     */
    +    @VisibleForTesting
    +    static void calculateProbes(Map<InetAddressAndPort, AnnotatedMeasurement> samples, long intervalMillis) {
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry: samples.entrySet())
    +        {
    +            if (entry.getKey().equals(FBUtilities.getBroadcastAddressAndPort()))
    +                continue;
    +
    +            AnnotatedMeasurement measurement = entry.getValue();
    +            long lastMeasure = measurement.millisSinceLastMeasure.getAndAdd(intervalMillis);
    +            long lastRequest = measurement.millisSinceLastRequest.getAndAdd(intervalMillis);
    +
    +            if (lastMeasure >= MIN_PROBE_INTERVAL_MS && lastRequest < MAX_PROBE_INTERVAL_MS)
    +            {
    +                if (measurement.probeTimerMillis == 0)
    +                {
    +                    measurement.probeTimerMillis = intervalMillis;
    +                }
    +                else if (measurement.probeFuture != null && measurement.probeFuture.isDone())
    +                {
    +                    measurement.probeTimerMillis = Math.min(MAX_PROBE_INTERVAL_MS, measurement.probeTimerMillis * 2);
    +                }
    +            }
    +            else
    +            {
    +                measurement.probeTimerMillis = 0;
    +            }
    +        }
    +    }
    +
    +    @VisibleForTesting
    +    void schedulePings(Map<InetAddressAndPort, AnnotatedMeasurement> samples)
    +    {
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry: samples.entrySet())
    +        {
    +            AnnotatedMeasurement measurement = entry.getValue();
    +            long delay = measurement.probeTimerMillis;
    +            long millisSinceLastRequest = measurement.millisSinceLastRequest.get();
    +
    +            if (millisSinceLastRequest > MAX_PROBE_INTERVAL_MS && !Gossiper.instance.isAlive(entry.getKey()))
    +            {
    +                samples.remove(entry.getKey());
    +            }
    +
    +            if (delay > 0 && millisSinceLastRequest < MAX_PROBE_INTERVAL_MS &&
    --- End diff --
    
    Can remove the millis since last request check if you remove it in calculateProbes


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225329424
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -22,121 +22,194 @@
     import java.net.InetAddress;
     import java.net.UnknownHostException;
     import java.util.*;
    -import java.util.concurrent.ConcurrentHashMap;
     import java.util.concurrent.ScheduledFuture;
     import java.util.concurrent.TimeUnit;
     import java.util.stream.Collectors;
     
    -import com.codahale.metrics.ExponentiallyDecayingReservoir;
     import javax.management.MBeanServer;
     import javax.management.ObjectName;
     
    -import com.codahale.metrics.Snapshot;
    +import com.google.common.annotations.VisibleForTesting;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
     import org.apache.cassandra.concurrent.ScheduledExecutors;
     import org.apache.cassandra.config.DatabaseDescriptor;
     import org.apache.cassandra.gms.ApplicationState;
     import org.apache.cassandra.gms.EndpointState;
     import org.apache.cassandra.gms.Gossiper;
     import org.apache.cassandra.gms.VersionedValue;
    +import org.apache.cassandra.net.IAsyncCallback;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +import org.apache.cassandra.net.MessageIn;
    +import org.apache.cassandra.net.MessageOut;
     import org.apache.cassandra.net.MessagingService;
    +import org.apache.cassandra.net.PingMessage;
     import org.apache.cassandra.service.StorageService;
     import org.apache.cassandra.utils.FBUtilities;
     
    +import static org.apache.cassandra.net.MessagingService.Verb.PING;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.LARGE_MESSAGE;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.SMALL_MESSAGE;
    +
    +
     /**
      * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector
    + * Note that the subclasses (e.g. {@link DynamicEndpointSnitchHistogram}) are responsible for actually measuring
    + * latency and populating the {@link #scores} map.
      */
    -public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
    +public abstract class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
     {
    -    private static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    -
    -    private static final double ALPHA = 0.75; // set to 0.75 to make EDS more biased to towards the newer values
    -    private static final int WINDOW_SIZE = 100;
    -
    -    private volatile int dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    -    private volatile int dynamicResetInterval = DatabaseDescriptor.getDynamicResetInterval();
    -    private volatile double dynamicBadnessThreshold = DatabaseDescriptor.getDynamicBadnessThreshold();
    +    private static final Logger logger = LoggerFactory.getLogger(DynamicEndpointSnitch.class);
    +
    +    // Subclass specific functionality
    +    protected static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    protected boolean registered = false;
    +    // The scores map is updated via copy in updateScores
    +    // We keep it in the base class for performance reasons (so it can be easily aliased)
    +    protected volatile Map<InetAddressAndPort, Double> scores = new HashMap<>();
    +
    +    // Rate limit how often we generate latency probes
    +    protected long nextAllowedProbeGenerationTime;
    +    protected long nextProbeGenerationTime;
    +    protected int currentProbePosition = 0;
    +    protected final List<InetAddressAndPort> latencyProbeSequence = new ArrayList<>();
    +
    +    // DES general functionality
    +    protected volatile int dynamicUpdateInterval = -1;
    +    protected volatile int dynamicLatencyProbeInterval = -1;
    +    protected volatile double dynamicBadnessThreshold = 0;
     
         // the score for a merged set of endpoints must be this much worse than the score for separate endpoints to
         // warrant not merging two ranges into a single range
         private static final double RANGE_MERGING_PREFERENCE = 1.5;
     
         private String mbeanName;
    -    private boolean registered = false;
    -
    -    private volatile HashMap<InetAddressAndPort, Double> scores = new HashMap<>();
    -    private final ConcurrentHashMap<InetAddressAndPort, ExponentiallyDecayingReservoir> samples = new ConcurrentHashMap<>();
    +    private boolean mbeanRegistered = false;
     
         public final IEndpointSnitch subsnitch;
     
    -    private volatile ScheduledFuture<?> updateSchedular;
    -    private volatile ScheduledFuture<?> resetSchedular;
    +    private volatile ScheduledFuture<?> updateScheduler;
    +    private volatile ScheduledFuture<?> latencyProbeScheduler;
     
    -    private final Runnable update;
    -    private final Runnable reset;
    +    private final Runnable update = this::updateScores;
    +    private final Runnable latencyProbe = this::maybeSendLatencyProbe;
     
         public DynamicEndpointSnitch(IEndpointSnitch snitch)
         {
             this(snitch, null);
         }
     
    -    public DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
    +    protected DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
         {
             mbeanName = "org.apache.cassandra.db:type=DynamicEndpointSnitch";
             if (instance != null)
                 mbeanName += ",instance=" + instance;
             subsnitch = snitch;
    -        update = new Runnable()
    -        {
    -            public void run()
    -            {
    -                updateScores();
    -            }
    -        };
    -        reset = new Runnable()
    -        {
    -            public void run()
    -            {
    -                // we do this so that a host considered bad has a chance to recover, otherwise would we never try
    -                // to read from it, which would cause its score to never change
    -                reset();
    -            }
    -        };
     
             if (DatabaseDescriptor.isDaemonInitialized())
             {
    -            updateSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(update, dynamicUpdateInterval, dynamicUpdateInterval, TimeUnit.MILLISECONDS);
    -            resetSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(reset, dynamicResetInterval, dynamicResetInterval, TimeUnit.MILLISECONDS);
    +            applyConfigChanges();
                 registerMBean();
             }
    +        nextProbeGenerationTime = System.nanoTime();
    +    }
    +
    +    /**
    +     * Records a latency. This MUST be cheap as it is called in the fast path
    +     */
    +    abstract public void receiveTiming(InetAddressAndPort address, long latency, LatencyMeasurementType measurementType);
    +
    +    /**
    +     * Called periodically from {@link DynamicEndpointSnitch#updateScores()}. This method returns a new score map
    +     * that will be assigned to the {@link DynamicEndpointSnitch#scores} map.
    +     * This is generally more expensive than {@link DynamicEndpointSnitch#receiveTiming}
    +     */
    +    abstract public Map<InetAddressAndPort, Double> calculateScores();
    --- End diff --
    
    This calculation is the same as well across implementations? Or it should be? The only difference is what we are using to store timings and what it spits out?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225332528
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -154,20 +228,101 @@ private void registerMBean()
     
         public void close()
         {
    -        updateSchedular.cancel(false);
    -        resetSchedular.cancel(false);
    +        if (updateScheduler != null)
    +            updateScheduler.cancel(false);
    +        if (latencyProbeScheduler != null)
    +            latencyProbeScheduler.cancel(false);
     
             MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
             try
             {
    -            mbs.unregisterMBean(new ObjectName(mbeanName));
    +            if (mbeanRegistered)
    +                mbs.unregisterMBean(new ObjectName(mbeanName));
             }
             catch (Exception e)
             {
                 throw new RuntimeException(e);
             }
         }
     
    +    /**
    +     * Determines if latency probes need to be sent, and potentially sends a single latency probe per invocation
    +     */
    +    protected void maybeSendLatencyProbe()
    +    {
    +        long currentTimeNS = System.nanoTime();
    +        markNextAllowedProbeGenerationTime(currentTimeNS);
    +
    +        Optional<InetAddressAndPort> needsProbe = latencyProbeNeeded(currentTimeNS);
    +        needsProbe.ifPresent(this::sendPingMessageToPeer);
    +    }
    +
    +    /**
    +     * Determines which peers need latency at a particular time. Note that this takes currentTimeNS for testability
    +     * of this code path.
    +     * @param currentTimeNS The current time to evaluate. Used mostly for testing.
    +     * @return An Optional that if present contains a host to probe.
    +     */
    +    @VisibleForTesting
    +    Optional<InetAddressAndPort> latencyProbeNeeded(long currentTimeNS) {
    +        if (currentProbePosition >= latencyProbeSequence.size() && (currentTimeNS > nextProbeGenerationTime))
    +        {
    +            nextProbeGenerationTime = nextAllowedProbeGenerationTime;
    +            latencyProbeSequence.clear();
    +
    +            // Delegate to the subclass to actually figure out what the probe sequence should be
    +            updateLatencyProbeSequence(latencyProbeSequence);
    +
    +            if (latencyProbeSequence.size() > 0)
    +                Collections.shuffle(latencyProbeSequence);
    +
    +            currentProbePosition = 0;
    +        }
    +
    +        if (currentProbePosition < latencyProbeSequence.size())
    +        {
    +            try
    +            {
    +                return Optional.of(latencyProbeSequence.get(currentProbePosition++));
    +            }
    +            catch (IndexOutOfBoundsException ignored) {}
    +        }
    +
    +        return Optional.empty();
    +    }
    +
    +    private void sendPingMessageToPeer(InetAddressAndPort to)
    +    {
    +        logger.trace("Sending a small and large PingMessage to {}", to);
    +        IAsyncCallback latencyProbeHandler = new IAsyncCallback()
    --- End diff --
    
    You can create two settable futures here (or the completable equivalent) and separate callbacks for small and large that point to the settable futures. You can then create a composite future and add a listener to it that does the right thing with the resulting largest/worst latency value.
    
    I think this means you still need the enum, but we don't actually need to change the API of callbacks since we will never report a probe via a callback directly. 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225347623
  
    --- Diff: src/java/org/apache/cassandra/gms/Gossiper.java ---
    @@ -38,6 +38,7 @@
     import com.google.common.util.concurrent.Uninterruptibles;
     
     import org.apache.cassandra.locator.InetAddressAndPort;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    --- End diff --
    
    ack, fixed


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r229469611
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -154,31 +326,203 @@ private void registerMBean()
     
         public void close()
         {
    -        updateSchedular.cancel(false);
    -        resetSchedular.cancel(false);
    +        if (updateScoresScheduler != null)
    +            updateScoresScheduler.cancel(false);
    +        if (updateSamplesScheduler != null)
    +            updateSamplesScheduler.cancel(false);
    +
    +        for (AnnotatedMeasurement measurement : samples.values())
    +        {
    +            if (measurement.probeFuture != null)
    +                measurement.probeFuture.cancel(false);
    +
    +            measurement.millisSinceLastMeasure.set(0);
    +            measurement.millisSinceLastRequest.set(MAX_PROBE_INTERVAL_MS);
    +            measurement.probeTimerMillis = 0;
    +        }
     
             MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
             try
             {
    -            mbs.unregisterMBean(new ObjectName(mbeanName));
    +            if (mbeanRegistered)
    +                mbs.unregisterMBean(new ObjectName(mbeanName));
             }
             catch (Exception e)
             {
                 throw new RuntimeException(e);
             }
         }
     
    +    /**
    +     * Background task running on the samples dictionary. The default implementation sends latency probes (PING)
    +     * messages to explore nodes that we have not received timings for recently but have ranked in
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}.
    +     */
    +    protected void updateSamples()
    +    {
    +        // Split calculation of probe timers from sending probes for testability
    +        calculateProbes(samples, dynamicLatencyProbeInterval);
    +
    +        if (!StorageService.instance.isGossipActive())
    +            return;
    +
    +        schedulePings(samples);
    +    }
    +
    +    /**
    +     * This method mutates the passed AnnotatedMeasurements to implement capped exponential backoff per endpoint.
    +     *
    +     * The algorithm is as follows:
    +     * 1. All samples get their millisSinceLastMeasure and millisSinceLastRequest fields
    +     *    incremented by the passed interval
    +     * 2. Any recently requested (ranked) endpoints that have not been measured recently (e.g. because the snitch
    +     *    has sent them no traffic) get probes with exponential backoff.
    +     *
    +     * The backoff is capped at MAX_PROBE_INTERVAL_MS. Furthermore the probes are stopped after
    +     * MAX_PROBE_INTERVAL_MS of no ranking requests as well.
    +     *
    +     * At the end of this method, any passed AnnotatedMeasurements that need latency probes will have non zero
    +     * probeTimerMillis members set.
    +     */
    +    @VisibleForTesting
    +    static void calculateProbes(Map<InetAddressAndPort, AnnotatedMeasurement> samples, long intervalMillis) {
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry: samples.entrySet())
    +        {
    +            if (entry.getKey().equals(FBUtilities.getBroadcastAddressAndPort()))
    +                continue;
    +
    +            AnnotatedMeasurement measurement = entry.getValue();
    +            long lastMeasure = measurement.millisSinceLastMeasure.getAndAdd(intervalMillis);
    +            long lastRequest = measurement.millisSinceLastRequest.getAndAdd(intervalMillis);
    +
    +            if (lastMeasure >= MIN_PROBE_INTERVAL_MS && lastRequest < MAX_PROBE_INTERVAL_MS)
    +            {
    +                if (measurement.probeTimerMillis == 0)
    +                {
    +                    measurement.probeTimerMillis = intervalMillis;
    --- End diff --
    
    assert probe future is null here?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r229487316
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -154,31 +326,203 @@ private void registerMBean()
     
         public void close()
         {
    -        updateSchedular.cancel(false);
    -        resetSchedular.cancel(false);
    +        if (updateScoresScheduler != null)
    +            updateScoresScheduler.cancel(false);
    +        if (updateSamplesScheduler != null)
    +            updateSamplesScheduler.cancel(false);
    +
    +        for (AnnotatedMeasurement measurement : samples.values())
    +        {
    +            if (measurement.probeFuture != null)
    +                measurement.probeFuture.cancel(false);
    +
    +            measurement.millisSinceLastMeasure.set(0);
    +            measurement.millisSinceLastRequest.set(MAX_PROBE_INTERVAL_MS);
    +            measurement.probeTimerMillis = 0;
    +        }
     
             MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
             try
             {
    -            mbs.unregisterMBean(new ObjectName(mbeanName));
    +            if (mbeanRegistered)
    +                mbs.unregisterMBean(new ObjectName(mbeanName));
             }
             catch (Exception e)
             {
                 throw new RuntimeException(e);
             }
         }
     
    +    /**
    +     * Background task running on the samples dictionary. The default implementation sends latency probes (PING)
    +     * messages to explore nodes that we have not received timings for recently but have ranked in
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}.
    +     */
    +    protected void updateSamples()
    +    {
    +        // Split calculation of probe timers from sending probes for testability
    +        calculateProbes(samples, dynamicLatencyProbeInterval);
    +
    +        if (!StorageService.instance.isGossipActive())
    +            return;
    +
    +        schedulePings(samples);
    +    }
    +
    +    /**
    +     * This method mutates the passed AnnotatedMeasurements to implement capped exponential backoff per endpoint.
    +     *
    +     * The algorithm is as follows:
    +     * 1. All samples get their millisSinceLastMeasure and millisSinceLastRequest fields
    +     *    incremented by the passed interval
    +     * 2. Any recently requested (ranked) endpoints that have not been measured recently (e.g. because the snitch
    +     *    has sent them no traffic) get probes with exponential backoff.
    +     *
    +     * The backoff is capped at MAX_PROBE_INTERVAL_MS. Furthermore the probes are stopped after
    +     * MAX_PROBE_INTERVAL_MS of no ranking requests as well.
    +     *
    +     * At the end of this method, any passed AnnotatedMeasurements that need latency probes will have non zero
    +     * probeTimerMillis members set.
    +     */
    +    @VisibleForTesting
    +    static void calculateProbes(Map<InetAddressAndPort, AnnotatedMeasurement> samples, long intervalMillis) {
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry: samples.entrySet())
    +        {
    +            if (entry.getKey().equals(FBUtilities.getBroadcastAddressAndPort()))
    +                continue;
    +
    +            AnnotatedMeasurement measurement = entry.getValue();
    +            long lastMeasure = measurement.millisSinceLastMeasure.getAndAdd(intervalMillis);
    +            long lastRequest = measurement.millisSinceLastRequest.getAndAdd(intervalMillis);
    +
    +            if (lastMeasure >= MIN_PROBE_INTERVAL_MS && lastRequest < MAX_PROBE_INTERVAL_MS)
    +            {
    +                if (measurement.probeTimerMillis == 0)
    +                {
    +                    measurement.probeTimerMillis = intervalMillis;
    +                }
    +                else if (measurement.probeFuture != null && measurement.probeFuture.isDone())
    +                {
    +                    measurement.probeTimerMillis = Math.min(MAX_PROBE_INTERVAL_MS, measurement.probeTimerMillis * 2);
    --- End diff --
    
    I take this back. I think being a bounded amount slower at scheduling pings is fine and I like how things are relatively simple right now.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225634901
  
    --- Diff: src/java/org/apache/cassandra/net/IAsyncCallback.java ---
    @@ -31,11 +31,23 @@
         void response(MessageIn<T> msg);
     
         /**
    +     * @deprecated Use {@link #latencyMeasurementType()}. This is left for backwards
    +     *             compatibility with existing IAsyncCallback implementations but will
    +     *             be removed at a future date.
          * @return true if this callback is on the read path and its latency should be
          * given as input to the dynamic snitch.
          */
    +    @Deprecated
    --- End diff --
    
    Yes we have way too many mock snitches. I think there are some things that require one to be constructed. I would still make the change, and maybe if possible consolidate the mock snitches into as few as possible? We definitely don't want to keep around deprecated internal code and we have to pay whatever the cost is to get rid of it.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r229327855
  
    --- Diff: src/java/org/apache/cassandra/service/StorageService.java ---
    @@ -4940,42 +4940,61 @@ public int getDynamicUpdateInterval()
             return DatabaseDescriptor.getDynamicUpdateInterval();
         }
     
    +    @Override
         public void updateSnitch(String epSnitchClassName, Boolean dynamic, Integer dynamicUpdateInterval, Integer dynamicResetInterval, Double dynamicBadnessThreshold) throws ClassNotFoundException
    +    {
    +        updateSnitch(epSnitchClassName, dynamic ? "" : null, dynamicUpdateInterval, null, dynamicBadnessThreshold);
    +    }
    +
    +    @Override
    +    public void updateSnitch(String epSnitchClassName, String dynamicSnitchClassName, Integer dynamicUpdateInterval, Integer dynamicSampleUpdateInterval, Double dynamicBadnessThreshold) throws ClassNotFoundException
         {
             // apply dynamic snitch configuration
             if (dynamicUpdateInterval != null)
                 DatabaseDescriptor.setDynamicUpdateInterval(dynamicUpdateInterval);
    -        if (dynamicResetInterval != null)
    -            DatabaseDescriptor.setDynamicResetInterval(dynamicResetInterval);
             if (dynamicBadnessThreshold != null)
                 DatabaseDescriptor.setDynamicBadnessThreshold(dynamicBadnessThreshold);
    +        if (dynamicSampleUpdateInterval != null)
    --- End diff --
    
    Wow I wasn't expecting you to make it a hot prop. I thought we had left it that it was fine without it. Good to have it since people can go back to the old snitch without bouncing the cluster.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r236761954
  
    --- Diff: test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java ---
    @@ -60,57 +130,229 @@ private static EndpointsForRange full(InetAddressAndPort... endpoints)
         }
     
         @Test
    -    public void testSnitch() throws InterruptedException, IOException, ConfigurationException
    +    public void testSortedByProximity() throws InterruptedException, IOException, ConfigurationException
         {
    -        // do this because SS needs to be initialized before DES can work properly.
    -        StorageService.instance.unsafeInitialize();
    -        SimpleSnitch ss = new SimpleSnitch();
    -        DynamicEndpointSnitch dsnitch = new DynamicEndpointSnitch(ss, String.valueOf(ss.hashCode()));
    -        InetAddressAndPort self = FBUtilities.getBroadcastAddressAndPort();
    -        InetAddressAndPort host1 = InetAddressAndPort.getByName("127.0.0.2");
    -        InetAddressAndPort host2 = InetAddressAndPort.getByName("127.0.0.3");
    -        InetAddressAndPort host3 = InetAddressAndPort.getByName("127.0.0.4");
    -        InetAddressAndPort host4 = InetAddressAndPort.getByName("127.0.0.5");
    -        List<InetAddressAndPort> hosts = Arrays.asList(host1, host2, host3);
    +        InetAddressAndPort self = hosts[0];
    +        List<InetAddressAndPort> allHosts = Arrays.asList(hosts[1], hosts[2], hosts[3]);
     
             // first, make all hosts equal
    -        setScores(dsnitch, 1, hosts, 10, 10, 10);
    -        EndpointsForRange order = full(host1, host2, host3);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    -
    -        // make host1 a little worse
    -        setScores(dsnitch, 1, hosts, 20, 10, 10);
    -        order = full(host2, host3, host1);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    -
    -        // make host2 as bad as host1
    -        setScores(dsnitch, 2, hosts, 15, 20, 10);
    -        order = full(host3, host1, host2);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    -
    -        // make host3 the worst
    -        setScores(dsnitch, 3, hosts, 10, 10, 30);
    -        order = full(host1, host2, host3);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    -
    -        // make host3 equal to the others
    -        setScores(dsnitch, 5, hosts, 10, 10, 10);
    -        order = full(host1, host2, host3);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    +        setScores(dsnitch, 1, allHosts, 10, 10, 10);
    +        EndpointsForRange order = full(hosts[1], hosts[2], hosts[3]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        // make hosts[1] a little worse
    +        setScores(dsnitch, 2, allHosts, 20, 10, 10);
    +        order = full(hosts[2], hosts[3], hosts[1]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        // make hosts[2] as bad as hosts[1]
    +        setScores(dsnitch, 4, allHosts, 15, 20, 10);
    +        order = full(hosts[3], hosts[1], hosts[2]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        // make hosts[3] the worst
    +        setScores(dsnitch, 10, allHosts, 10, 10, 30);
    +        order = full(hosts[1], hosts[2], hosts[3]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        // make hosts[3] equal to the others
    +        setScores(dsnitch, 15, allHosts, 10, 10, 10);
    +        order = full(hosts[1], hosts[2], hosts[3]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
     
             /// Tests CASSANDRA-6683 improvements
             // make the scores differ enough from the ideal order that we sort by score; under the old
             // dynamic snitch behavior (where we only compared neighbors), these wouldn't get sorted
    -        setScores(dsnitch, 20, hosts, 10, 70, 20);
    -        order = full(host1, host3, host2);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    +        dsnitch.reset();
    +        setScores(dsnitch, 20, allHosts, 10, 70, 20);
    +        order = full(hosts[1], hosts[3], hosts[2]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        order = full(hosts[4], hosts[1], hosts[3], hosts[2]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3], hosts[4])));
    +
    +        setScores(dsnitch, 40, allHosts, 10, 10, 11);
    +        order = full(hosts[4], hosts[1], hosts[2], hosts[3]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3], hosts[4])));
    +    }
    +
    +    // CASSANDRA-14459 improvements to add latency probes instead of resets
    +    @Test
    +    public void testLatencyProbeNeeded()
    +    {
    +        InetAddressAndPort self = hosts[0];
    +
    +        // Four hosts, two are very latent, but all are requested for ranking
    +        dsnitch.receiveTiming(hosts[1], 20, LatencyMeasurementType.READ);
    +        dsnitch.receiveTiming(hosts[2], 10, LatencyMeasurementType.READ);
    +        dsnitch.receiveTiming(hosts[3], 1000, LatencyMeasurementType.READ);
    +        dsnitch.receiveTiming(hosts[4], 1000, LatencyMeasurementType.READ);
    +        dsnitch.updateScores();
    +
    +        EndpointsForRange orderBefore = full(hosts[2], hosts[1], hosts[3], hosts[4]);
    +
    +        Map<InetAddressAndPort, DynamicEndpointSnitch.AnnotatedMeasurement> measurements = dsnitch.getMeasurementsWithPort();
    +        // At this point we haven't ranked any of hosts so we should need no probes even
    +        DynamicEndpointSnitch.calculateProbes(measurements, dsnitch.dynamicLatencyProbeInterval);
    +        assertFalse(measurements.values().stream().anyMatch(m -> m.probeTimerMillis > 0));
    +        assertTrue(measurements.values().stream().allMatch(m -> m.probeFuture == null));
    +
    +        // Two hosts continue receiving traffic but the last two are always ranked by the snitch
    +        for (int i = 0; i < 10; i++)
    +        {
    +            dsnitch.receiveTiming(hosts[1], 20, LatencyMeasurementType.READ);
    +            dsnitch.receiveTiming(hosts[2], 10, LatencyMeasurementType.READ);
    +            Util.assertRCEquals(orderBefore, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3], hosts[4])));
    +        }
    +        dsnitch.updateScores();
    +
    +        int probesBefore = (int) ((DynamicEndpointSnitch.MIN_PROBE_INTERVAL_MS / dsnitch.dynamicLatencyProbeInterval) + 1);
    --- End diff --
    
    Ack, renamed to `intervalsBeforeProbesStart` to try to make this clearer.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r229494906
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -23,128 +23,300 @@
     import java.net.UnknownHostException;
     import java.util.*;
     import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ExecutionException;
     import java.util.concurrent.ScheduledFuture;
     import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
     import java.util.stream.Collectors;
     
    -import com.codahale.metrics.ExponentiallyDecayingReservoir;
     import javax.management.MBeanServer;
     import javax.management.ObjectName;
     
    -import com.codahale.metrics.Snapshot;
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.util.concurrent.Futures;
    +import com.google.common.util.concurrent.MoreExecutors;
    +import com.google.common.util.concurrent.RateLimiter;
    +import com.google.common.util.concurrent.SettableFuture;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
     import org.apache.cassandra.concurrent.ScheduledExecutors;
     import org.apache.cassandra.config.DatabaseDescriptor;
    +import org.apache.cassandra.exceptions.RequestFailureReason;
     import org.apache.cassandra.gms.ApplicationState;
     import org.apache.cassandra.gms.EndpointState;
     import org.apache.cassandra.gms.Gossiper;
     import org.apache.cassandra.gms.VersionedValue;
    +import org.apache.cassandra.locator.dynamicsnitch.DynamicEndpointSnitchHistogram;
    +import org.apache.cassandra.net.IAsyncCallbackWithFailure;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +import org.apache.cassandra.net.MessageIn;
    +import org.apache.cassandra.net.MessageOut;
     import org.apache.cassandra.net.MessagingService;
    +import org.apache.cassandra.net.PingMessage;
     import org.apache.cassandra.service.StorageService;
     import org.apache.cassandra.utils.FBUtilities;
     
    +import static org.apache.cassandra.net.MessagingService.Verb.PING;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.LARGE_MESSAGE;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.SMALL_MESSAGE;
    +
    +
     /**
      * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector
    + * Note that the subclasses (e.g. {@link DynamicEndpointSnitchHistogram}) are responsible for actually measuring
    + * latency and providing an ISnitchMeasurement implementation back to this class.
      */
    -public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
    +public abstract class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
     {
    -    private static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    -
    -    private static final double ALPHA = 0.75; // set to 0.75 to make EDS more biased to towards the newer values
    -    private static final int WINDOW_SIZE = 100;
    -
    -    private volatile int dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    -    private volatile int dynamicResetInterval = DatabaseDescriptor.getDynamicResetInterval();
    -    private volatile double dynamicBadnessThreshold = DatabaseDescriptor.getDynamicBadnessThreshold();
    +    private static final Logger logger = LoggerFactory.getLogger(DynamicEndpointSnitch.class);
    +
    +    // Latency measurement and ranking. The samples contain latency measurements and the scores are used for ranking
    +    protected boolean registered = false;
    +    protected static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    protected volatile Map<InetAddressAndPort, Double> scores = new HashMap<>();
    +    protected final Map<InetAddressAndPort, AnnotatedMeasurement> samples = new ConcurrentHashMap<>();
    +
    +    // Latency probe functionality for actively probing endpoints that we haven't measured recently but are ranking
    +    public static final long MAX_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_max_probe_interval_ms", 60 * 10 * 1000L);
    +    public static final long MIN_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_min_probe_interval_ms", 60 * 1000L) ;
    +    // The probe rate is set later when configuration is read
    +    protected static final RateLimiter probeRateLimiter = RateLimiter.create(1);
    +    protected static final DebuggableScheduledThreadPoolExecutor latencyProbeExecutor = new DebuggableScheduledThreadPoolExecutor(1, "LatencyProbes", Thread.MIN_PRIORITY);
    --- End diff --
    
    I am kind of shocked at the number of places we use thread priorities. I have to wonder if anyone has measured the impact of doing that.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r229461610
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -23,128 +23,300 @@
     import java.net.UnknownHostException;
     import java.util.*;
     import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ExecutionException;
     import java.util.concurrent.ScheduledFuture;
     import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
     import java.util.stream.Collectors;
     
    -import com.codahale.metrics.ExponentiallyDecayingReservoir;
     import javax.management.MBeanServer;
     import javax.management.ObjectName;
     
    -import com.codahale.metrics.Snapshot;
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.util.concurrent.Futures;
    +import com.google.common.util.concurrent.MoreExecutors;
    +import com.google.common.util.concurrent.RateLimiter;
    +import com.google.common.util.concurrent.SettableFuture;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
     import org.apache.cassandra.concurrent.ScheduledExecutors;
     import org.apache.cassandra.config.DatabaseDescriptor;
    +import org.apache.cassandra.exceptions.RequestFailureReason;
     import org.apache.cassandra.gms.ApplicationState;
     import org.apache.cassandra.gms.EndpointState;
     import org.apache.cassandra.gms.Gossiper;
     import org.apache.cassandra.gms.VersionedValue;
    +import org.apache.cassandra.locator.dynamicsnitch.DynamicEndpointSnitchHistogram;
    +import org.apache.cassandra.net.IAsyncCallbackWithFailure;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +import org.apache.cassandra.net.MessageIn;
    +import org.apache.cassandra.net.MessageOut;
     import org.apache.cassandra.net.MessagingService;
    +import org.apache.cassandra.net.PingMessage;
     import org.apache.cassandra.service.StorageService;
     import org.apache.cassandra.utils.FBUtilities;
     
    +import static org.apache.cassandra.net.MessagingService.Verb.PING;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.LARGE_MESSAGE;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.SMALL_MESSAGE;
    +
    +
     /**
      * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector
    + * Note that the subclasses (e.g. {@link DynamicEndpointSnitchHistogram}) are responsible for actually measuring
    + * latency and providing an ISnitchMeasurement implementation back to this class.
      */
    -public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
    +public abstract class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
     {
    -    private static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    -
    -    private static final double ALPHA = 0.75; // set to 0.75 to make EDS more biased to towards the newer values
    -    private static final int WINDOW_SIZE = 100;
    -
    -    private volatile int dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    -    private volatile int dynamicResetInterval = DatabaseDescriptor.getDynamicResetInterval();
    -    private volatile double dynamicBadnessThreshold = DatabaseDescriptor.getDynamicBadnessThreshold();
    +    private static final Logger logger = LoggerFactory.getLogger(DynamicEndpointSnitch.class);
    +
    +    // Latency measurement and ranking. The samples contain latency measurements and the scores are used for ranking
    +    protected boolean registered = false;
    +    protected static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    protected volatile Map<InetAddressAndPort, Double> scores = new HashMap<>();
    +    protected final Map<InetAddressAndPort, AnnotatedMeasurement> samples = new ConcurrentHashMap<>();
    +
    +    // Latency probe functionality for actively probing endpoints that we haven't measured recently but are ranking
    +    public static final long MAX_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_max_probe_interval_ms", 60 * 10 * 1000L);
    +    public static final long MIN_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_min_probe_interval_ms", 60 * 1000L) ;
    +    // The probe rate is set later when configuration is read
    +    protected static final RateLimiter probeRateLimiter = RateLimiter.create(1);
    +    protected static final DebuggableScheduledThreadPoolExecutor latencyProbeExecutor = new DebuggableScheduledThreadPoolExecutor(1, "LatencyProbes", Thread.MIN_PRIORITY);
    +
    +    // User configuration of the snitch tunables
    +    protected volatile int dynamicUpdateInterval = -1;
    +    protected volatile int dynamicLatencyProbeInterval = -1;
    +    protected volatile double dynamicBadnessThreshold = 0;
     
         // the score for a merged set of endpoints must be this much worse than the score for separate endpoints to
         // warrant not merging two ranges into a single range
         private static final double RANGE_MERGING_PREFERENCE = 1.5;
     
         private String mbeanName;
    -    private boolean registered = false;
    -
    -    private volatile HashMap<InetAddressAndPort, Double> scores = new HashMap<>();
    -    private final ConcurrentHashMap<InetAddressAndPort, ExponentiallyDecayingReservoir> samples = new ConcurrentHashMap<>();
    +    private boolean mbeanRegistered = false;
     
         public final IEndpointSnitch subsnitch;
     
    -    private volatile ScheduledFuture<?> updateSchedular;
    -    private volatile ScheduledFuture<?> resetSchedular;
    -
    -    private final Runnable update;
    -    private final Runnable reset;
    +    private volatile ScheduledFuture<?> updateScoresScheduler;
    +    private volatile ScheduledFuture<?> updateSamplesScheduler;
     
         public DynamicEndpointSnitch(IEndpointSnitch snitch)
         {
             this(snitch, null);
         }
     
    -    public DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
    +    protected DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
         {
             mbeanName = "org.apache.cassandra.db:type=DynamicEndpointSnitch";
             if (instance != null)
                 mbeanName += ",instance=" + instance;
             subsnitch = snitch;
    -        update = new Runnable()
    +
    +        if (DatabaseDescriptor.isDaemonInitialized())
             {
    -            public void run()
    -            {
    -                updateScores();
    -            }
    -        };
    -        reset = new Runnable()
    +            applyConfigChanges(DatabaseDescriptor.getDynamicUpdateInterval(),
    +                               DatabaseDescriptor.getDynamicSampleUpdateInterval(),
    +                               DatabaseDescriptor.getDynamicBadnessThreshold());
    +            open();
    +        }
    +    }
    +
    +    /**
    +     * Allows subclasses to inject new ways of measuring latency back to this abstract base class.
    +     */
    +    protected interface ISnitchMeasurement
    +    {
    +        void sample(long value);
    +        double measure();
    +        Iterable<Double> measurements();
    +    }
    +
    +    /**
    +     * Adds some boookeeping that the DES uses over top of the various metrics techniques used by the
    +     * implementations. This is used to allow CASSANDRA-14459 latency probes as well as further experimentation
    +     * on new latency measurement techniques in CASSANDRA-14817
    +     *
    +     * {@link AnnotatedMeasurement#millisSinceLastRequest} is set to zero through
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}. It defaults to
    +     * the maximum interval so that we only start probing once it has been ranked at least once
    +     * {@link AnnotatedMeasurement#millisSinceLastMeasure} is set to zero from
    +     * {@link DynamicEndpointSnitch#receiveTiming(InetAddressAndPort, long, LatencyMeasurementType)}.
    +     *
    +     * {@link AnnotatedMeasurement#millisSinceLastMeasure and {@link AnnotatedMeasurement#probeTimerMillis}
    +     * are incremented via {@link DynamicEndpointSnitch#updateSamples()}
    +     */
    +    protected static class AnnotatedMeasurement
    +    {
    +        // Used to optimally target latency probes only on nodes that are both requested for ranking
    +        // and are not being measured. For example with token aware clients a large portion of the cluster will never
    +        // be ranked at all and therefore we won't probe them.
    +        public AtomicLong millisSinceLastRequest = new AtomicLong(MAX_PROBE_INTERVAL_MS);
    +        public AtomicLong millisSinceLastMeasure = new AtomicLong(0);
    +        public long probeTimerMillis = 0;
    +        public ScheduledFuture<?> probeFuture = null;
    --- End diff --
    
    Technically should be volatile for toString()


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225362794
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -210,8 +368,10 @@ public String getDatacenter(InetAddressAndPort endpoint)
     
             // TODO: avoid copy
             replicas = subsnitch.sortedByProximity(address, replicas);
    -        HashMap<InetAddressAndPort, Double> scores = this.scores; // Make sure the score don't change in the middle of the loop below
    -                                                           // (which wouldn't really matter here but its cleaner that way).
    +        // Make sure the score don't change in the middle of the loop below
    +        // (which wouldn't really matter here but its cleaner that way).
    +        final Map<InetAddressAndPort, Double> scores = this.scores;
    --- End diff --
    
    Yea I was somewhat surprised to see that too, I assumed there was a reason that it was final in `sortedByProximityWithBadness` and just made `sortedByProximityWithScores` consistent, but tbh I don't care and will switch to whichever way is better for you ;-)


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r239953888
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -21,133 +21,505 @@
     import java.net.InetAddress;
     import java.net.UnknownHostException;
     import java.util.*;
    +import java.util.concurrent.CancellationException;
     import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ExecutionException;
     import java.util.concurrent.ScheduledFuture;
     import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
    +import java.util.concurrent.atomic.AtomicLong;
     import java.util.stream.Collectors;
     
    -import com.codahale.metrics.ExponentiallyDecayingReservoir;
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.util.concurrent.Futures;
    +import com.google.common.util.concurrent.MoreExecutors;
    +import com.google.common.util.concurrent.RateLimiter;
    +import com.google.common.util.concurrent.SettableFuture;
     
    -import com.codahale.metrics.Snapshot;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
     import org.apache.cassandra.concurrent.ScheduledExecutors;
     import org.apache.cassandra.config.DatabaseDescriptor;
    +import org.apache.cassandra.exceptions.RequestFailureReason;
     import org.apache.cassandra.gms.ApplicationState;
     import org.apache.cassandra.gms.EndpointState;
     import org.apache.cassandra.gms.Gossiper;
     import org.apache.cassandra.gms.VersionedValue;
    +import org.apache.cassandra.locator.dynamicsnitch.DynamicEndpointSnitchHistogram;
    +import org.apache.cassandra.net.IAsyncCallbackWithFailure;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +import org.apache.cassandra.net.MessageIn;
    +import org.apache.cassandra.net.MessageOut;
     import org.apache.cassandra.net.MessagingService;
    +import org.apache.cassandra.net.PingMessage;
     import org.apache.cassandra.service.StorageService;
     import org.apache.cassandra.utils.FBUtilities;
     import org.apache.cassandra.utils.MBeanWrapper;
     
    +import static org.apache.cassandra.net.MessagingService.Verb.PING;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.LARGE_MESSAGE;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.SMALL_MESSAGE;
    +
    +
     /**
      * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector
    + * Note that the subclasses (e.g. {@link DynamicEndpointSnitchHistogram}) are responsible for actually measuring
    + * latency and providing an ISnitchMeasurement implementation back to this class.
      */
    -public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
    +public abstract class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
     {
    -    private static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    private static final Logger logger = LoggerFactory.getLogger(DynamicEndpointSnitch.class);
    +
    +    // Latency measurement and ranking. The samples contain latency measurements and the scores are used for ranking
    +    protected boolean registered = false;
    +    protected static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    protected volatile Map<InetAddressAndPort, Double> scores = new HashMap<>();
     
    -    private static final double ALPHA = 0.75; // set to 0.75 to make EDS more biased to towards the newer values
    -    private static final int WINDOW_SIZE = 100;
    +    protected final Map<InetAddressAndPort, AnnotatedMeasurement> samples = new ConcurrentHashMap<>();
     
    -    private volatile int dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    -    private volatile int dynamicResetInterval = DatabaseDescriptor.getDynamicResetInterval();
    -    private volatile double dynamicBadnessThreshold = DatabaseDescriptor.getDynamicBadnessThreshold();
    +    // Latency probe functionality for actively probing endpoints that we haven't measured recently but are ranking
    +    public static final long MAX_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_max_probe_interval_ms", 60 * 10 * 1000L);
    +    public static final long MIN_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_min_probe_interval_ms", 60 * 1000L) ;
    +    // The probe rate is set later when configuration is read in applyConfigChanges
    +    protected static final RateLimiter probeRateLimiter = RateLimiter.create(1);
    +    protected static final DebuggableScheduledThreadPoolExecutor latencyProbeExecutor = new DebuggableScheduledThreadPoolExecutor("LatencyProbes");
    +    private long lastUpdateSamplesNanos = System.nanoTime();
    +
    +    // User configuration of the snitch tunables
    +    protected volatile int dynamicUpdateInterval = -1;
    +    protected volatile int dynamicSampleUpdateInterval = -1;
    +    protected volatile double dynamicBadnessThreshold = 0;
     
         // the score for a merged set of endpoints must be this much worse than the score for separate endpoints to
         // warrant not merging two ranges into a single range
         private static final double RANGE_MERGING_PREFERENCE = 1.5;
     
         private String mbeanName;
    -    private boolean registered = false;
    -
    -    private volatile HashMap<InetAddressAndPort, Double> scores = new HashMap<>();
    -    private final ConcurrentHashMap<InetAddressAndPort, ExponentiallyDecayingReservoir> samples = new ConcurrentHashMap<>();
    +    private boolean mbeanRegistered = false;
     
         public final IEndpointSnitch subsnitch;
     
    -    private volatile ScheduledFuture<?> updateSchedular;
    -    private volatile ScheduledFuture<?> resetSchedular;
    -
    -    private final Runnable update;
    -    private final Runnable reset;
    +    private volatile ScheduledFuture<?> updateScoresScheduler = null;
    +    private volatile ScheduledFuture<?> updateSamplesScheduler = null;
     
         public DynamicEndpointSnitch(IEndpointSnitch snitch)
         {
             this(snitch, null);
         }
     
    -    public DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
    +    protected DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
         {
             mbeanName = "org.apache.cassandra.db:type=DynamicEndpointSnitch";
             if (instance != null)
                 mbeanName += ",instance=" + instance;
             subsnitch = snitch;
    -        update = new Runnable()
    +
    +        if (DatabaseDescriptor.isDaemonInitialized())
             {
    -            public void run()
    +            open();
    +        }
    +    }
    +
    +    /**
    +     * Update configuration of the background tasks and restart the various scheduler tasks
    +     * if the configured rates for these tasks have changed.
    +     */
    +    public void applyConfigChanges(int newDynamicUpdateInternal, int newDynamicSampleUpdateInterval, double newDynamicBadnessThreshold)
    +    {
    +        if (DatabaseDescriptor.isDaemonInitialized())
    +        {
    +            if (dynamicUpdateInterval != newDynamicUpdateInternal || updateScoresScheduler == null)
                 {
    -                updateScores();
    +                cancelAndWait(updateScoresScheduler, Math.max(1, dynamicUpdateInterval), TimeUnit.MILLISECONDS);
    +                updateScoresScheduler = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(this::updateScores, newDynamicUpdateInternal, newDynamicUpdateInternal, TimeUnit.MILLISECONDS);
                 }
    -        };
    -        reset = new Runnable()
    +
    +            if (dynamicSampleUpdateInterval != newDynamicSampleUpdateInterval || updateSamplesScheduler == null)
    +            {
    +                cancelAndWait(updateSamplesScheduler, Math.max(1, dynamicSampleUpdateInterval), TimeUnit.MILLISECONDS);
    +                if (newDynamicSampleUpdateInterval > 0)
    +                    updateSamplesScheduler = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(this::updateSamples, newDynamicSampleUpdateInterval, newDynamicSampleUpdateInterval, TimeUnit.MILLISECONDS);
    +            }
    +        }
    +
    +        dynamicUpdateInterval = newDynamicUpdateInternal;
    +        dynamicSampleUpdateInterval = newDynamicSampleUpdateInterval;
    +        dynamicBadnessThreshold = newDynamicBadnessThreshold;
    +
    +        if (dynamicSampleUpdateInterval > 0)
    +            probeRateLimiter.setRate(dynamicSampleUpdateInterval);
    +    }
    +
    +    public synchronized void open()
    +    {
    +        applyConfigChanges(DatabaseDescriptor.getDynamicUpdateInterval(),
    +                           DatabaseDescriptor.getDynamicSampleUpdateInterval(),
    +                           DatabaseDescriptor.getDynamicBadnessThreshold());
    +
    +        MBeanWrapper.instance.registerMBean(this, mbeanName);
    +        mbeanRegistered = true;
    +    }
    +
    +    public synchronized void close()
    +    {
    +        cancelAndWait(updateScoresScheduler, Math.max(1, dynamicUpdateInterval), TimeUnit.MILLISECONDS);
    +        cancelAndWait(updateSamplesScheduler, Math.max(1, dynamicSampleUpdateInterval), TimeUnit.MILLISECONDS);
    +        updateScoresScheduler = null;
    +        updateSamplesScheduler = null;
    +
    +        for (AnnotatedMeasurement measurement : samples.values())
             {
    -            public void run()
    +            cancelAndWait(measurement.probeFuture, PING.getTimeout(), TimeUnit.MILLISECONDS);
    --- End diff --
    
    I don't get why the PING timeout is appropriate here?
    
    Also this should cancel all of them and then wait on them I think.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r237960721
  
    --- Diff: src/java/org/apache/cassandra/service/StorageService.java ---
    @@ -4940,42 +4940,61 @@ public int getDynamicUpdateInterval()
             return DatabaseDescriptor.getDynamicUpdateInterval();
         }
     
    +    @Override
         public void updateSnitch(String epSnitchClassName, Boolean dynamic, Integer dynamicUpdateInterval, Integer dynamicResetInterval, Double dynamicBadnessThreshold) throws ClassNotFoundException
    +    {
    +        updateSnitch(epSnitchClassName, dynamic ? "" : null, dynamicUpdateInterval, null, dynamicBadnessThreshold);
    +    }
    +
    +    @Override
    +    public void updateSnitch(String epSnitchClassName, String dynamicSnitchClassName, Integer dynamicUpdateInterval, Integer dynamicSampleUpdateInterval, Double dynamicBadnessThreshold) throws ClassNotFoundException
         {
             // apply dynamic snitch configuration
             if (dynamicUpdateInterval != null)
                 DatabaseDescriptor.setDynamicUpdateInterval(dynamicUpdateInterval);
    -        if (dynamicResetInterval != null)
    -            DatabaseDescriptor.setDynamicResetInterval(dynamicResetInterval);
             if (dynamicBadnessThreshold != null)
                 DatabaseDescriptor.setDynamicBadnessThreshold(dynamicBadnessThreshold);
    +        if (dynamicSampleUpdateInterval != null)
    --- End diff --
    
    That's what I figured, also makes testing a lot easier. I added a bunch of tests of the various swaps as well.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r232435653
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -23,128 +23,300 @@
     import java.net.UnknownHostException;
     import java.util.*;
     import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ExecutionException;
     import java.util.concurrent.ScheduledFuture;
     import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
     import java.util.stream.Collectors;
     
    -import com.codahale.metrics.ExponentiallyDecayingReservoir;
     import javax.management.MBeanServer;
     import javax.management.ObjectName;
     
    -import com.codahale.metrics.Snapshot;
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.util.concurrent.Futures;
    +import com.google.common.util.concurrent.MoreExecutors;
    +import com.google.common.util.concurrent.RateLimiter;
    +import com.google.common.util.concurrent.SettableFuture;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
     import org.apache.cassandra.concurrent.ScheduledExecutors;
     import org.apache.cassandra.config.DatabaseDescriptor;
    +import org.apache.cassandra.exceptions.RequestFailureReason;
     import org.apache.cassandra.gms.ApplicationState;
     import org.apache.cassandra.gms.EndpointState;
     import org.apache.cassandra.gms.Gossiper;
     import org.apache.cassandra.gms.VersionedValue;
    +import org.apache.cassandra.locator.dynamicsnitch.DynamicEndpointSnitchHistogram;
    +import org.apache.cassandra.net.IAsyncCallbackWithFailure;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +import org.apache.cassandra.net.MessageIn;
    +import org.apache.cassandra.net.MessageOut;
     import org.apache.cassandra.net.MessagingService;
    +import org.apache.cassandra.net.PingMessage;
     import org.apache.cassandra.service.StorageService;
     import org.apache.cassandra.utils.FBUtilities;
     
    +import static org.apache.cassandra.net.MessagingService.Verb.PING;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.LARGE_MESSAGE;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.SMALL_MESSAGE;
    +
    +
     /**
      * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector
    + * Note that the subclasses (e.g. {@link DynamicEndpointSnitchHistogram}) are responsible for actually measuring
    + * latency and providing an ISnitchMeasurement implementation back to this class.
      */
    -public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
    +public abstract class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
     {
    -    private static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    -
    -    private static final double ALPHA = 0.75; // set to 0.75 to make EDS more biased to towards the newer values
    -    private static final int WINDOW_SIZE = 100;
    -
    -    private volatile int dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    -    private volatile int dynamicResetInterval = DatabaseDescriptor.getDynamicResetInterval();
    -    private volatile double dynamicBadnessThreshold = DatabaseDescriptor.getDynamicBadnessThreshold();
    +    private static final Logger logger = LoggerFactory.getLogger(DynamicEndpointSnitch.class);
    +
    +    // Latency measurement and ranking. The samples contain latency measurements and the scores are used for ranking
    +    protected boolean registered = false;
    +    protected static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    protected volatile Map<InetAddressAndPort, Double> scores = new HashMap<>();
    +    protected final Map<InetAddressAndPort, AnnotatedMeasurement> samples = new ConcurrentHashMap<>();
    +
    +    // Latency probe functionality for actively probing endpoints that we haven't measured recently but are ranking
    +    public static final long MAX_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_max_probe_interval_ms", 60 * 10 * 1000L);
    +    public static final long MIN_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_min_probe_interval_ms", 60 * 1000L) ;
    +    // The probe rate is set later when configuration is read
    +    protected static final RateLimiter probeRateLimiter = RateLimiter.create(1);
    +    protected static final DebuggableScheduledThreadPoolExecutor latencyProbeExecutor = new DebuggableScheduledThreadPoolExecutor(1, "LatencyProbes", Thread.MIN_PRIORITY);
    +
    +    // User configuration of the snitch tunables
    +    protected volatile int dynamicUpdateInterval = -1;
    +    protected volatile int dynamicLatencyProbeInterval = -1;
    +    protected volatile double dynamicBadnessThreshold = 0;
     
         // the score for a merged set of endpoints must be this much worse than the score for separate endpoints to
         // warrant not merging two ranges into a single range
         private static final double RANGE_MERGING_PREFERENCE = 1.5;
     
         private String mbeanName;
    -    private boolean registered = false;
    -
    -    private volatile HashMap<InetAddressAndPort, Double> scores = new HashMap<>();
    -    private final ConcurrentHashMap<InetAddressAndPort, ExponentiallyDecayingReservoir> samples = new ConcurrentHashMap<>();
    +    private boolean mbeanRegistered = false;
     
         public final IEndpointSnitch subsnitch;
     
    -    private volatile ScheduledFuture<?> updateSchedular;
    -    private volatile ScheduledFuture<?> resetSchedular;
    -
    -    private final Runnable update;
    -    private final Runnable reset;
    +    private volatile ScheduledFuture<?> updateScoresScheduler;
    +    private volatile ScheduledFuture<?> updateSamplesScheduler;
     
         public DynamicEndpointSnitch(IEndpointSnitch snitch)
         {
             this(snitch, null);
         }
     
    -    public DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
    +    protected DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
         {
             mbeanName = "org.apache.cassandra.db:type=DynamicEndpointSnitch";
             if (instance != null)
                 mbeanName += ",instance=" + instance;
             subsnitch = snitch;
    -        update = new Runnable()
    +
    +        if (DatabaseDescriptor.isDaemonInitialized())
             {
    -            public void run()
    -            {
    -                updateScores();
    -            }
    -        };
    -        reset = new Runnable()
    +            applyConfigChanges(DatabaseDescriptor.getDynamicUpdateInterval(),
    +                               DatabaseDescriptor.getDynamicSampleUpdateInterval(),
    --- End diff --
    
    It's set within `applyConfigChanges`, I'll reorganize that method to make it more readable.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r236775944
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -23,128 +23,300 @@
     import java.net.UnknownHostException;
     import java.util.*;
     import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ExecutionException;
     import java.util.concurrent.ScheduledFuture;
     import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
     import java.util.stream.Collectors;
     
    -import com.codahale.metrics.ExponentiallyDecayingReservoir;
     import javax.management.MBeanServer;
     import javax.management.ObjectName;
     
    -import com.codahale.metrics.Snapshot;
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.util.concurrent.Futures;
    +import com.google.common.util.concurrent.MoreExecutors;
    +import com.google.common.util.concurrent.RateLimiter;
    +import com.google.common.util.concurrent.SettableFuture;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
     import org.apache.cassandra.concurrent.ScheduledExecutors;
     import org.apache.cassandra.config.DatabaseDescriptor;
    +import org.apache.cassandra.exceptions.RequestFailureReason;
     import org.apache.cassandra.gms.ApplicationState;
     import org.apache.cassandra.gms.EndpointState;
     import org.apache.cassandra.gms.Gossiper;
     import org.apache.cassandra.gms.VersionedValue;
    +import org.apache.cassandra.locator.dynamicsnitch.DynamicEndpointSnitchHistogram;
    +import org.apache.cassandra.net.IAsyncCallbackWithFailure;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +import org.apache.cassandra.net.MessageIn;
    +import org.apache.cassandra.net.MessageOut;
     import org.apache.cassandra.net.MessagingService;
    +import org.apache.cassandra.net.PingMessage;
     import org.apache.cassandra.service.StorageService;
     import org.apache.cassandra.utils.FBUtilities;
     
    +import static org.apache.cassandra.net.MessagingService.Verb.PING;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.LARGE_MESSAGE;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.SMALL_MESSAGE;
    +
    +
     /**
      * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector
    + * Note that the subclasses (e.g. {@link DynamicEndpointSnitchHistogram}) are responsible for actually measuring
    + * latency and providing an ISnitchMeasurement implementation back to this class.
      */
    -public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
    +public abstract class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
     {
    -    private static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    -
    -    private static final double ALPHA = 0.75; // set to 0.75 to make EDS more biased to towards the newer values
    -    private static final int WINDOW_SIZE = 100;
    -
    -    private volatile int dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    -    private volatile int dynamicResetInterval = DatabaseDescriptor.getDynamicResetInterval();
    -    private volatile double dynamicBadnessThreshold = DatabaseDescriptor.getDynamicBadnessThreshold();
    +    private static final Logger logger = LoggerFactory.getLogger(DynamicEndpointSnitch.class);
    +
    +    // Latency measurement and ranking. The samples contain latency measurements and the scores are used for ranking
    +    protected boolean registered = false;
    +    protected static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    protected volatile Map<InetAddressAndPort, Double> scores = new HashMap<>();
    +    protected final Map<InetAddressAndPort, AnnotatedMeasurement> samples = new ConcurrentHashMap<>();
    +
    +    // Latency probe functionality for actively probing endpoints that we haven't measured recently but are ranking
    +    public static final long MAX_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_max_probe_interval_ms", 60 * 10 * 1000L);
    +    public static final long MIN_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_min_probe_interval_ms", 60 * 1000L) ;
    +    // The probe rate is set later when configuration is read
    +    protected static final RateLimiter probeRateLimiter = RateLimiter.create(1);
    +    protected static final DebuggableScheduledThreadPoolExecutor latencyProbeExecutor = new DebuggableScheduledThreadPoolExecutor(1, "LatencyProbes", Thread.MIN_PRIORITY);
    --- End diff --
    
    I took it out per your recommendation.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r229483854
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -154,31 +326,203 @@ private void registerMBean()
     
         public void close()
         {
    -        updateSchedular.cancel(false);
    -        resetSchedular.cancel(false);
    +        if (updateScoresScheduler != null)
    +            updateScoresScheduler.cancel(false);
    +        if (updateSamplesScheduler != null)
    +            updateSamplesScheduler.cancel(false);
    +
    +        for (AnnotatedMeasurement measurement : samples.values())
    +        {
    +            if (measurement.probeFuture != null)
    +                measurement.probeFuture.cancel(false);
    +
    +            measurement.millisSinceLastMeasure.set(0);
    +            measurement.millisSinceLastRequest.set(MAX_PROBE_INTERVAL_MS);
    +            measurement.probeTimerMillis = 0;
    +        }
     
             MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
             try
             {
    -            mbs.unregisterMBean(new ObjectName(mbeanName));
    +            if (mbeanRegistered)
    +                mbs.unregisterMBean(new ObjectName(mbeanName));
             }
             catch (Exception e)
             {
                 throw new RuntimeException(e);
             }
         }
     
    +    /**
    +     * Background task running on the samples dictionary. The default implementation sends latency probes (PING)
    +     * messages to explore nodes that we have not received timings for recently but have ranked in
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}.
    +     */
    +    protected void updateSamples()
    +    {
    +        // Split calculation of probe timers from sending probes for testability
    +        calculateProbes(samples, dynamicLatencyProbeInterval);
    +
    +        if (!StorageService.instance.isGossipActive())
    +            return;
    +
    +        schedulePings(samples);
    +    }
    +
    +    /**
    +     * This method mutates the passed AnnotatedMeasurements to implement capped exponential backoff per endpoint.
    +     *
    +     * The algorithm is as follows:
    +     * 1. All samples get their millisSinceLastMeasure and millisSinceLastRequest fields
    +     *    incremented by the passed interval
    +     * 2. Any recently requested (ranked) endpoints that have not been measured recently (e.g. because the snitch
    +     *    has sent them no traffic) get probes with exponential backoff.
    +     *
    +     * The backoff is capped at MAX_PROBE_INTERVAL_MS. Furthermore the probes are stopped after
    +     * MAX_PROBE_INTERVAL_MS of no ranking requests as well.
    +     *
    +     * At the end of this method, any passed AnnotatedMeasurements that need latency probes will have non zero
    +     * probeTimerMillis members set.
    +     */
    +    @VisibleForTesting
    +    static void calculateProbes(Map<InetAddressAndPort, AnnotatedMeasurement> samples, long intervalMillis) {
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry: samples.entrySet())
    +        {
    +            if (entry.getKey().equals(FBUtilities.getBroadcastAddressAndPort()))
    +                continue;
    +
    +            AnnotatedMeasurement measurement = entry.getValue();
    +            long lastMeasure = measurement.millisSinceLastMeasure.getAndAdd(intervalMillis);
    --- End diff --
    
    I don't care too much now that `MIN_PROBE_INTERVAL` is 60s, when it's smaller (like 1s) I felt that we were probing too soon. I'll change it.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225361405
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -22,121 +22,194 @@
     import java.net.InetAddress;
     import java.net.UnknownHostException;
     import java.util.*;
    -import java.util.concurrent.ConcurrentHashMap;
     import java.util.concurrent.ScheduledFuture;
     import java.util.concurrent.TimeUnit;
     import java.util.stream.Collectors;
     
    -import com.codahale.metrics.ExponentiallyDecayingReservoir;
     import javax.management.MBeanServer;
     import javax.management.ObjectName;
     
    -import com.codahale.metrics.Snapshot;
    +import com.google.common.annotations.VisibleForTesting;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
     import org.apache.cassandra.concurrent.ScheduledExecutors;
     import org.apache.cassandra.config.DatabaseDescriptor;
     import org.apache.cassandra.gms.ApplicationState;
     import org.apache.cassandra.gms.EndpointState;
     import org.apache.cassandra.gms.Gossiper;
     import org.apache.cassandra.gms.VersionedValue;
    +import org.apache.cassandra.net.IAsyncCallback;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +import org.apache.cassandra.net.MessageIn;
    +import org.apache.cassandra.net.MessageOut;
     import org.apache.cassandra.net.MessagingService;
    +import org.apache.cassandra.net.PingMessage;
     import org.apache.cassandra.service.StorageService;
     import org.apache.cassandra.utils.FBUtilities;
     
    +import static org.apache.cassandra.net.MessagingService.Verb.PING;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.LARGE_MESSAGE;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.SMALL_MESSAGE;
    +
    +
     /**
      * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector
    + * Note that the subclasses (e.g. {@link DynamicEndpointSnitchHistogram}) are responsible for actually measuring
    + * latency and populating the {@link #scores} map.
      */
    -public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
    +public abstract class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
     {
    -    private static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    -
    -    private static final double ALPHA = 0.75; // set to 0.75 to make EDS more biased to towards the newer values
    -    private static final int WINDOW_SIZE = 100;
    -
    -    private volatile int dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    -    private volatile int dynamicResetInterval = DatabaseDescriptor.getDynamicResetInterval();
    -    private volatile double dynamicBadnessThreshold = DatabaseDescriptor.getDynamicBadnessThreshold();
    +    private static final Logger logger = LoggerFactory.getLogger(DynamicEndpointSnitch.class);
    +
    +    // Subclass specific functionality
    +    protected static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    protected boolean registered = false;
    +    // The scores map is updated via copy in updateScores
    +    // We keep it in the base class for performance reasons (so it can be easily aliased)
    +    protected volatile Map<InetAddressAndPort, Double> scores = new HashMap<>();
    +
    +    // Rate limit how often we generate latency probes
    +    protected long nextAllowedProbeGenerationTime;
    +    protected long nextProbeGenerationTime;
    +    protected int currentProbePosition = 0;
    +    protected final List<InetAddressAndPort> latencyProbeSequence = new ArrayList<>();
    +
    +    // DES general functionality
    +    protected volatile int dynamicUpdateInterval = -1;
    +    protected volatile int dynamicLatencyProbeInterval = -1;
    +    protected volatile double dynamicBadnessThreshold = 0;
     
         // the score for a merged set of endpoints must be this much worse than the score for separate endpoints to
         // warrant not merging two ranges into a single range
         private static final double RANGE_MERGING_PREFERENCE = 1.5;
     
         private String mbeanName;
    -    private boolean registered = false;
    -
    -    private volatile HashMap<InetAddressAndPort, Double> scores = new HashMap<>();
    -    private final ConcurrentHashMap<InetAddressAndPort, ExponentiallyDecayingReservoir> samples = new ConcurrentHashMap<>();
    +    private boolean mbeanRegistered = false;
     
         public final IEndpointSnitch subsnitch;
     
    -    private volatile ScheduledFuture<?> updateSchedular;
    -    private volatile ScheduledFuture<?> resetSchedular;
    +    private volatile ScheduledFuture<?> updateScheduler;
    +    private volatile ScheduledFuture<?> latencyProbeScheduler;
     
    -    private final Runnable update;
    -    private final Runnable reset;
    +    private final Runnable update = this::updateScores;
    +    private final Runnable latencyProbe = this::maybeSendLatencyProbe;
     
         public DynamicEndpointSnitch(IEndpointSnitch snitch)
         {
             this(snitch, null);
         }
     
    -    public DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
    +    protected DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
         {
             mbeanName = "org.apache.cassandra.db:type=DynamicEndpointSnitch";
             if (instance != null)
                 mbeanName += ",instance=" + instance;
             subsnitch = snitch;
    -        update = new Runnable()
    -        {
    -            public void run()
    -            {
    -                updateScores();
    -            }
    -        };
    -        reset = new Runnable()
    -        {
    -            public void run()
    -            {
    -                // we do this so that a host considered bad has a chance to recover, otherwise would we never try
    -                // to read from it, which would cause its score to never change
    -                reset();
    -            }
    -        };
     
             if (DatabaseDescriptor.isDaemonInitialized())
             {
    -            updateSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(update, dynamicUpdateInterval, dynamicUpdateInterval, TimeUnit.MILLISECONDS);
    -            resetSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(reset, dynamicResetInterval, dynamicResetInterval, TimeUnit.MILLISECONDS);
    +            applyConfigChanges();
                 registerMBean();
             }
    +        nextProbeGenerationTime = System.nanoTime();
    +    }
    +
    +    /**
    +     * Records a latency. This MUST be cheap as it is called in the fast path
    +     */
    +    abstract public void receiveTiming(InetAddressAndPort address, long latency, LatencyMeasurementType measurementType);
    --- End diff --
    
    correct, and storing the latency is basically all either implementation does. As part of pulling samples (along with the annotations) back to the base class I'll see how much of this I can unify.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r238167792
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -154,20 +228,101 @@ private void registerMBean()
     
         public void close()
         {
    -        updateSchedular.cancel(false);
    -        resetSchedular.cancel(false);
    +        if (updateScheduler != null)
    +            updateScheduler.cancel(false);
    +        if (latencyProbeScheduler != null)
    +            latencyProbeScheduler.cancel(false);
     
             MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
             try
             {
    -            mbs.unregisterMBean(new ObjectName(mbeanName));
    +            if (mbeanRegistered)
    +                mbs.unregisterMBean(new ObjectName(mbeanName));
             }
             catch (Exception e)
             {
                 throw new RuntimeException(e);
             }
         }
     
    +    /**
    +     * Determines if latency probes need to be sent, and potentially sends a single latency probe per invocation
    +     */
    +    protected void maybeSendLatencyProbe()
    +    {
    +        long currentTimeNS = System.nanoTime();
    +        markNextAllowedProbeGenerationTime(currentTimeNS);
    +
    +        Optional<InetAddressAndPort> needsProbe = latencyProbeNeeded(currentTimeNS);
    +        needsProbe.ifPresent(this::sendPingMessageToPeer);
    +    }
    +
    +    /**
    +     * Determines which peers need latency at a particular time. Note that this takes currentTimeNS for testability
    +     * of this code path.
    +     * @param currentTimeNS The current time to evaluate. Used mostly for testing.
    +     * @return An Optional that if present contains a host to probe.
    +     */
    +    @VisibleForTesting
    +    Optional<InetAddressAndPort> latencyProbeNeeded(long currentTimeNS) {
    +        if (currentProbePosition >= latencyProbeSequence.size() && (currentTimeNS > nextProbeGenerationTime))
    +        {
    +            nextProbeGenerationTime = nextAllowedProbeGenerationTime;
    +            latencyProbeSequence.clear();
    +
    +            // Delegate to the subclass to actually figure out what the probe sequence should be
    +            updateLatencyProbeSequence(latencyProbeSequence);
    +
    +            if (latencyProbeSequence.size() > 0)
    +                Collections.shuffle(latencyProbeSequence);
    +
    +            currentProbePosition = 0;
    +        }
    +
    +        if (currentProbePosition < latencyProbeSequence.size())
    +        {
    +            try
    +            {
    +                return Optional.of(latencyProbeSequence.get(currentProbePosition++));
    +            }
    +            catch (IndexOutOfBoundsException ignored) {}
    +        }
    +
    +        return Optional.empty();
    +    }
    +
    +    private void sendPingMessageToPeer(InetAddressAndPort to)
    +    {
    +        logger.trace("Sending a small and large PingMessage to {}", to);
    +        IAsyncCallback latencyProbeHandler = new IAsyncCallback()
    --- End diff --
    
    In the latest version I kept the changed API but I can remove it you think it'll make the PR easier to review/merge.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r229470860
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -154,31 +326,203 @@ private void registerMBean()
     
         public void close()
         {
    -        updateSchedular.cancel(false);
    -        resetSchedular.cancel(false);
    +        if (updateScoresScheduler != null)
    +            updateScoresScheduler.cancel(false);
    +        if (updateSamplesScheduler != null)
    +            updateSamplesScheduler.cancel(false);
    +
    +        for (AnnotatedMeasurement measurement : samples.values())
    +        {
    +            if (measurement.probeFuture != null)
    +                measurement.probeFuture.cancel(false);
    +
    +            measurement.millisSinceLastMeasure.set(0);
    +            measurement.millisSinceLastRequest.set(MAX_PROBE_INTERVAL_MS);
    +            measurement.probeTimerMillis = 0;
    +        }
     
             MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
             try
             {
    -            mbs.unregisterMBean(new ObjectName(mbeanName));
    +            if (mbeanRegistered)
    +                mbs.unregisterMBean(new ObjectName(mbeanName));
             }
             catch (Exception e)
             {
                 throw new RuntimeException(e);
             }
         }
     
    +    /**
    +     * Background task running on the samples dictionary. The default implementation sends latency probes (PING)
    +     * messages to explore nodes that we have not received timings for recently but have ranked in
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}.
    +     */
    +    protected void updateSamples()
    +    {
    +        // Split calculation of probe timers from sending probes for testability
    +        calculateProbes(samples, dynamicLatencyProbeInterval);
    +
    +        if (!StorageService.instance.isGossipActive())
    +            return;
    +
    +        schedulePings(samples);
    +    }
    +
    +    /**
    +     * This method mutates the passed AnnotatedMeasurements to implement capped exponential backoff per endpoint.
    +     *
    +     * The algorithm is as follows:
    +     * 1. All samples get their millisSinceLastMeasure and millisSinceLastRequest fields
    +     *    incremented by the passed interval
    +     * 2. Any recently requested (ranked) endpoints that have not been measured recently (e.g. because the snitch
    +     *    has sent them no traffic) get probes with exponential backoff.
    +     *
    +     * The backoff is capped at MAX_PROBE_INTERVAL_MS. Furthermore the probes are stopped after
    +     * MAX_PROBE_INTERVAL_MS of no ranking requests as well.
    +     *
    +     * At the end of this method, any passed AnnotatedMeasurements that need latency probes will have non zero
    +     * probeTimerMillis members set.
    +     */
    +    @VisibleForTesting
    +    static void calculateProbes(Map<InetAddressAndPort, AnnotatedMeasurement> samples, long intervalMillis) {
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry: samples.entrySet())
    +        {
    +            if (entry.getKey().equals(FBUtilities.getBroadcastAddressAndPort()))
    +                continue;
    +
    +            AnnotatedMeasurement measurement = entry.getValue();
    +            long lastMeasure = measurement.millisSinceLastMeasure.getAndAdd(intervalMillis);
    +            long lastRequest = measurement.millisSinceLastRequest.getAndAdd(intervalMillis);
    +
    +            if (lastMeasure >= MIN_PROBE_INTERVAL_MS && lastRequest < MAX_PROBE_INTERVAL_MS)
    +            {
    +                if (measurement.probeTimerMillis == 0)
    +                {
    +                    measurement.probeTimerMillis = intervalMillis;
    +                }
    +                else if (measurement.probeFuture != null && measurement.probeFuture.isDone())
    +                {
    +                    measurement.probeTimerMillis = Math.min(MAX_PROBE_INTERVAL_MS, measurement.probeTimerMillis * 2);
    --- End diff --
    
    So this is going to only schedule the next probe MIN_PROBE_INTERVAL_MS after the previous probe has been sent? So every probe is going to have MIN_PROBE_INTERVAL_MS delay added to it on top of whatever the delay is supposed to be?
    
    Seems like the first probe always goes out after 2x MIN_PROBE_INTERVAL_MS? It takes MIN_PROBE_INTERVAL_MS before we check if it is needed and then it's delayed for 2x MIN_PROBE_INTERVAL_MS when the right thing to do is probe immediately.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225360796
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -22,121 +22,194 @@
     import java.net.InetAddress;
     import java.net.UnknownHostException;
     import java.util.*;
    -import java.util.concurrent.ConcurrentHashMap;
     import java.util.concurrent.ScheduledFuture;
     import java.util.concurrent.TimeUnit;
     import java.util.stream.Collectors;
     
    -import com.codahale.metrics.ExponentiallyDecayingReservoir;
     import javax.management.MBeanServer;
     import javax.management.ObjectName;
     
    -import com.codahale.metrics.Snapshot;
    +import com.google.common.annotations.VisibleForTesting;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
     import org.apache.cassandra.concurrent.ScheduledExecutors;
     import org.apache.cassandra.config.DatabaseDescriptor;
     import org.apache.cassandra.gms.ApplicationState;
     import org.apache.cassandra.gms.EndpointState;
     import org.apache.cassandra.gms.Gossiper;
     import org.apache.cassandra.gms.VersionedValue;
    +import org.apache.cassandra.net.IAsyncCallback;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +import org.apache.cassandra.net.MessageIn;
    +import org.apache.cassandra.net.MessageOut;
     import org.apache.cassandra.net.MessagingService;
    +import org.apache.cassandra.net.PingMessage;
     import org.apache.cassandra.service.StorageService;
     import org.apache.cassandra.utils.FBUtilities;
     
    +import static org.apache.cassandra.net.MessagingService.Verb.PING;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.LARGE_MESSAGE;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.SMALL_MESSAGE;
    +
    +
     /**
      * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector
    + * Note that the subclasses (e.g. {@link DynamicEndpointSnitchHistogram}) are responsible for actually measuring
    + * latency and populating the {@link #scores} map.
      */
    -public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
    +public abstract class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
     {
    -    private static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    -
    -    private static final double ALPHA = 0.75; // set to 0.75 to make EDS more biased to towards the newer values
    -    private static final int WINDOW_SIZE = 100;
    -
    -    private volatile int dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    -    private volatile int dynamicResetInterval = DatabaseDescriptor.getDynamicResetInterval();
    -    private volatile double dynamicBadnessThreshold = DatabaseDescriptor.getDynamicBadnessThreshold();
    +    private static final Logger logger = LoggerFactory.getLogger(DynamicEndpointSnitch.class);
    +
    +    // Subclass specific functionality
    +    protected static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    protected boolean registered = false;
    +    // The scores map is updated via copy in updateScores
    +    // We keep it in the base class for performance reasons (so it can be easily aliased)
    +    protected volatile Map<InetAddressAndPort, Double> scores = new HashMap<>();
    +
    +    // Rate limit how often we generate latency probes
    +    protected long nextAllowedProbeGenerationTime;
    +    protected long nextProbeGenerationTime;
    +    protected int currentProbePosition = 0;
    +    protected final List<InetAddressAndPort> latencyProbeSequence = new ArrayList<>();
    +
    +    // DES general functionality
    +    protected volatile int dynamicUpdateInterval = -1;
    +    protected volatile int dynamicLatencyProbeInterval = -1;
    +    protected volatile double dynamicBadnessThreshold = 0;
     
         // the score for a merged set of endpoints must be this much worse than the score for separate endpoints to
         // warrant not merging two ranges into a single range
         private static final double RANGE_MERGING_PREFERENCE = 1.5;
     
         private String mbeanName;
    -    private boolean registered = false;
    -
    -    private volatile HashMap<InetAddressAndPort, Double> scores = new HashMap<>();
    -    private final ConcurrentHashMap<InetAddressAndPort, ExponentiallyDecayingReservoir> samples = new ConcurrentHashMap<>();
    +    private boolean mbeanRegistered = false;
     
         public final IEndpointSnitch subsnitch;
     
    -    private volatile ScheduledFuture<?> updateSchedular;
    -    private volatile ScheduledFuture<?> resetSchedular;
    +    private volatile ScheduledFuture<?> updateScheduler;
    +    private volatile ScheduledFuture<?> latencyProbeScheduler;
     
    -    private final Runnable update;
    -    private final Runnable reset;
    +    private final Runnable update = this::updateScores;
    +    private final Runnable latencyProbe = this::maybeSendLatencyProbe;
     
         public DynamicEndpointSnitch(IEndpointSnitch snitch)
         {
             this(snitch, null);
         }
     
    -    public DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
    +    protected DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
         {
             mbeanName = "org.apache.cassandra.db:type=DynamicEndpointSnitch";
             if (instance != null)
                 mbeanName += ",instance=" + instance;
             subsnitch = snitch;
    -        update = new Runnable()
    -        {
    -            public void run()
    -            {
    -                updateScores();
    -            }
    -        };
    -        reset = new Runnable()
    -        {
    -            public void run()
    -            {
    -                // we do this so that a host considered bad has a chance to recover, otherwise would we never try
    -                // to read from it, which would cause its score to never change
    -                reset();
    -            }
    -        };
     
             if (DatabaseDescriptor.isDaemonInitialized())
             {
    -            updateSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(update, dynamicUpdateInterval, dynamicUpdateInterval, TimeUnit.MILLISECONDS);
    -            resetSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(reset, dynamicResetInterval, dynamicResetInterval, TimeUnit.MILLISECONDS);
    +            applyConfigChanges();
                 registerMBean();
             }
    +        nextProbeGenerationTime = System.nanoTime();
    +    }
    +
    +    /**
    +     * Records a latency. This MUST be cheap as it is called in the fast path
    +     */
    +    abstract public void receiveTiming(InetAddressAndPort address, long latency, LatencyMeasurementType measurementType);
    +
    +    /**
    +     * Called periodically from {@link DynamicEndpointSnitch#updateScores()}. This method returns a new score map
    +     * that will be assigned to the {@link DynamicEndpointSnitch#scores} map.
    +     * This is generally more expensive than {@link DynamicEndpointSnitch#receiveTiming}
    +     */
    +    abstract public Map<InetAddressAndPort, Double> calculateScores();
    +
    +    /**
    +     * Signals that we actually tried to rank vs this host. This way any latency probing can just focus
    +     * on hosts that this peer would potentially talk to.
    +     * @param address
    +     */
    +    abstract protected void markRequested(InetAddressAndPort address);
    --- End diff --
    
    It was there because I had the wrapped metrics (reservoir/EMA) in the subclasses (as they don't both e.g. conform to some interface that I can then compose on top of). I'm trying to implement your suggestion of moving samples back to base class and along the way I wouldn't need this anymore.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r232435926
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -23,128 +23,300 @@
     import java.net.UnknownHostException;
     import java.util.*;
     import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ExecutionException;
     import java.util.concurrent.ScheduledFuture;
     import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
     import java.util.stream.Collectors;
     
    -import com.codahale.metrics.ExponentiallyDecayingReservoir;
     import javax.management.MBeanServer;
     import javax.management.ObjectName;
     
    -import com.codahale.metrics.Snapshot;
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.util.concurrent.Futures;
    +import com.google.common.util.concurrent.MoreExecutors;
    +import com.google.common.util.concurrent.RateLimiter;
    +import com.google.common.util.concurrent.SettableFuture;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
     import org.apache.cassandra.concurrent.ScheduledExecutors;
     import org.apache.cassandra.config.DatabaseDescriptor;
    +import org.apache.cassandra.exceptions.RequestFailureReason;
     import org.apache.cassandra.gms.ApplicationState;
     import org.apache.cassandra.gms.EndpointState;
     import org.apache.cassandra.gms.Gossiper;
     import org.apache.cassandra.gms.VersionedValue;
    +import org.apache.cassandra.locator.dynamicsnitch.DynamicEndpointSnitchHistogram;
    +import org.apache.cassandra.net.IAsyncCallbackWithFailure;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +import org.apache.cassandra.net.MessageIn;
    +import org.apache.cassandra.net.MessageOut;
     import org.apache.cassandra.net.MessagingService;
    +import org.apache.cassandra.net.PingMessage;
     import org.apache.cassandra.service.StorageService;
     import org.apache.cassandra.utils.FBUtilities;
     
    +import static org.apache.cassandra.net.MessagingService.Verb.PING;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.LARGE_MESSAGE;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.SMALL_MESSAGE;
    +
    +
     /**
      * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector
    + * Note that the subclasses (e.g. {@link DynamicEndpointSnitchHistogram}) are responsible for actually measuring
    + * latency and providing an ISnitchMeasurement implementation back to this class.
      */
    -public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
    +public abstract class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
     {
    -    private static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    -
    -    private static final double ALPHA = 0.75; // set to 0.75 to make EDS more biased to towards the newer values
    -    private static final int WINDOW_SIZE = 100;
    -
    -    private volatile int dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    -    private volatile int dynamicResetInterval = DatabaseDescriptor.getDynamicResetInterval();
    -    private volatile double dynamicBadnessThreshold = DatabaseDescriptor.getDynamicBadnessThreshold();
    +    private static final Logger logger = LoggerFactory.getLogger(DynamicEndpointSnitch.class);
    +
    +    // Latency measurement and ranking. The samples contain latency measurements and the scores are used for ranking
    +    protected boolean registered = false;
    +    protected static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    protected volatile Map<InetAddressAndPort, Double> scores = new HashMap<>();
    +    protected final Map<InetAddressAndPort, AnnotatedMeasurement> samples = new ConcurrentHashMap<>();
    +
    +    // Latency probe functionality for actively probing endpoints that we haven't measured recently but are ranking
    +    public static final long MAX_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_max_probe_interval_ms", 60 * 10 * 1000L);
    +    public static final long MIN_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_min_probe_interval_ms", 60 * 1000L) ;
    +    // The probe rate is set later when configuration is read
    +    protected static final RateLimiter probeRateLimiter = RateLimiter.create(1);
    +    protected static final DebuggableScheduledThreadPoolExecutor latencyProbeExecutor = new DebuggableScheduledThreadPoolExecutor(1, "LatencyProbes", Thread.MIN_PRIORITY);
    +
    +    // User configuration of the snitch tunables
    +    protected volatile int dynamicUpdateInterval = -1;
    +    protected volatile int dynamicLatencyProbeInterval = -1;
    +    protected volatile double dynamicBadnessThreshold = 0;
     
         // the score for a merged set of endpoints must be this much worse than the score for separate endpoints to
         // warrant not merging two ranges into a single range
         private static final double RANGE_MERGING_PREFERENCE = 1.5;
     
         private String mbeanName;
    -    private boolean registered = false;
    -
    -    private volatile HashMap<InetAddressAndPort, Double> scores = new HashMap<>();
    -    private final ConcurrentHashMap<InetAddressAndPort, ExponentiallyDecayingReservoir> samples = new ConcurrentHashMap<>();
    +    private boolean mbeanRegistered = false;
     
         public final IEndpointSnitch subsnitch;
     
    -    private volatile ScheduledFuture<?> updateSchedular;
    -    private volatile ScheduledFuture<?> resetSchedular;
    -
    -    private final Runnable update;
    -    private final Runnable reset;
    +    private volatile ScheduledFuture<?> updateScoresScheduler;
    +    private volatile ScheduledFuture<?> updateSamplesScheduler;
     
         public DynamicEndpointSnitch(IEndpointSnitch snitch)
         {
             this(snitch, null);
         }
     
    -    public DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
    +    protected DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
         {
             mbeanName = "org.apache.cassandra.db:type=DynamicEndpointSnitch";
             if (instance != null)
                 mbeanName += ",instance=" + instance;
             subsnitch = snitch;
    -        update = new Runnable()
    +
    +        if (DatabaseDescriptor.isDaemonInitialized())
             {
    -            public void run()
    -            {
    -                updateScores();
    -            }
    -        };
    -        reset = new Runnable()
    +            applyConfigChanges(DatabaseDescriptor.getDynamicUpdateInterval(),
    +                               DatabaseDescriptor.getDynamicSampleUpdateInterval(),
    +                               DatabaseDescriptor.getDynamicBadnessThreshold());
    +            open();
    +        }
    +    }
    +
    +    /**
    +     * Allows subclasses to inject new ways of measuring latency back to this abstract base class.
    +     */
    +    protected interface ISnitchMeasurement
    +    {
    +        void sample(long value);
    +        double measure();
    +        Iterable<Double> measurements();
    +    }
    +
    +    /**
    +     * Adds some boookeeping that the DES uses over top of the various metrics techniques used by the
    +     * implementations. This is used to allow CASSANDRA-14459 latency probes as well as further experimentation
    +     * on new latency measurement techniques in CASSANDRA-14817
    +     *
    +     * {@link AnnotatedMeasurement#millisSinceLastRequest} is set to zero through
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}. It defaults to
    +     * the maximum interval so that we only start probing once it has been ranked at least once
    +     * {@link AnnotatedMeasurement#millisSinceLastMeasure} is set to zero from
    +     * {@link DynamicEndpointSnitch#receiveTiming(InetAddressAndPort, long, LatencyMeasurementType)}.
    +     *
    +     * {@link AnnotatedMeasurement#millisSinceLastMeasure and {@link AnnotatedMeasurement#probeTimerMillis}
    +     * are incremented via {@link DynamicEndpointSnitch#updateSamples()}
    +     */
    +    protected static class AnnotatedMeasurement
    +    {
    +        // Used to optimally target latency probes only on nodes that are both requested for ranking
    +        // and are not being measured. For example with token aware clients a large portion of the cluster will never
    +        // be ranked at all and therefore we won't probe them.
    +        public AtomicLong millisSinceLastRequest = new AtomicLong(MAX_PROBE_INTERVAL_MS);
    +        public AtomicLong millisSinceLastMeasure = new AtomicLong(0);
    +        public long probeTimerMillis = 0;
    +        public ScheduledFuture<?> probeFuture = null;
    --- End diff --
    
    Done.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r229459705
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -154,31 +326,203 @@ private void registerMBean()
     
         public void close()
         {
    -        updateSchedular.cancel(false);
    -        resetSchedular.cancel(false);
    +        if (updateScoresScheduler != null)
    +            updateScoresScheduler.cancel(false);
    +        if (updateSamplesScheduler != null)
    +            updateSamplesScheduler.cancel(false);
    +
    +        for (AnnotatedMeasurement measurement : samples.values())
    +        {
    +            if (measurement.probeFuture != null)
    +                measurement.probeFuture.cancel(false);
    +
    +            measurement.millisSinceLastMeasure.set(0);
    +            measurement.millisSinceLastRequest.set(MAX_PROBE_INTERVAL_MS);
    +            measurement.probeTimerMillis = 0;
    +        }
     
             MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
             try
             {
    -            mbs.unregisterMBean(new ObjectName(mbeanName));
    +            if (mbeanRegistered)
    +                mbs.unregisterMBean(new ObjectName(mbeanName));
             }
             catch (Exception e)
             {
                 throw new RuntimeException(e);
             }
         }
     
    +    /**
    +     * Background task running on the samples dictionary. The default implementation sends latency probes (PING)
    +     * messages to explore nodes that we have not received timings for recently but have ranked in
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}.
    +     */
    +    protected void updateSamples()
    +    {
    +        // Split calculation of probe timers from sending probes for testability
    +        calculateProbes(samples, dynamicLatencyProbeInterval);
    +
    +        if (!StorageService.instance.isGossipActive())
    +            return;
    +
    +        schedulePings(samples);
    +    }
    +
    +    /**
    +     * This method mutates the passed AnnotatedMeasurements to implement capped exponential backoff per endpoint.
    +     *
    +     * The algorithm is as follows:
    +     * 1. All samples get their millisSinceLastMeasure and millisSinceLastRequest fields
    +     *    incremented by the passed interval
    +     * 2. Any recently requested (ranked) endpoints that have not been measured recently (e.g. because the snitch
    +     *    has sent them no traffic) get probes with exponential backoff.
    +     *
    +     * The backoff is capped at MAX_PROBE_INTERVAL_MS. Furthermore the probes are stopped after
    +     * MAX_PROBE_INTERVAL_MS of no ranking requests as well.
    +     *
    +     * At the end of this method, any passed AnnotatedMeasurements that need latency probes will have non zero
    +     * probeTimerMillis members set.
    +     */
    +    @VisibleForTesting
    +    static void calculateProbes(Map<InetAddressAndPort, AnnotatedMeasurement> samples, long intervalMillis) {
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry: samples.entrySet())
    +        {
    +            if (entry.getKey().equals(FBUtilities.getBroadcastAddressAndPort()))
    +                continue;
    +
    +            AnnotatedMeasurement measurement = entry.getValue();
    +            long lastMeasure = measurement.millisSinceLastMeasure.getAndAdd(intervalMillis);
    --- End diff --
    
    Should these be addAndGet?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225347227
  
    --- Diff: doc/source/operating/snitch.rst ---
    @@ -28,16 +28,18 @@ In cassandra, the snitch has two functions:
     
     Dynamic snitching
     ^^^^^^^^^^^^^^^^^
    -
     The dynamic snitch monitor read latencies to avoid reading from hosts that have slowed down. The dynamic snitch is
     configured with the following properties on ``cassandra.yaml``:
     
     - ``dynamic_snitch``: whether the dynamic snitch should be enabled or disabled.
     - ``dynamic_snitch_update_interval_in_ms``: controls how often to perform the more expensive part of host score
       calculation.
    -- ``dynamic_snitch_reset_interval_in_ms``: if set greater than zero, this will allow 'pinning' of replicas to hosts
    -  in order to increase cache capacity.
    -- ``dynamic_snitch_badness_threshold:``: The badness threshold will control how much worse the pinned host has to be
    +- ``dynamic_snitch_latency_probe_interval_in_ms``: controls how often this node
    --- End diff --
    
    Hm, is this pluggability something that we want to expose to users? I thought that it was just going to be so that devs can experiment with different implementations without going through a cassandra release every time (e.g. we can actually run various versions "in production").
    
    If you think it's a good idea to let users know that they can plugin their own DES class, I can make the change, but I thought it was just a way to allow us to experiment until we actually have good data on which approach is "best".


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r229477785
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -23,128 +23,300 @@
     import java.net.UnknownHostException;
     import java.util.*;
     import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ExecutionException;
     import java.util.concurrent.ScheduledFuture;
     import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
     import java.util.stream.Collectors;
     
    -import com.codahale.metrics.ExponentiallyDecayingReservoir;
     import javax.management.MBeanServer;
     import javax.management.ObjectName;
     
    -import com.codahale.metrics.Snapshot;
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.util.concurrent.Futures;
    +import com.google.common.util.concurrent.MoreExecutors;
    +import com.google.common.util.concurrent.RateLimiter;
    +import com.google.common.util.concurrent.SettableFuture;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
     import org.apache.cassandra.concurrent.ScheduledExecutors;
     import org.apache.cassandra.config.DatabaseDescriptor;
    +import org.apache.cassandra.exceptions.RequestFailureReason;
     import org.apache.cassandra.gms.ApplicationState;
     import org.apache.cassandra.gms.EndpointState;
     import org.apache.cassandra.gms.Gossiper;
     import org.apache.cassandra.gms.VersionedValue;
    +import org.apache.cassandra.locator.dynamicsnitch.DynamicEndpointSnitchHistogram;
    +import org.apache.cassandra.net.IAsyncCallbackWithFailure;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +import org.apache.cassandra.net.MessageIn;
    +import org.apache.cassandra.net.MessageOut;
     import org.apache.cassandra.net.MessagingService;
    +import org.apache.cassandra.net.PingMessage;
     import org.apache.cassandra.service.StorageService;
     import org.apache.cassandra.utils.FBUtilities;
     
    +import static org.apache.cassandra.net.MessagingService.Verb.PING;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.LARGE_MESSAGE;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.SMALL_MESSAGE;
    +
    +
     /**
      * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector
    + * Note that the subclasses (e.g. {@link DynamicEndpointSnitchHistogram}) are responsible for actually measuring
    + * latency and providing an ISnitchMeasurement implementation back to this class.
      */
    -public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
    +public abstract class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
     {
    -    private static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    -
    -    private static final double ALPHA = 0.75; // set to 0.75 to make EDS more biased to towards the newer values
    -    private static final int WINDOW_SIZE = 100;
    -
    -    private volatile int dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    -    private volatile int dynamicResetInterval = DatabaseDescriptor.getDynamicResetInterval();
    -    private volatile double dynamicBadnessThreshold = DatabaseDescriptor.getDynamicBadnessThreshold();
    +    private static final Logger logger = LoggerFactory.getLogger(DynamicEndpointSnitch.class);
    +
    +    // Latency measurement and ranking. The samples contain latency measurements and the scores are used for ranking
    +    protected boolean registered = false;
    +    protected static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    protected volatile Map<InetAddressAndPort, Double> scores = new HashMap<>();
    +    protected final Map<InetAddressAndPort, AnnotatedMeasurement> samples = new ConcurrentHashMap<>();
    +
    +    // Latency probe functionality for actively probing endpoints that we haven't measured recently but are ranking
    +    public static final long MAX_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_max_probe_interval_ms", 60 * 10 * 1000L);
    +    public static final long MIN_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_min_probe_interval_ms", 60 * 1000L) ;
    +    // The probe rate is set later when configuration is read
    +    protected static final RateLimiter probeRateLimiter = RateLimiter.create(1);
    +    protected static final DebuggableScheduledThreadPoolExecutor latencyProbeExecutor = new DebuggableScheduledThreadPoolExecutor(1, "LatencyProbes", Thread.MIN_PRIORITY);
    +
    +    // User configuration of the snitch tunables
    +    protected volatile int dynamicUpdateInterval = -1;
    +    protected volatile int dynamicLatencyProbeInterval = -1;
    +    protected volatile double dynamicBadnessThreshold = 0;
     
         // the score for a merged set of endpoints must be this much worse than the score for separate endpoints to
         // warrant not merging two ranges into a single range
         private static final double RANGE_MERGING_PREFERENCE = 1.5;
     
         private String mbeanName;
    -    private boolean registered = false;
    -
    -    private volatile HashMap<InetAddressAndPort, Double> scores = new HashMap<>();
    -    private final ConcurrentHashMap<InetAddressAndPort, ExponentiallyDecayingReservoir> samples = new ConcurrentHashMap<>();
    +    private boolean mbeanRegistered = false;
     
         public final IEndpointSnitch subsnitch;
     
    -    private volatile ScheduledFuture<?> updateSchedular;
    -    private volatile ScheduledFuture<?> resetSchedular;
    -
    -    private final Runnable update;
    -    private final Runnable reset;
    +    private volatile ScheduledFuture<?> updateScoresScheduler;
    +    private volatile ScheduledFuture<?> updateSamplesScheduler;
     
         public DynamicEndpointSnitch(IEndpointSnitch snitch)
         {
             this(snitch, null);
         }
     
    -    public DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
    +    protected DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
         {
             mbeanName = "org.apache.cassandra.db:type=DynamicEndpointSnitch";
             if (instance != null)
                 mbeanName += ",instance=" + instance;
             subsnitch = snitch;
    -        update = new Runnable()
    +
    +        if (DatabaseDescriptor.isDaemonInitialized())
             {
    -            public void run()
    -            {
    -                updateScores();
    -            }
    -        };
    -        reset = new Runnable()
    +            applyConfigChanges(DatabaseDescriptor.getDynamicUpdateInterval(),
    +                               DatabaseDescriptor.getDynamicSampleUpdateInterval(),
    +                               DatabaseDescriptor.getDynamicBadnessThreshold());
    +            open();
    +        }
    +    }
    +
    +    /**
    +     * Allows subclasses to inject new ways of measuring latency back to this abstract base class.
    +     */
    +    protected interface ISnitchMeasurement
    +    {
    +        void sample(long value);
    +        double measure();
    +        Iterable<Double> measurements();
    +    }
    +
    +    /**
    +     * Adds some boookeeping that the DES uses over top of the various metrics techniques used by the
    +     * implementations. This is used to allow CASSANDRA-14459 latency probes as well as further experimentation
    +     * on new latency measurement techniques in CASSANDRA-14817
    +     *
    +     * {@link AnnotatedMeasurement#millisSinceLastRequest} is set to zero through
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}. It defaults to
    +     * the maximum interval so that we only start probing once it has been ranked at least once
    +     * {@link AnnotatedMeasurement#millisSinceLastMeasure} is set to zero from
    +     * {@link DynamicEndpointSnitch#receiveTiming(InetAddressAndPort, long, LatencyMeasurementType)}.
    +     *
    +     * {@link AnnotatedMeasurement#millisSinceLastMeasure and {@link AnnotatedMeasurement#probeTimerMillis}
    +     * are incremented via {@link DynamicEndpointSnitch#updateSamples()}
    +     */
    +    protected static class AnnotatedMeasurement
    +    {
    +        // Used to optimally target latency probes only on nodes that are both requested for ranking
    +        // and are not being measured. For example with token aware clients a large portion of the cluster will never
    +        // be ranked at all and therefore we won't probe them.
    +        public AtomicLong millisSinceLastRequest = new AtomicLong(MAX_PROBE_INTERVAL_MS);
    +        public AtomicLong millisSinceLastMeasure = new AtomicLong(0);
    +        public long probeTimerMillis = 0;
    +        public ScheduledFuture<?> probeFuture = null;
    +
    +        // The underlying measurement technique. E.g. a median filter (histogram) or an EMA filter, or ...
    +        public final ISnitchMeasurement measurement;
    +
    +        public AnnotatedMeasurement(ISnitchMeasurement measurement)
    +        {
    +            this.measurement = measurement;
    +        }
    +
    +        @Override
    +        public String toString()
    +        {
    +            return "AnnotatedMeasurement{" +
    +                   "millisSinceLastRequest=" + millisSinceLastRequest +
    +                   ", millisSinceLastMeasure=" + millisSinceLastMeasure +
    +                   ", probeTimerMillis=" + probeTimerMillis +
    +                   ", probeFuturePending=" + (probeFuture != null && !probeFuture.isDone()) +
    +                   ", measurementClass=" + measurement.getClass().getSimpleName() +
    +                   '}';
    +        }
    +    }
    +
    +    /**
    +     * Allows the subclasses to inject ISnitchMeasurement instances back into this common class so that common
    +     * functionality can be handled here
    +     * @param initialValue The initial value of the measurement, some implementations may use this, others may not
    +     * @return a constructed instance of an ISnitchMeasurement interface
    +     */
    +    abstract protected ISnitchMeasurement measurementImpl(long initialValue);
    +
    +    /**
    +     * Records a latency. This MUST be cheap as it is called in the fast path
    +     */
    +    public void receiveTiming(InetAddressAndPort address, long latency, LatencyMeasurementType measurementType)
    +    {
    +        if (measurementType == LatencyMeasurementType.IGNORE)
    +           return;
    +
    +        AnnotatedMeasurement sample = samples.get(address);
    +
    +        if (sample == null)
    +        {
    +            AnnotatedMeasurement maybeNewSample = new AnnotatedMeasurement(measurementImpl(latency));
    +            sample = samples.putIfAbsent(address, maybeNewSample);
    +            if (sample == null)
    +                sample = maybeNewSample;
    +        }
    +
    +        if (measurementType == LatencyMeasurementType.READ && sample.millisSinceLastMeasure.get() > 0)
    +            sample.millisSinceLastMeasure.lazySet(0);
    +
    +        sample.measurement.sample(latency);
    +    }
    +
    +    @VisibleForTesting
    +    protected void reset()
    +    {
    +        scores.clear();
    +        samples.clear();
    +    }
    +
    +    @VisibleForTesting
    +    void updateScores()
    +    {
    +        if (!StorageService.instance.isGossipActive())
    +            return;
    +
    +        if (!registered)
             {
    -            public void run()
    +            if (MessagingService.instance() != null)
                 {
    -                // we do this so that a host considered bad has a chance to recover, otherwise would we never try
    -                // to read from it, which would cause its score to never change
    -                reset();
    +                MessagingService.instance().register(this);
    +                registered = true;
                 }
    -        };
    +        }
     
    -        if (DatabaseDescriptor.isDaemonInitialized())
    +        this.scores = calculateScores();
    +    }
    +
    +    /**
    +     * This is generally expensive and is called periodically (semi-frequently) not on the fast path.
    +     * @return a freshly constructed scores map.
    +     */
    +    public Map<InetAddressAndPort, Double> calculateScores()
    +    {
    +        double maxLatency = 1;
    +
    +        Map<InetAddressAndPort, Double> measurements = new HashMap<>(samples.size());
    +
    +        // We're going to weight the latency for each host against the worst one we see, to
    +        // arrive at sort of a 'badness percentage' for them. First, find the worst for each:
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry : samples.entrySet())
             {
    -            updateSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(update, dynamicUpdateInterval, dynamicUpdateInterval, TimeUnit.MILLISECONDS);
    -            resetSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(reset, dynamicResetInterval, dynamicResetInterval, TimeUnit.MILLISECONDS);
    -            registerMBean();
    +            // This is expensive for e.g. the Histogram, so do it once and cache the result
    +            double measure = entry.getValue().measurement.measure();
    +            if (measure > maxLatency)
    +                maxLatency = measure;
    +            measurements.put(entry.getKey(), measure);
             }
    +
    +        HashMap<InetAddressAndPort, Double> newScores = new HashMap<>(measurements.size());
    +
    +        // now make another pass to do the weighting based on the maximums we found before
    +        for (Map.Entry<InetAddressAndPort, Double> entry : measurements.entrySet())
    +        {
    +            double score = entry.getValue() / maxLatency;
    +            // finally, add the severity without any weighting, since hosts scale this relative to their own load and the size of the task causing the severity.
    +            // "Severity" is basically a measure of compaction activity (CASSANDRA-3722).
    +            if (USE_SEVERITY)
    +                score += getSeverity(entry.getKey());
    +            // lowest score (least amount of badness) wins.
    +            newScores.put(entry.getKey(), score);
    +        }
    +
    +        return newScores;
         }
     
         /**
    -     * Update configuration from {@link DatabaseDescriptor} and estart the update-scheduler and reset-scheduler tasks
    +     * Update configuration from {@link DatabaseDescriptor} and restart the various scheduler tasks
          * if the configured rates for these tasks have changed.
          */
    -    public void applyConfigChanges()
    +    public void applyConfigChanges(int newDynamicUpdateInternal, int newDynamicLatencyProbeInterval, double newDynamicBadnessThreshold)
         {
    -        if (dynamicUpdateInterval != DatabaseDescriptor.getDynamicUpdateInterval())
    +        if (dynamicUpdateInterval != newDynamicUpdateInternal)
             {
    -            dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    +            dynamicUpdateInterval = newDynamicUpdateInternal;
                 if (DatabaseDescriptor.isDaemonInitialized())
                 {
    -                updateSchedular.cancel(false);
    -                updateSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(update, dynamicUpdateInterval, dynamicUpdateInterval, TimeUnit.MILLISECONDS);
    +                if (updateScoresScheduler != null)
    +                    updateScoresScheduler.cancel(false);
    +                updateScoresScheduler = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(this::updateScores, dynamicUpdateInterval, dynamicUpdateInterval, TimeUnit.MILLISECONDS);
                 }
             }
     
    -        if (dynamicResetInterval != DatabaseDescriptor.getDynamicResetInterval())
    +        if (this.dynamicLatencyProbeInterval != newDynamicLatencyProbeInterval)
             {
    -            dynamicResetInterval = DatabaseDescriptor.getDynamicResetInterval();
    +            this.dynamicLatencyProbeInterval = newDynamicLatencyProbeInterval;
                 if (DatabaseDescriptor.isDaemonInitialized())
                 {
    -                resetSchedular.cancel(false);
    -                resetSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(reset, dynamicResetInterval, dynamicResetInterval, TimeUnit.MILLISECONDS);
    +                if (updateSamplesScheduler != null)
    +                    updateSamplesScheduler.cancel(false);
    +
    +                if (newDynamicLatencyProbeInterval > 0)
    +                {
    +                    probeRateLimiter.setRate(dynamicLatencyProbeInterval);
    +                    updateSamplesScheduler = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(this::updateSamples, dynamicLatencyProbeInterval, dynamicLatencyProbeInterval, TimeUnit.MILLISECONDS);
    --- End diff --
    
    I considered that but didn't like it because then we can't block in the acquire of the ratelimiter tokens (as that would block the logical clock increments). I think that the amount of work we're doing here is on the same order as before (iterating the samples map) so we should be ok right?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r229505703
  
    --- Diff: test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java ---
    @@ -60,57 +130,229 @@ private static EndpointsForRange full(InetAddressAndPort... endpoints)
         }
     
         @Test
    -    public void testSnitch() throws InterruptedException, IOException, ConfigurationException
    +    public void testSortedByProximity() throws InterruptedException, IOException, ConfigurationException
         {
    -        // do this because SS needs to be initialized before DES can work properly.
    -        StorageService.instance.unsafeInitialize();
    -        SimpleSnitch ss = new SimpleSnitch();
    -        DynamicEndpointSnitch dsnitch = new DynamicEndpointSnitch(ss, String.valueOf(ss.hashCode()));
    -        InetAddressAndPort self = FBUtilities.getBroadcastAddressAndPort();
    -        InetAddressAndPort host1 = InetAddressAndPort.getByName("127.0.0.2");
    -        InetAddressAndPort host2 = InetAddressAndPort.getByName("127.0.0.3");
    -        InetAddressAndPort host3 = InetAddressAndPort.getByName("127.0.0.4");
    -        InetAddressAndPort host4 = InetAddressAndPort.getByName("127.0.0.5");
    -        List<InetAddressAndPort> hosts = Arrays.asList(host1, host2, host3);
    +        InetAddressAndPort self = hosts[0];
    +        List<InetAddressAndPort> allHosts = Arrays.asList(hosts[1], hosts[2], hosts[3]);
     
             // first, make all hosts equal
    -        setScores(dsnitch, 1, hosts, 10, 10, 10);
    -        EndpointsForRange order = full(host1, host2, host3);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    -
    -        // make host1 a little worse
    -        setScores(dsnitch, 1, hosts, 20, 10, 10);
    -        order = full(host2, host3, host1);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    -
    -        // make host2 as bad as host1
    -        setScores(dsnitch, 2, hosts, 15, 20, 10);
    -        order = full(host3, host1, host2);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    -
    -        // make host3 the worst
    -        setScores(dsnitch, 3, hosts, 10, 10, 30);
    -        order = full(host1, host2, host3);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    -
    -        // make host3 equal to the others
    -        setScores(dsnitch, 5, hosts, 10, 10, 10);
    -        order = full(host1, host2, host3);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    +        setScores(dsnitch, 1, allHosts, 10, 10, 10);
    +        EndpointsForRange order = full(hosts[1], hosts[2], hosts[3]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        // make hosts[1] a little worse
    +        setScores(dsnitch, 2, allHosts, 20, 10, 10);
    +        order = full(hosts[2], hosts[3], hosts[1]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        // make hosts[2] as bad as hosts[1]
    +        setScores(dsnitch, 4, allHosts, 15, 20, 10);
    +        order = full(hosts[3], hosts[1], hosts[2]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        // make hosts[3] the worst
    +        setScores(dsnitch, 10, allHosts, 10, 10, 30);
    +        order = full(hosts[1], hosts[2], hosts[3]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        // make hosts[3] equal to the others
    +        setScores(dsnitch, 15, allHosts, 10, 10, 10);
    +        order = full(hosts[1], hosts[2], hosts[3]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
     
             /// Tests CASSANDRA-6683 improvements
             // make the scores differ enough from the ideal order that we sort by score; under the old
             // dynamic snitch behavior (where we only compared neighbors), these wouldn't get sorted
    -        setScores(dsnitch, 20, hosts, 10, 70, 20);
    -        order = full(host1, host3, host2);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    +        dsnitch.reset();
    +        setScores(dsnitch, 20, allHosts, 10, 70, 20);
    +        order = full(hosts[1], hosts[3], hosts[2]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        order = full(hosts[4], hosts[1], hosts[3], hosts[2]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3], hosts[4])));
    +
    +        setScores(dsnitch, 40, allHosts, 10, 10, 11);
    +        order = full(hosts[4], hosts[1], hosts[2], hosts[3]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3], hosts[4])));
    +    }
    +
    +    // CASSANDRA-14459 improvements to add latency probes instead of resets
    +    @Test
    +    public void testLatencyProbeNeeded()
    +    {
    +        InetAddressAndPort self = hosts[0];
    +
    +        // Four hosts, two are very latent, but all are requested for ranking
    +        dsnitch.receiveTiming(hosts[1], 20, LatencyMeasurementType.READ);
    +        dsnitch.receiveTiming(hosts[2], 10, LatencyMeasurementType.READ);
    +        dsnitch.receiveTiming(hosts[3], 1000, LatencyMeasurementType.READ);
    +        dsnitch.receiveTiming(hosts[4], 1000, LatencyMeasurementType.READ);
    +        dsnitch.updateScores();
    +
    +        EndpointsForRange orderBefore = full(hosts[2], hosts[1], hosts[3], hosts[4]);
    +
    +        Map<InetAddressAndPort, DynamicEndpointSnitch.AnnotatedMeasurement> measurements = dsnitch.getMeasurementsWithPort();
    +        // At this point we haven't ranked any of hosts so we should need no probes even
    --- End diff --
    
    even?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225337008
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitchEMA.java ---
    @@ -0,0 +1,181 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.cassandra.locator;
    +
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.concurrent.ConcurrentHashMap;
    +
    +import org.apache.cassandra.gms.Gossiper;
    +import org.apache.cassandra.metrics.ExponentialMovingAverage;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +
    +
    +/**
    + * A dynamic snitching implementation that uses Exponentially Decaying Histograms to prefer or
    + * de-prefer hosts
    + *
    + * This was the default implementation prior to Cassandra 4.0 and is being left as the default
    + * in 4.0
    + */
    +public class DynamicEndpointSnitchEMA extends DynamicEndpointSnitch
    +{
    +    // A ~10 sample EMA
    +    private static final double EMA_ALPHA = 0.10;
    +
    +    private final ConcurrentHashMap<InetAddressAndPort, AnnotatedEMA> samples = new ConcurrentHashMap<>();
    +
    +    /**
    +     * Adds two boolean markers to the ExponentialMovingAverage for telling if the data has been
    +     * updated or requested recently.
    +     *
    +     * recentlyMeasured is updated through {@link AnnotatedEMA#update(long, boolean)}
    +     * recentlyRequested is updated through {@link DynamicEndpointSnitch#markRequested}
    +     *
    +     * Both markers are periodically reset via {@link DynamicEndpointSnitch#latencyProbeNeeded(long)}
    +     */
    +    private static class AnnotatedEMA extends ExponentialMovingAverage
    +    {
    +        volatile boolean recentlyRequested = false;
    +        volatile boolean recentlyMeasured = false;
    +
    +        AnnotatedEMA(double alpha, double initialValue)
    +        {
    +            super(alpha, initialValue);
    +        }
    +
    +        public void update(long value, boolean isRealRead)
    +        {
    +            recentlyMeasured = recentlyMeasured || isRealRead;
    +            super.update(value);
    +        }
    +    }
    +
    +    // Called via reflection
    +    public DynamicEndpointSnitchEMA(IEndpointSnitch snitch)
    +    {
    +        this(snitch, "ema");
    +    }
    +
    +    protected DynamicEndpointSnitchEMA(IEndpointSnitch snitch, String instance)
    +    {
    +        super(snitch, instance);
    +    }
    +
    +    private void receiveTiming(InetAddressAndPort host, long latency, boolean isRealRead) // this is cheap
    +    {
    +        AnnotatedEMA sample = samples.get(host);
    +        if (sample == null)
    +        {
    +            AnnotatedEMA maybeNewSample = new AnnotatedEMA(EMA_ALPHA, latency);
    +            sample = samples.putIfAbsent(host, maybeNewSample);
    +            if (sample == null)
    +                sample = maybeNewSample;
    +        }
    +        sample.update(latency, isRealRead);
    +    }
    +
    +    public void receiveTiming(InetAddressAndPort address, long latency, LatencyMeasurementType measurementType)
    +    {
    +        if (measurementType == LatencyMeasurementType.READ)
    +            receiveTiming(address, latency, true);
    +        else if (measurementType == LatencyMeasurementType.PROBE)
    +            receiveTiming(address, latency, false);
    +    }
    +
    +    @Override
    +    protected void reset() {
    +        this.samples.clear();
    +        super.reset();
    +    }
    +
    +    @Override
    +    public Map<InetAddressAndPort, Double> calculateScores()
    +    {
    +        // We're going to weight the latency for each host against the worst one we see, to
    +        // arrive at sort of a 'badness percentage' for them. First, find the worst for each:
    +        HashMap<InetAddressAndPort, Double> newScores = new HashMap<>(samples.size());
    +        Optional<Double> maxObservedAvgLatency = samples.values().stream()
    +                                                        .map(ExponentialMovingAverage::getAvg)
    +                                                        .max(Double::compare);
    +
    +        final double maxAvgLatency = maxObservedAvgLatency.isPresent() ? maxObservedAvgLatency.get() : 1;
    +
    +        // now make another pass to do the weighting based on the maximums we found before
    +        for (Map.Entry<InetAddressAndPort, AnnotatedEMA> entry : samples.entrySet())
    +        {
    +            // Samples may have changed but rather than creating garbage by copying samples we just ensure
    +            // that all scores are less than 1.0
    +            double addrAvg = entry.getValue().getAvg();
    +            double score = addrAvg / Math.max(addrAvg, maxAvgLatency);
    +            // finally, add the severity without any weighting, since hosts scale this relative to their own load
    +            // and the size of the task causing the severity.
    +            // "Severity" is basically a measure of compaction activity (CASSANDRA-3722).
    +            if (USE_SEVERITY)
    +                score += getSeverity(entry.getKey());
    +            // lowest score (least amount of badness) wins.
    +            newScores.put(entry.getKey(), score);
    +        }
    +        return newScores;
    +    }
    +
    +    @Override
    +    public List<Double> dumpTimings(String hostname) throws UnknownHostException
    +    {
    +        InetAddressAndPort host = InetAddressAndPort.getByName(hostname);
    +        ArrayList<Double> timings = new ArrayList<>();
    +        ExponentialMovingAverage avg = samples.get(host);
    +        if (avg != null)
    +        {
    +            timings.add(avg.getAvg());
    +        }
    +        return timings;
    +    }
    +
    +    @Override
    +    protected void updateLatencyProbeSequence(List<InetAddressAndPort> probeSequence)
    +    {
    +        samples.keySet().retainAll(Gossiper.instance.getLiveMembers());
    +
    +        for (Map.Entry<InetAddressAndPort, AnnotatedEMA> entry: samples.entrySet())
    +        {
    +            // We only send latency probes to nodes that we may plausibly talk to (requested is true)
    +            // but we have not talked to since the last reset of this information (recentlyMeasured is false)
    +            if (entry.getValue().recentlyRequested && !entry.getValue().recentlyMeasured)
    +            {
    +                probeSequence.add(entry.getKey());
    +            }
    +
    +            entry.getValue().recentlyMeasured = false;
    +            entry.getValue().recentlyRequested = false;
    +        }
    +    }
    +
    +    @Override
    +    protected void markRequested(InetAddressAndPort address)
    +    {
    +        AnnotatedEMA ema = samples.get(address);
    +        if (ema != null)
    +            ema.recentlyRequested = true;
    --- End diff --
    
    When we set this boolean we also want to check if it's already set before setting it.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225337690
  
    --- Diff: src/java/org/apache/cassandra/net/MessagingService.java ---
    @@ -665,9 +665,9 @@ public void updateBackPressureOnSend(InetAddressAndPort host, IAsyncCallback cal
         /**
          * Updates the back-pressure state on reception from the given host if enabled and the given message callback supports it.
          *
    -     * @param host The replica host the back-pressure state refers to.
    +     * @param host     The replica host the back-pressure state refers to.
    --- End diff --
    
    Your IDE is set up to fix whitespace on unchanged lines. That's generally not great just because it introduces a lot of extra changes, and if two people have IDEs with different preferences they end up fighting and introducing random conflicts all the time.
    
    Can you go through this change set and remove whitespace changes for things that haven't actually been touched?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r227617249
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitchEMA.java ---
    @@ -0,0 +1,181 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.cassandra.locator;
    +
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.concurrent.ConcurrentHashMap;
    +
    +import org.apache.cassandra.gms.Gossiper;
    +import org.apache.cassandra.metrics.ExponentialMovingAverage;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +
    +
    +/**
    + * A dynamic snitching implementation that uses Exponentially Decaying Histograms to prefer or
    + * de-prefer hosts
    + *
    + * This was the default implementation prior to Cassandra 4.0 and is being left as the default
    + * in 4.0
    + */
    +public class DynamicEndpointSnitchEMA extends DynamicEndpointSnitch
    +{
    +    // A ~10 sample EMA
    +    private static final double EMA_ALPHA = 0.10;
    +
    +    private final ConcurrentHashMap<InetAddressAndPort, AnnotatedEMA> samples = new ConcurrentHashMap<>();
    +
    +    /**
    +     * Adds two boolean markers to the ExponentialMovingAverage for telling if the data has been
    +     * updated or requested recently.
    +     *
    +     * recentlyMeasured is updated through {@link AnnotatedEMA#update(long, boolean)}
    +     * recentlyRequested is updated through {@link DynamicEndpointSnitch#markRequested}
    +     *
    +     * Both markers are periodically reset via {@link DynamicEndpointSnitch#latencyProbeNeeded(long)}
    +     */
    +    private static class AnnotatedEMA extends ExponentialMovingAverage
    --- End diff --
    
    This should be done. Almost everything is in the super classes (only method is one that provides an instance of `ISnitchMeasurement` for the superclass).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225324472
  
    --- Diff: src/java/org/apache/cassandra/gms/Gossiper.java ---
    @@ -1055,7 +1056,6 @@ public boolean isLatencyForSnitch()
                 {
                     return false;
                 }
    -
    --- End diff --
    
    This whitespace change doesn't make sense?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225335637
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitchEMA.java ---
    @@ -0,0 +1,181 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.cassandra.locator;
    +
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.concurrent.ConcurrentHashMap;
    +
    +import org.apache.cassandra.gms.Gossiper;
    +import org.apache.cassandra.metrics.ExponentialMovingAverage;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +
    +
    +/**
    + * A dynamic snitching implementation that uses Exponentially Decaying Histograms to prefer or
    + * de-prefer hosts
    + *
    + * This was the default implementation prior to Cassandra 4.0 and is being left as the default
    + * in 4.0
    + */
    +public class DynamicEndpointSnitchEMA extends DynamicEndpointSnitch
    +{
    +    // A ~10 sample EMA
    +    private static final double EMA_ALPHA = 0.10;
    +
    +    private final ConcurrentHashMap<InetAddressAndPort, AnnotatedEMA> samples = new ConcurrentHashMap<>();
    +
    +    /**
    +     * Adds two boolean markers to the ExponentialMovingAverage for telling if the data has been
    +     * updated or requested recently.
    +     *
    +     * recentlyMeasured is updated through {@link AnnotatedEMA#update(long, boolean)}
    +     * recentlyRequested is updated through {@link DynamicEndpointSnitch#markRequested}
    +     *
    +     * Both markers are periodically reset via {@link DynamicEndpointSnitch#latencyProbeNeeded(long)}
    +     */
    +    private static class AnnotatedEMA extends ExponentialMovingAverage
    --- End diff --
    
    You can use a common base with and compose instead of extending so that the base class can contain all the type specific functionality. I think you want to wrap anyways so we can adapt the methods from the underlying reservoir.
    
    Then the map can be in the base class since it's basically common anyways.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r226138379
  
    --- Diff: conf/cassandra.yaml ---
    @@ -947,10 +947,10 @@ endpoint_snitch: SimpleSnitch
     
     # controls how often to perform the more expensive part of host score
     # calculation
    -dynamic_snitch_update_interval_in_ms: 100 
    -# controls how often to reset all host scores, allowing a bad host to
    -# possibly recover
    -dynamic_snitch_reset_interval_in_ms: 600000
    +dynamic_snitch_update_interval_in_ms: 100
    +# controls how often this node is permitted to send a latency probe to
    +# determine if previously slow nodes have recovered. Set to -1 to disable
    +dynamic_snitch_latency_probe_interval_in_ms: 1000
    --- End diff --
    
    I'm resolving this since I think we agreed not to do this.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225697042
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -22,121 +22,194 @@
     import java.net.InetAddress;
     import java.net.UnknownHostException;
     import java.util.*;
    -import java.util.concurrent.ConcurrentHashMap;
     import java.util.concurrent.ScheduledFuture;
     import java.util.concurrent.TimeUnit;
     import java.util.stream.Collectors;
     
    -import com.codahale.metrics.ExponentiallyDecayingReservoir;
     import javax.management.MBeanServer;
     import javax.management.ObjectName;
     
    -import com.codahale.metrics.Snapshot;
    +import com.google.common.annotations.VisibleForTesting;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
     import org.apache.cassandra.concurrent.ScheduledExecutors;
     import org.apache.cassandra.config.DatabaseDescriptor;
     import org.apache.cassandra.gms.ApplicationState;
     import org.apache.cassandra.gms.EndpointState;
     import org.apache.cassandra.gms.Gossiper;
     import org.apache.cassandra.gms.VersionedValue;
    +import org.apache.cassandra.net.IAsyncCallback;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +import org.apache.cassandra.net.MessageIn;
    +import org.apache.cassandra.net.MessageOut;
     import org.apache.cassandra.net.MessagingService;
    +import org.apache.cassandra.net.PingMessage;
     import org.apache.cassandra.service.StorageService;
     import org.apache.cassandra.utils.FBUtilities;
     
    +import static org.apache.cassandra.net.MessagingService.Verb.PING;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.LARGE_MESSAGE;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.SMALL_MESSAGE;
    +
    +
     /**
      * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector
    + * Note that the subclasses (e.g. {@link DynamicEndpointSnitchHistogram}) are responsible for actually measuring
    + * latency and populating the {@link #scores} map.
      */
    -public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
    +public abstract class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
     {
    -    private static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    -
    -    private static final double ALPHA = 0.75; // set to 0.75 to make EDS more biased to towards the newer values
    -    private static final int WINDOW_SIZE = 100;
    -
    -    private volatile int dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    -    private volatile int dynamicResetInterval = DatabaseDescriptor.getDynamicResetInterval();
    -    private volatile double dynamicBadnessThreshold = DatabaseDescriptor.getDynamicBadnessThreshold();
    +    private static final Logger logger = LoggerFactory.getLogger(DynamicEndpointSnitch.class);
    +
    +    // Subclass specific functionality
    +    protected static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    protected boolean registered = false;
    +    // The scores map is updated via copy in updateScores
    +    // We keep it in the base class for performance reasons (so it can be easily aliased)
    +    protected volatile Map<InetAddressAndPort, Double> scores = new HashMap<>();
    +
    +    // Rate limit how often we generate latency probes
    +    protected long nextAllowedProbeGenerationTime;
    +    protected long nextProbeGenerationTime;
    +    protected int currentProbePosition = 0;
    +    protected final List<InetAddressAndPort> latencyProbeSequence = new ArrayList<>();
    +
    +    // DES general functionality
    +    protected volatile int dynamicUpdateInterval = -1;
    +    protected volatile int dynamicLatencyProbeInterval = -1;
    +    protected volatile double dynamicBadnessThreshold = 0;
     
         // the score for a merged set of endpoints must be this much worse than the score for separate endpoints to
         // warrant not merging two ranges into a single range
         private static final double RANGE_MERGING_PREFERENCE = 1.5;
     
         private String mbeanName;
    -    private boolean registered = false;
    -
    -    private volatile HashMap<InetAddressAndPort, Double> scores = new HashMap<>();
    -    private final ConcurrentHashMap<InetAddressAndPort, ExponentiallyDecayingReservoir> samples = new ConcurrentHashMap<>();
    +    private boolean mbeanRegistered = false;
     
         public final IEndpointSnitch subsnitch;
     
    -    private volatile ScheduledFuture<?> updateSchedular;
    -    private volatile ScheduledFuture<?> resetSchedular;
    +    private volatile ScheduledFuture<?> updateScheduler;
    +    private volatile ScheduledFuture<?> latencyProbeScheduler;
     
    -    private final Runnable update;
    -    private final Runnable reset;
    +    private final Runnable update = this::updateScores;
    +    private final Runnable latencyProbe = this::maybeSendLatencyProbe;
     
         public DynamicEndpointSnitch(IEndpointSnitch snitch)
         {
             this(snitch, null);
         }
     
    -    public DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
    +    protected DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
         {
             mbeanName = "org.apache.cassandra.db:type=DynamicEndpointSnitch";
             if (instance != null)
                 mbeanName += ",instance=" + instance;
             subsnitch = snitch;
    -        update = new Runnable()
    -        {
    -            public void run()
    -            {
    -                updateScores();
    -            }
    -        };
    -        reset = new Runnable()
    -        {
    -            public void run()
    -            {
    -                // we do this so that a host considered bad has a chance to recover, otherwise would we never try
    -                // to read from it, which would cause its score to never change
    -                reset();
    -            }
    -        };
     
             if (DatabaseDescriptor.isDaemonInitialized())
             {
    -            updateSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(update, dynamicUpdateInterval, dynamicUpdateInterval, TimeUnit.MILLISECONDS);
    -            resetSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(reset, dynamicResetInterval, dynamicResetInterval, TimeUnit.MILLISECONDS);
    +            applyConfigChanges();
                 registerMBean();
             }
    +        nextProbeGenerationTime = System.nanoTime();
    +    }
    +
    +    /**
    +     * Records a latency. This MUST be cheap as it is called in the fast path
    +     */
    +    abstract public void receiveTiming(InetAddressAndPort address, long latency, LatencyMeasurementType measurementType);
    +
    +    /**
    +     * Called periodically from {@link DynamicEndpointSnitch#updateScores()}. This method returns a new score map
    +     * that will be assigned to the {@link DynamicEndpointSnitch#scores} map.
    +     * This is generally more expensive than {@link DynamicEndpointSnitch#receiveTiming}
    +     */
    +    abstract public Map<InetAddressAndPort, Double> calculateScores();
    --- End diff --
    
    So I think future implementations just override the superclass method if they want?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225698446
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -188,6 +343,9 @@ public String getDatacenter(InetAddressAndPort endpoint)
         public <C extends ReplicaCollection<? extends C>> C sortedByProximity(final InetAddressAndPort address, C unsortedAddresses)
         {
             assert address.equals(FBUtilities.getBroadcastAddressAndPort()); // we only know about ourself
    +        for (Replica unsortedAddress : unsortedAddresses)
    --- End diff --
    
    Hmm... yeah... seems like the score maybe be merged with the other map containing the reservoir and the score is just an attribute of the annotated thing.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r226739145
  
    --- Diff: src/java/org/apache/cassandra/net/IAsyncCallback.java ---
    @@ -31,11 +31,23 @@
         void response(MessageIn<T> msg);
     
         /**
    +     * @deprecated Use {@link #latencyMeasurementType()}. This is left for backwards
    +     *             compatibility with existing IAsyncCallback implementations but will
    +     *             be removed at a future date.
          * @return true if this callback is on the read path and its latency should be
          * given as input to the dynamic snitch.
          */
    +    @Deprecated
    --- End diff --
    
    Sure I'll squash before merging to trunk


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225362450
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -188,6 +343,9 @@ public String getDatacenter(InetAddressAndPort endpoint)
         public <C extends ReplicaCollection<? extends C>> C sortedByProximity(final InetAddressAndPort address, C unsortedAddresses)
         {
             assert address.equals(FBUtilities.getBroadcastAddressAndPort()); // we only know about ourself
    +        for (Replica unsortedAddress : unsortedAddresses)
    --- End diff --
    
    So I can easily eliminate the extra iteration when calling into `sortedByProximityWithBadness`, but `sortedByProximityWithScore` only calls `compareEndpoints` and putting two more dict lookups there seemed likely slower (to me) than just iterating the unsortedAddresses in `sortedByProximity`. I can make this method more complicated to special case `sortedByProximity -> sortedByProximityWithScore` (as opposed to the normal path of `sortedByProximity` -> sortedByProximityWithBadness -> sortedByProximityWithScore`) ... sure 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r226759144
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -154,20 +310,155 @@ private void registerMBean()
     
         public void close()
         {
    -        updateSchedular.cancel(false);
    -        resetSchedular.cancel(false);
    +        if (updateScheduler != null)
    +            updateScheduler.cancel(false);
    +        if (latencyProbeScheduler != null)
    +            latencyProbeScheduler.cancel(false);
     
             MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
             try
             {
    -            mbs.unregisterMBean(new ObjectName(mbeanName));
    +            if (mbeanRegistered)
    +                mbs.unregisterMBean(new ObjectName(mbeanName));
             }
             catch (Exception e)
             {
                 throw new RuntimeException(e);
             }
         }
     
    +    /**
    +     * Determines if latency probes need to be sent, and potentially sends a single latency probe per invocation
    +     */
    +    protected void maybeSendLatencyProbe()
    +    {
    +        if (!StorageService.instance.isGossipActive())
    +            return;
    +
    +        currentProbePosition = latencyProbeNeeded(samples, latencyProbeSequence, currentProbePosition);
    +
    +        if (currentProbePosition < latencyProbeSequence.size())
    +        {
    +            try
    +            {
    +                InetAddressAndPort peer = latencyProbeSequence.get(currentProbePosition);
    +                sendPingMessageToPeer(peer);
    +            }
    +            catch (IndexOutOfBoundsException ignored) {}
    +        }
    +    }
    +
    +    /**
    +     * This method (unfortunately) mutates a lot of state so that it doesn't create any garbage and only iterates the
    +     * sample map a single time . In particular on every call we:
    +     *  - increment every sample's intervalsSinceLastMeasure
    +     *
    +     * When probes should be generated we also potentially:
    +     *  - reset sample's recentlyRequested that have reached the "CONSTANT" phase of probing (10 minutes by default)
    +     *  - add any InetAddressAndPort's that need probing to the provided endpointsToProbe
    +     *  - shuffle the endpointsToProbe
    +     *
    +     * If there are probes to be sent, this method short circuits all generation of probes and just returns the
    +     * passed probePosition plus one.
    +     * @return The position of the passed endpointsToProbe that should be probed.
    +     */
    +    @VisibleForTesting
    +    int latencyProbeNeeded(Map<InetAddressAndPort, AnnotatedMeasurement> samples,
    +                                     List<InetAddressAndPort> endpointsToProbe, int probePosition) {
    +        boolean shouldGenerateProbes = (probePosition >= endpointsToProbe.size());
    +
    +        if (shouldGenerateProbes)
    +        {
    +            endpointsToProbe.clear();
    +            samples.keySet().retainAll(Gossiper.instance.getLiveMembers());
    +        }
    +
    +        // We have to increment intervalsSinceLastMeasure regardless of if we generate probes
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry: samples.entrySet())
    +        {
    +            AnnotatedMeasurement measurement = entry.getValue();
    +            long intervalsSinceLastMeasure = measurement.intervalsSinceLastMeasure.getAndIncrement();
    --- End diff --
    
    I am so confused now. This doesn't work because it only sends one probe every 10 seconds. To probe every node in a 1k node non-token aware cluster it would take 1000 * 10 / 60 = 166 minutes. Do we need to do the every ten seconds thing if we are also implementing exponential backoff? Once it's backed off it's 1.6 probes/second which seems totally reasonable.
    
    Also I don't really understand why we are doing this in terms of measuring intervals when what we are trying to track here is time. Every time we send a probe we can double the time to the next probe, and calculate the time in the future when the next probe should occur. 
    
    This seems a more complicated than it has to be. A single threaded scheduled threadpool executor with a RateLimiter seems like it could do this much more clearly.
    
    Receive timing could check for a ScheduleFuture, if it exists cancel it and null it out. The scheduled future can reschedule itself if no real timings have been received.
    
    The existing STPE can check at the minimum interval if any nodes need to start being probed and create the scheduled future. It can also do the cleanup for live nodes.
    
    Determining constant vs exponential can be done by applying Math.min to the result of doubling the backoff vs the limit.
    
    Another plus is because everything is probed independently you don't need to shuffle anymore. We can set the rate limit to place a ceiling on how much intra-cluster probing can occur if there is a thundering herd. Like maybe 5 per second?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r237971740
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -154,31 +326,203 @@ private void registerMBean()
     
         public void close()
         {
    -        updateSchedular.cancel(false);
    -        resetSchedular.cancel(false);
    +        if (updateScoresScheduler != null)
    +            updateScoresScheduler.cancel(false);
    +        if (updateSamplesScheduler != null)
    +            updateSamplesScheduler.cancel(false);
    +
    +        for (AnnotatedMeasurement measurement : samples.values())
    +        {
    +            if (measurement.probeFuture != null)
    +                measurement.probeFuture.cancel(false);
    +
    +            measurement.millisSinceLastMeasure.set(0);
    +            measurement.millisSinceLastRequest.set(MAX_PROBE_INTERVAL_MS);
    +            measurement.probeTimerMillis = 0;
    +        }
     
             MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
             try
             {
    -            mbs.unregisterMBean(new ObjectName(mbeanName));
    +            if (mbeanRegistered)
    +                mbs.unregisterMBean(new ObjectName(mbeanName));
             }
             catch (Exception e)
             {
                 throw new RuntimeException(e);
             }
         }
     
    +    /**
    +     * Background task running on the samples dictionary. The default implementation sends latency probes (PING)
    +     * messages to explore nodes that we have not received timings for recently but have ranked in
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}.
    +     */
    +    protected void updateSamples()
    +    {
    +        // Split calculation of probe timers from sending probes for testability
    +        calculateProbes(samples, dynamicLatencyProbeInterval);
    +
    +        if (!StorageService.instance.isGossipActive())
    +            return;
    +
    +        schedulePings(samples);
    +    }
    +
    +    /**
    +     * This method mutates the passed AnnotatedMeasurements to implement capped exponential backoff per endpoint.
    +     *
    +     * The algorithm is as follows:
    +     * 1. All samples get their millisSinceLastMeasure and millisSinceLastRequest fields
    +     *    incremented by the passed interval
    +     * 2. Any recently requested (ranked) endpoints that have not been measured recently (e.g. because the snitch
    +     *    has sent them no traffic) get probes with exponential backoff.
    +     *
    +     * The backoff is capped at MAX_PROBE_INTERVAL_MS. Furthermore the probes are stopped after
    +     * MAX_PROBE_INTERVAL_MS of no ranking requests as well.
    +     *
    +     * At the end of this method, any passed AnnotatedMeasurements that need latency probes will have non zero
    +     * probeTimerMillis members set.
    +     */
    +    @VisibleForTesting
    +    static void calculateProbes(Map<InetAddressAndPort, AnnotatedMeasurement> samples, long intervalMillis) {
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry: samples.entrySet())
    +        {
    +            if (entry.getKey().equals(FBUtilities.getBroadcastAddressAndPort()))
    +                continue;
    +
    +            AnnotatedMeasurement measurement = entry.getValue();
    +            long lastMeasure = measurement.millisSinceLastMeasure.getAndAdd(intervalMillis);
    --- End diff --
    
    The messaging rate is so low let's punt on the metric for now? I'm happy to add it as a follow up patch but I'd like to get the pluggability and probes in first.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225698939
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitchEMA.java ---
    @@ -0,0 +1,181 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.cassandra.locator;
    +
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.concurrent.ConcurrentHashMap;
    +
    +import org.apache.cassandra.gms.Gossiper;
    +import org.apache.cassandra.metrics.ExponentialMovingAverage;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +
    +
    +/**
    + * A dynamic snitching implementation that uses Exponentially Decaying Histograms to prefer or
    + * de-prefer hosts
    + *
    + * This was the default implementation prior to Cassandra 4.0 and is being left as the default
    + * in 4.0
    + */
    +public class DynamicEndpointSnitchEMA extends DynamicEndpointSnitch
    +{
    +    // A ~10 sample EMA
    +    private static final double EMA_ALPHA = 0.10;
    +
    +    private final ConcurrentHashMap<InetAddressAndPort, AnnotatedEMA> samples = new ConcurrentHashMap<>();
    +
    +    /**
    +     * Adds two boolean markers to the ExponentialMovingAverage for telling if the data has been
    +     * updated or requested recently.
    +     *
    +     * recentlyMeasured is updated through {@link AnnotatedEMA#update(long, boolean)}
    +     * recentlyRequested is updated through {@link DynamicEndpointSnitch#markRequested}
    +     *
    +     * Both markers are periodically reset via {@link DynamicEndpointSnitch#latencyProbeNeeded(long)}
    +     */
    +    private static class AnnotatedEMA extends ExponentialMovingAverage
    +    {
    +        volatile boolean recentlyRequested = false;
    +        volatile boolean recentlyMeasured = false;
    +
    +        AnnotatedEMA(double alpha, double initialValue)
    +        {
    +            super(alpha, initialValue);
    +        }
    +
    +        public void update(long value, boolean isRealRead)
    +        {
    +            recentlyMeasured = recentlyMeasured || isRealRead;
    --- End diff --
    
    You are right I did say boolean. The enum thing occurred to me later as it's usually a response to seeing a boolean in a method call and being "what does that do?" It's a new habit I am trying to build.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r229456581
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -23,128 +23,300 @@
     import java.net.UnknownHostException;
     import java.util.*;
     import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ExecutionException;
     import java.util.concurrent.ScheduledFuture;
     import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
     import java.util.stream.Collectors;
     
    -import com.codahale.metrics.ExponentiallyDecayingReservoir;
     import javax.management.MBeanServer;
     import javax.management.ObjectName;
     
    -import com.codahale.metrics.Snapshot;
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.util.concurrent.Futures;
    +import com.google.common.util.concurrent.MoreExecutors;
    +import com.google.common.util.concurrent.RateLimiter;
    +import com.google.common.util.concurrent.SettableFuture;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
     import org.apache.cassandra.concurrent.ScheduledExecutors;
     import org.apache.cassandra.config.DatabaseDescriptor;
    +import org.apache.cassandra.exceptions.RequestFailureReason;
     import org.apache.cassandra.gms.ApplicationState;
     import org.apache.cassandra.gms.EndpointState;
     import org.apache.cassandra.gms.Gossiper;
     import org.apache.cassandra.gms.VersionedValue;
    +import org.apache.cassandra.locator.dynamicsnitch.DynamicEndpointSnitchHistogram;
    +import org.apache.cassandra.net.IAsyncCallbackWithFailure;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +import org.apache.cassandra.net.MessageIn;
    +import org.apache.cassandra.net.MessageOut;
     import org.apache.cassandra.net.MessagingService;
    +import org.apache.cassandra.net.PingMessage;
     import org.apache.cassandra.service.StorageService;
     import org.apache.cassandra.utils.FBUtilities;
     
    +import static org.apache.cassandra.net.MessagingService.Verb.PING;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.LARGE_MESSAGE;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.SMALL_MESSAGE;
    +
    +
     /**
      * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector
    + * Note that the subclasses (e.g. {@link DynamicEndpointSnitchHistogram}) are responsible for actually measuring
    + * latency and providing an ISnitchMeasurement implementation back to this class.
      */
    -public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
    +public abstract class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
     {
    -    private static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    -
    -    private static final double ALPHA = 0.75; // set to 0.75 to make EDS more biased to towards the newer values
    -    private static final int WINDOW_SIZE = 100;
    -
    -    private volatile int dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    -    private volatile int dynamicResetInterval = DatabaseDescriptor.getDynamicResetInterval();
    -    private volatile double dynamicBadnessThreshold = DatabaseDescriptor.getDynamicBadnessThreshold();
    +    private static final Logger logger = LoggerFactory.getLogger(DynamicEndpointSnitch.class);
    +
    +    // Latency measurement and ranking. The samples contain latency measurements and the scores are used for ranking
    +    protected boolean registered = false;
    +    protected static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    protected volatile Map<InetAddressAndPort, Double> scores = new HashMap<>();
    +    protected final Map<InetAddressAndPort, AnnotatedMeasurement> samples = new ConcurrentHashMap<>();
    +
    +    // Latency probe functionality for actively probing endpoints that we haven't measured recently but are ranking
    +    public static final long MAX_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_max_probe_interval_ms", 60 * 10 * 1000L);
    +    public static final long MIN_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_min_probe_interval_ms", 60 * 1000L) ;
    +    // The probe rate is set later when configuration is read
    +    protected static final RateLimiter probeRateLimiter = RateLimiter.create(1);
    +    protected static final DebuggableScheduledThreadPoolExecutor latencyProbeExecutor = new DebuggableScheduledThreadPoolExecutor(1, "LatencyProbes", Thread.MIN_PRIORITY);
    +
    +    // User configuration of the snitch tunables
    +    protected volatile int dynamicUpdateInterval = -1;
    +    protected volatile int dynamicLatencyProbeInterval = -1;
    +    protected volatile double dynamicBadnessThreshold = 0;
     
         // the score for a merged set of endpoints must be this much worse than the score for separate endpoints to
         // warrant not merging two ranges into a single range
         private static final double RANGE_MERGING_PREFERENCE = 1.5;
     
         private String mbeanName;
    -    private boolean registered = false;
    -
    -    private volatile HashMap<InetAddressAndPort, Double> scores = new HashMap<>();
    -    private final ConcurrentHashMap<InetAddressAndPort, ExponentiallyDecayingReservoir> samples = new ConcurrentHashMap<>();
    +    private boolean mbeanRegistered = false;
     
         public final IEndpointSnitch subsnitch;
     
    -    private volatile ScheduledFuture<?> updateSchedular;
    -    private volatile ScheduledFuture<?> resetSchedular;
    -
    -    private final Runnable update;
    -    private final Runnable reset;
    +    private volatile ScheduledFuture<?> updateScoresScheduler;
    +    private volatile ScheduledFuture<?> updateSamplesScheduler;
     
         public DynamicEndpointSnitch(IEndpointSnitch snitch)
         {
             this(snitch, null);
         }
     
    -    public DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
    +    protected DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
         {
             mbeanName = "org.apache.cassandra.db:type=DynamicEndpointSnitch";
             if (instance != null)
                 mbeanName += ",instance=" + instance;
             subsnitch = snitch;
    -        update = new Runnable()
    +
    +        if (DatabaseDescriptor.isDaemonInitialized())
             {
    -            public void run()
    -            {
    -                updateScores();
    -            }
    -        };
    -        reset = new Runnable()
    +            applyConfigChanges(DatabaseDescriptor.getDynamicUpdateInterval(),
    +                               DatabaseDescriptor.getDynamicSampleUpdateInterval(),
    +                               DatabaseDescriptor.getDynamicBadnessThreshold());
    +            open();
    +        }
    +    }
    +
    +    /**
    +     * Allows subclasses to inject new ways of measuring latency back to this abstract base class.
    +     */
    +    protected interface ISnitchMeasurement
    +    {
    +        void sample(long value);
    +        double measure();
    +        Iterable<Double> measurements();
    +    }
    +
    +    /**
    +     * Adds some boookeeping that the DES uses over top of the various metrics techniques used by the
    +     * implementations. This is used to allow CASSANDRA-14459 latency probes as well as further experimentation
    +     * on new latency measurement techniques in CASSANDRA-14817
    +     *
    +     * {@link AnnotatedMeasurement#millisSinceLastRequest} is set to zero through
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}. It defaults to
    +     * the maximum interval so that we only start probing once it has been ranked at least once
    +     * {@link AnnotatedMeasurement#millisSinceLastMeasure} is set to zero from
    +     * {@link DynamicEndpointSnitch#receiveTiming(InetAddressAndPort, long, LatencyMeasurementType)}.
    +     *
    +     * {@link AnnotatedMeasurement#millisSinceLastMeasure and {@link AnnotatedMeasurement#probeTimerMillis}
    +     * are incremented via {@link DynamicEndpointSnitch#updateSamples()}
    +     */
    +    protected static class AnnotatedMeasurement
    +    {
    +        // Used to optimally target latency probes only on nodes that are both requested for ranking
    +        // and are not being measured. For example with token aware clients a large portion of the cluster will never
    +        // be ranked at all and therefore we won't probe them.
    +        public AtomicLong millisSinceLastRequest = new AtomicLong(MAX_PROBE_INTERVAL_MS);
    +        public AtomicLong millisSinceLastMeasure = new AtomicLong(0);
    +        public long probeTimerMillis = 0;
    +        public ScheduledFuture<?> probeFuture = null;
    +
    +        // The underlying measurement technique. E.g. a median filter (histogram) or an EMA filter, or ...
    +        public final ISnitchMeasurement measurement;
    +
    +        public AnnotatedMeasurement(ISnitchMeasurement measurement)
    +        {
    +            this.measurement = measurement;
    +        }
    +
    +        @Override
    +        public String toString()
    +        {
    +            return "AnnotatedMeasurement{" +
    +                   "millisSinceLastRequest=" + millisSinceLastRequest +
    +                   ", millisSinceLastMeasure=" + millisSinceLastMeasure +
    +                   ", probeTimerMillis=" + probeTimerMillis +
    +                   ", probeFuturePending=" + (probeFuture != null && !probeFuture.isDone()) +
    +                   ", measurementClass=" + measurement.getClass().getSimpleName() +
    +                   '}';
    +        }
    +    }
    +
    +    /**
    +     * Allows the subclasses to inject ISnitchMeasurement instances back into this common class so that common
    +     * functionality can be handled here
    +     * @param initialValue The initial value of the measurement, some implementations may use this, others may not
    +     * @return a constructed instance of an ISnitchMeasurement interface
    +     */
    +    abstract protected ISnitchMeasurement measurementImpl(long initialValue);
    +
    +    /**
    +     * Records a latency. This MUST be cheap as it is called in the fast path
    +     */
    +    public void receiveTiming(InetAddressAndPort address, long latency, LatencyMeasurementType measurementType)
    +    {
    +        if (measurementType == LatencyMeasurementType.IGNORE)
    +           return;
    +
    +        AnnotatedMeasurement sample = samples.get(address);
    +
    +        if (sample == null)
    +        {
    +            AnnotatedMeasurement maybeNewSample = new AnnotatedMeasurement(measurementImpl(latency));
    +            sample = samples.putIfAbsent(address, maybeNewSample);
    +            if (sample == null)
    +                sample = maybeNewSample;
    +        }
    +
    +        if (measurementType == LatencyMeasurementType.READ && sample.millisSinceLastMeasure.get() > 0)
    +            sample.millisSinceLastMeasure.lazySet(0);
    +
    +        sample.measurement.sample(latency);
    +    }
    +
    +    @VisibleForTesting
    +    protected void reset()
    +    {
    +        scores.clear();
    +        samples.clear();
    +    }
    +
    +    @VisibleForTesting
    +    void updateScores()
    +    {
    +        if (!StorageService.instance.isGossipActive())
    +            return;
    +
    +        if (!registered)
             {
    -            public void run()
    +            if (MessagingService.instance() != null)
                 {
    -                // we do this so that a host considered bad has a chance to recover, otherwise would we never try
    -                // to read from it, which would cause its score to never change
    -                reset();
    +                MessagingService.instance().register(this);
    +                registered = true;
                 }
    -        };
    +        }
     
    -        if (DatabaseDescriptor.isDaemonInitialized())
    +        this.scores = calculateScores();
    +    }
    +
    +    /**
    +     * This is generally expensive and is called periodically (semi-frequently) not on the fast path.
    +     * @return a freshly constructed scores map.
    +     */
    +    public Map<InetAddressAndPort, Double> calculateScores()
    +    {
    +        double maxLatency = 1;
    +
    +        Map<InetAddressAndPort, Double> measurements = new HashMap<>(samples.size());
    +
    +        // We're going to weight the latency for each host against the worst one we see, to
    +        // arrive at sort of a 'badness percentage' for them. First, find the worst for each:
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry : samples.entrySet())
             {
    -            updateSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(update, dynamicUpdateInterval, dynamicUpdateInterval, TimeUnit.MILLISECONDS);
    -            resetSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(reset, dynamicResetInterval, dynamicResetInterval, TimeUnit.MILLISECONDS);
    -            registerMBean();
    +            // This is expensive for e.g. the Histogram, so do it once and cache the result
    +            double measure = entry.getValue().measurement.measure();
    +            if (measure > maxLatency)
    +                maxLatency = measure;
    +            measurements.put(entry.getKey(), measure);
             }
    +
    +        HashMap<InetAddressAndPort, Double> newScores = new HashMap<>(measurements.size());
    +
    +        // now make another pass to do the weighting based on the maximums we found before
    +        for (Map.Entry<InetAddressAndPort, Double> entry : measurements.entrySet())
    +        {
    +            double score = entry.getValue() / maxLatency;
    +            // finally, add the severity without any weighting, since hosts scale this relative to their own load and the size of the task causing the severity.
    +            // "Severity" is basically a measure of compaction activity (CASSANDRA-3722).
    +            if (USE_SEVERITY)
    +                score += getSeverity(entry.getKey());
    +            // lowest score (least amount of badness) wins.
    +            newScores.put(entry.getKey(), score);
    +        }
    +
    +        return newScores;
         }
     
         /**
    -     * Update configuration from {@link DatabaseDescriptor} and estart the update-scheduler and reset-scheduler tasks
    +     * Update configuration from {@link DatabaseDescriptor} and restart the various scheduler tasks
          * if the configured rates for these tasks have changed.
          */
    -    public void applyConfigChanges()
    +    public void applyConfigChanges(int newDynamicUpdateInternal, int newDynamicLatencyProbeInterval, double newDynamicBadnessThreshold)
         {
    -        if (dynamicUpdateInterval != DatabaseDescriptor.getDynamicUpdateInterval())
    +        if (dynamicUpdateInterval != newDynamicUpdateInternal)
             {
    -            dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    +            dynamicUpdateInterval = newDynamicUpdateInternal;
                 if (DatabaseDescriptor.isDaemonInitialized())
                 {
    -                updateSchedular.cancel(false);
    -                updateSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(update, dynamicUpdateInterval, dynamicUpdateInterval, TimeUnit.MILLISECONDS);
    +                if (updateScoresScheduler != null)
    +                    updateScoresScheduler.cancel(false);
    +                updateScoresScheduler = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(this::updateScores, dynamicUpdateInterval, dynamicUpdateInterval, TimeUnit.MILLISECONDS);
    --- End diff --
    
    Wow so we do this on the scheduled tasks threadpool? STPE doesn't add threads and it's a core pool size is 1. I wonder if other things get flakey because of that. There are quite a few things using it!


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r229460340
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -154,31 +326,203 @@ private void registerMBean()
     
         public void close()
         {
    -        updateSchedular.cancel(false);
    -        resetSchedular.cancel(false);
    +        if (updateScoresScheduler != null)
    +            updateScoresScheduler.cancel(false);
    +        if (updateSamplesScheduler != null)
    +            updateSamplesScheduler.cancel(false);
    --- End diff --
    
    Probably don't want to race with this so you should wait until it's completed using get()?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r229506212
  
    --- Diff: test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java ---
    @@ -60,57 +130,229 @@ private static EndpointsForRange full(InetAddressAndPort... endpoints)
         }
     
         @Test
    -    public void testSnitch() throws InterruptedException, IOException, ConfigurationException
    +    public void testSortedByProximity() throws InterruptedException, IOException, ConfigurationException
         {
    -        // do this because SS needs to be initialized before DES can work properly.
    -        StorageService.instance.unsafeInitialize();
    -        SimpleSnitch ss = new SimpleSnitch();
    -        DynamicEndpointSnitch dsnitch = new DynamicEndpointSnitch(ss, String.valueOf(ss.hashCode()));
    -        InetAddressAndPort self = FBUtilities.getBroadcastAddressAndPort();
    -        InetAddressAndPort host1 = InetAddressAndPort.getByName("127.0.0.2");
    -        InetAddressAndPort host2 = InetAddressAndPort.getByName("127.0.0.3");
    -        InetAddressAndPort host3 = InetAddressAndPort.getByName("127.0.0.4");
    -        InetAddressAndPort host4 = InetAddressAndPort.getByName("127.0.0.5");
    -        List<InetAddressAndPort> hosts = Arrays.asList(host1, host2, host3);
    +        InetAddressAndPort self = hosts[0];
    +        List<InetAddressAndPort> allHosts = Arrays.asList(hosts[1], hosts[2], hosts[3]);
     
             // first, make all hosts equal
    -        setScores(dsnitch, 1, hosts, 10, 10, 10);
    -        EndpointsForRange order = full(host1, host2, host3);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    -
    -        // make host1 a little worse
    -        setScores(dsnitch, 1, hosts, 20, 10, 10);
    -        order = full(host2, host3, host1);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    -
    -        // make host2 as bad as host1
    -        setScores(dsnitch, 2, hosts, 15, 20, 10);
    -        order = full(host3, host1, host2);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    -
    -        // make host3 the worst
    -        setScores(dsnitch, 3, hosts, 10, 10, 30);
    -        order = full(host1, host2, host3);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    -
    -        // make host3 equal to the others
    -        setScores(dsnitch, 5, hosts, 10, 10, 10);
    -        order = full(host1, host2, host3);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    +        setScores(dsnitch, 1, allHosts, 10, 10, 10);
    +        EndpointsForRange order = full(hosts[1], hosts[2], hosts[3]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        // make hosts[1] a little worse
    +        setScores(dsnitch, 2, allHosts, 20, 10, 10);
    +        order = full(hosts[2], hosts[3], hosts[1]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        // make hosts[2] as bad as hosts[1]
    +        setScores(dsnitch, 4, allHosts, 15, 20, 10);
    +        order = full(hosts[3], hosts[1], hosts[2]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        // make hosts[3] the worst
    +        setScores(dsnitch, 10, allHosts, 10, 10, 30);
    +        order = full(hosts[1], hosts[2], hosts[3]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        // make hosts[3] equal to the others
    +        setScores(dsnitch, 15, allHosts, 10, 10, 10);
    +        order = full(hosts[1], hosts[2], hosts[3]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
     
             /// Tests CASSANDRA-6683 improvements
             // make the scores differ enough from the ideal order that we sort by score; under the old
             // dynamic snitch behavior (where we only compared neighbors), these wouldn't get sorted
    -        setScores(dsnitch, 20, hosts, 10, 70, 20);
    -        order = full(host1, host3, host2);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    +        dsnitch.reset();
    +        setScores(dsnitch, 20, allHosts, 10, 70, 20);
    +        order = full(hosts[1], hosts[3], hosts[2]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        order = full(hosts[4], hosts[1], hosts[3], hosts[2]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3], hosts[4])));
    +
    +        setScores(dsnitch, 40, allHosts, 10, 10, 11);
    +        order = full(hosts[4], hosts[1], hosts[2], hosts[3]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3], hosts[4])));
    +    }
    +
    +    // CASSANDRA-14459 improvements to add latency probes instead of resets
    +    @Test
    +    public void testLatencyProbeNeeded()
    +    {
    +        InetAddressAndPort self = hosts[0];
    +
    +        // Four hosts, two are very latent, but all are requested for ranking
    +        dsnitch.receiveTiming(hosts[1], 20, LatencyMeasurementType.READ);
    +        dsnitch.receiveTiming(hosts[2], 10, LatencyMeasurementType.READ);
    +        dsnitch.receiveTiming(hosts[3], 1000, LatencyMeasurementType.READ);
    +        dsnitch.receiveTiming(hosts[4], 1000, LatencyMeasurementType.READ);
    +        dsnitch.updateScores();
    +
    +        EndpointsForRange orderBefore = full(hosts[2], hosts[1], hosts[3], hosts[4]);
    +
    +        Map<InetAddressAndPort, DynamicEndpointSnitch.AnnotatedMeasurement> measurements = dsnitch.getMeasurementsWithPort();
    +        // At this point we haven't ranked any of hosts so we should need no probes even
    +        DynamicEndpointSnitch.calculateProbes(measurements, dsnitch.dynamicLatencyProbeInterval);
    +        assertFalse(measurements.values().stream().anyMatch(m -> m.probeTimerMillis > 0));
    +        assertTrue(measurements.values().stream().allMatch(m -> m.probeFuture == null));
    +
    +        // Two hosts continue receiving traffic but the last two are always ranked by the snitch
    +        for (int i = 0; i < 10; i++)
    +        {
    +            dsnitch.receiveTiming(hosts[1], 20, LatencyMeasurementType.READ);
    +            dsnitch.receiveTiming(hosts[2], 10, LatencyMeasurementType.READ);
    +            Util.assertRCEquals(orderBefore, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3], hosts[4])));
    +        }
    +        dsnitch.updateScores();
    +
    +        int probesBefore = (int) ((DynamicEndpointSnitch.MIN_PROBE_INTERVAL_MS / dsnitch.dynamicLatencyProbeInterval) + 1);
    --- End diff --
    
    Probes before what? (Rename?)


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r237960560
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -154,20 +310,155 @@ private void registerMBean()
     
         public void close()
         {
    -        updateSchedular.cancel(false);
    -        resetSchedular.cancel(false);
    +        if (updateScheduler != null)
    +            updateScheduler.cancel(false);
    +        if (latencyProbeScheduler != null)
    +            latencyProbeScheduler.cancel(false);
     
             MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
             try
             {
    -            mbs.unregisterMBean(new ObjectName(mbeanName));
    +            if (mbeanRegistered)
    +                mbs.unregisterMBean(new ObjectName(mbeanName));
             }
             catch (Exception e)
             {
                 throw new RuntimeException(e);
             }
         }
     
    +    /**
    +     * Determines if latency probes need to be sent, and potentially sends a single latency probe per invocation
    +     */
    +    protected void maybeSendLatencyProbe()
    +    {
    +        if (!StorageService.instance.isGossipActive())
    +            return;
    +
    +        currentProbePosition = latencyProbeNeeded(samples, latencyProbeSequence, currentProbePosition);
    +
    +        if (currentProbePosition < latencyProbeSequence.size())
    +        {
    +            try
    +            {
    +                InetAddressAndPort peer = latencyProbeSequence.get(currentProbePosition);
    +                sendPingMessageToPeer(peer);
    +            }
    +            catch (IndexOutOfBoundsException ignored) {}
    +        }
    +    }
    +
    +    /**
    +     * This method (unfortunately) mutates a lot of state so that it doesn't create any garbage and only iterates the
    +     * sample map a single time . In particular on every call we:
    +     *  - increment every sample's intervalsSinceLastMeasure
    +     *
    +     * When probes should be generated we also potentially:
    +     *  - reset sample's recentlyRequested that have reached the "CONSTANT" phase of probing (10 minutes by default)
    +     *  - add any InetAddressAndPort's that need probing to the provided endpointsToProbe
    +     *  - shuffle the endpointsToProbe
    +     *
    +     * If there are probes to be sent, this method short circuits all generation of probes and just returns the
    +     * passed probePosition plus one.
    +     * @return The position of the passed endpointsToProbe that should be probed.
    +     */
    +    @VisibleForTesting
    +    int latencyProbeNeeded(Map<InetAddressAndPort, AnnotatedMeasurement> samples,
    +                                     List<InetAddressAndPort> endpointsToProbe, int probePosition) {
    +        boolean shouldGenerateProbes = (probePosition >= endpointsToProbe.size());
    +
    +        if (shouldGenerateProbes)
    +        {
    +            endpointsToProbe.clear();
    +            samples.keySet().retainAll(Gossiper.instance.getLiveMembers());
    +        }
    +
    +        // We have to increment intervalsSinceLastMeasure regardless of if we generate probes
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry: samples.entrySet())
    +        {
    +            AnnotatedMeasurement measurement = entry.getValue();
    +            long intervalsSinceLastMeasure = measurement.intervalsSinceLastMeasure.getAndIncrement();
    --- End diff --
    
    I believe that I've addressed this feedback, although I didn't end up looking at the futures in receiveTiming since that's performance critical (we'll maybe send an extra probe, not a big deal imo)


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r239899040
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -21,133 +21,505 @@
     import java.net.InetAddress;
     import java.net.UnknownHostException;
     import java.util.*;
    +import java.util.concurrent.CancellationException;
     import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ExecutionException;
     import java.util.concurrent.ScheduledFuture;
     import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
    +import java.util.concurrent.atomic.AtomicLong;
     import java.util.stream.Collectors;
     
    -import com.codahale.metrics.ExponentiallyDecayingReservoir;
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.util.concurrent.Futures;
    +import com.google.common.util.concurrent.MoreExecutors;
    +import com.google.common.util.concurrent.RateLimiter;
    +import com.google.common.util.concurrent.SettableFuture;
     
    -import com.codahale.metrics.Snapshot;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
     import org.apache.cassandra.concurrent.ScheduledExecutors;
     import org.apache.cassandra.config.DatabaseDescriptor;
    +import org.apache.cassandra.exceptions.RequestFailureReason;
     import org.apache.cassandra.gms.ApplicationState;
     import org.apache.cassandra.gms.EndpointState;
     import org.apache.cassandra.gms.Gossiper;
     import org.apache.cassandra.gms.VersionedValue;
    +import org.apache.cassandra.locator.dynamicsnitch.DynamicEndpointSnitchHistogram;
    +import org.apache.cassandra.net.IAsyncCallbackWithFailure;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +import org.apache.cassandra.net.MessageIn;
    +import org.apache.cassandra.net.MessageOut;
     import org.apache.cassandra.net.MessagingService;
    +import org.apache.cassandra.net.PingMessage;
     import org.apache.cassandra.service.StorageService;
     import org.apache.cassandra.utils.FBUtilities;
     import org.apache.cassandra.utils.MBeanWrapper;
     
    +import static org.apache.cassandra.net.MessagingService.Verb.PING;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.LARGE_MESSAGE;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.SMALL_MESSAGE;
    +
    +
     /**
      * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector
    + * Note that the subclasses (e.g. {@link DynamicEndpointSnitchHistogram}) are responsible for actually measuring
    + * latency and providing an ISnitchMeasurement implementation back to this class.
      */
    -public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
    +public abstract class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
     {
    -    private static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    private static final Logger logger = LoggerFactory.getLogger(DynamicEndpointSnitch.class);
    +
    +    // Latency measurement and ranking. The samples contain latency measurements and the scores are used for ranking
    +    protected boolean registered = false;
    +    protected static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    protected volatile Map<InetAddressAndPort, Double> scores = new HashMap<>();
     
    -    private static final double ALPHA = 0.75; // set to 0.75 to make EDS more biased to towards the newer values
    -    private static final int WINDOW_SIZE = 100;
    +    protected final Map<InetAddressAndPort, AnnotatedMeasurement> samples = new ConcurrentHashMap<>();
     
    -    private volatile int dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    -    private volatile int dynamicResetInterval = DatabaseDescriptor.getDynamicResetInterval();
    -    private volatile double dynamicBadnessThreshold = DatabaseDescriptor.getDynamicBadnessThreshold();
    +    // Latency probe functionality for actively probing endpoints that we haven't measured recently but are ranking
    +    public static final long MAX_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_max_probe_interval_ms", 60 * 10 * 1000L);
    +    public static final long MIN_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_min_probe_interval_ms", 60 * 1000L) ;
    +    // The probe rate is set later when configuration is read in applyConfigChanges
    +    protected static final RateLimiter probeRateLimiter = RateLimiter.create(1);
    +    protected static final DebuggableScheduledThreadPoolExecutor latencyProbeExecutor = new DebuggableScheduledThreadPoolExecutor("LatencyProbes");
    +    private long lastUpdateSamplesNanos = System.nanoTime();
    +
    +    // User configuration of the snitch tunables
    +    protected volatile int dynamicUpdateInterval = -1;
    +    protected volatile int dynamicSampleUpdateInterval = -1;
    +    protected volatile double dynamicBadnessThreshold = 0;
     
         // the score for a merged set of endpoints must be this much worse than the score for separate endpoints to
         // warrant not merging two ranges into a single range
         private static final double RANGE_MERGING_PREFERENCE = 1.5;
     
         private String mbeanName;
    -    private boolean registered = false;
    -
    -    private volatile HashMap<InetAddressAndPort, Double> scores = new HashMap<>();
    -    private final ConcurrentHashMap<InetAddressAndPort, ExponentiallyDecayingReservoir> samples = new ConcurrentHashMap<>();
    +    private boolean mbeanRegistered = false;
     
         public final IEndpointSnitch subsnitch;
     
    -    private volatile ScheduledFuture<?> updateSchedular;
    -    private volatile ScheduledFuture<?> resetSchedular;
    -
    -    private final Runnable update;
    -    private final Runnable reset;
    +    private volatile ScheduledFuture<?> updateScoresScheduler = null;
    +    private volatile ScheduledFuture<?> updateSamplesScheduler = null;
     
         public DynamicEndpointSnitch(IEndpointSnitch snitch)
         {
             this(snitch, null);
         }
     
    -    public DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
    +    protected DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
         {
             mbeanName = "org.apache.cassandra.db:type=DynamicEndpointSnitch";
             if (instance != null)
                 mbeanName += ",instance=" + instance;
             subsnitch = snitch;
    -        update = new Runnable()
    +
    +        if (DatabaseDescriptor.isDaemonInitialized())
             {
    -            public void run()
    +            open();
    +        }
    +    }
    +
    +    /**
    +     * Update configuration of the background tasks and restart the various scheduler tasks
    +     * if the configured rates for these tasks have changed.
    +     */
    +    public void applyConfigChanges(int newDynamicUpdateInternal, int newDynamicSampleUpdateInterval, double newDynamicBadnessThreshold)
    +    {
    +        if (DatabaseDescriptor.isDaemonInitialized())
    +        {
    +            if (dynamicUpdateInterval != newDynamicUpdateInternal || updateScoresScheduler == null)
                 {
    -                updateScores();
    +                cancelAndWait(updateScoresScheduler, Math.max(1, dynamicUpdateInterval), TimeUnit.MILLISECONDS);
    +                updateScoresScheduler = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(this::updateScores, newDynamicUpdateInternal, newDynamicUpdateInternal, TimeUnit.MILLISECONDS);
                 }
    -        };
    -        reset = new Runnable()
    +
    +            if (dynamicSampleUpdateInterval != newDynamicSampleUpdateInterval || updateSamplesScheduler == null)
    +            {
    +                cancelAndWait(updateSamplesScheduler, Math.max(1, dynamicSampleUpdateInterval), TimeUnit.MILLISECONDS);
    +                if (newDynamicSampleUpdateInterval > 0)
    +                    updateSamplesScheduler = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(this::updateSamples, newDynamicSampleUpdateInterval, newDynamicSampleUpdateInterval, TimeUnit.MILLISECONDS);
    +            }
    +        }
    +
    +        dynamicUpdateInterval = newDynamicUpdateInternal;
    +        dynamicSampleUpdateInterval = newDynamicSampleUpdateInterval;
    +        dynamicBadnessThreshold = newDynamicBadnessThreshold;
    +
    +        if (dynamicSampleUpdateInterval > 0)
    +            probeRateLimiter.setRate(dynamicSampleUpdateInterval);
    +    }
    +
    +    public synchronized void open()
    +    {
    +        applyConfigChanges(DatabaseDescriptor.getDynamicUpdateInterval(),
    +                           DatabaseDescriptor.getDynamicSampleUpdateInterval(),
    +                           DatabaseDescriptor.getDynamicBadnessThreshold());
    +
    +        MBeanWrapper.instance.registerMBean(this, mbeanName);
    +        mbeanRegistered = true;
    +    }
    +
    +    public synchronized void close()
    +    {
    +        cancelAndWait(updateScoresScheduler, Math.max(1, dynamicUpdateInterval), TimeUnit.MILLISECONDS);
    +        cancelAndWait(updateSamplesScheduler, Math.max(1, dynamicSampleUpdateInterval), TimeUnit.MILLISECONDS);
    +        updateScoresScheduler = null;
    +        updateSamplesScheduler = null;
    +
    +        for (AnnotatedMeasurement measurement : samples.values())
             {
    -            public void run()
    +            cancelAndWait(measurement.probeFuture, PING.getTimeout(), TimeUnit.MILLISECONDS);
    +
    +            measurement.millisSinceLastMeasure.set(0);
    +            measurement.millisSinceLastRequest.set(MAX_PROBE_INTERVAL_MS);
    +            measurement.nextProbeDelayMillis = 0;
    +        }
    +
    +        if (mbeanRegistered)
    +            MBeanWrapper.instance.unregisterMBean(mbeanName);
    +
    +        mbeanRegistered = false;
    +    }
    +
    +    private static void cancelAndWait(ScheduledFuture future, long timeout, TimeUnit unit)
    +    {
    +        if (future != null)
    +        {
    +            future.cancel(false);
    +            try
                 {
    -                // we do this so that a host considered bad has a chance to recover, otherwise would we never try
    -                // to read from it, which would cause its score to never change
    -                reset();
    +                future.get(timeout, unit);
                 }
    -        };
    +            catch (CancellationException | InterruptedException | ExecutionException | TimeoutException ignored)
    +            {
    +                // Exception is expected to happen eventually due to the cancel -> get
    +            }
    +        }
    +    }
     
    -        if (DatabaseDescriptor.isDaemonInitialized())
    +    /**
    +     * Allows subclasses to inject new ways of measuring latency back to this abstract base class.
    +     */
    +    protected interface ISnitchMeasurement
    +    {
    +        void sample(long value);
    +        double measure();
    +        Iterable<Double> measurements();
    +    }
    +
    +    /**
    +     * Adds some boookeeping that the DES uses over top of the various metrics techniques used by the
    +     * implementations. This is used to allow CASSANDRA-14459 latency probes as well as further safe
    +     * experimentation on new latency measurement techniques in CASSANDRA-14817
    +     *
    +     * {@link AnnotatedMeasurement#millisSinceLastRequest} is set to zero through
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}. It defaults to
    +     * the maximum interval so that we only start probing once it has been ranked at least once
    +     * {@link AnnotatedMeasurement#millisSinceLastMeasure} is set to zero from
    +     * {@link DynamicEndpointSnitch#receiveTiming(InetAddressAndPort, long, LatencyMeasurementType)}.
    +     *
    +     * {@link AnnotatedMeasurement#millisSinceLastMeasure and {@link AnnotatedMeasurement#nextProbeDelayMillis }
    +     * are incremented via {@link DynamicEndpointSnitch#updateSamples()}
    +     */
    +    protected static class AnnotatedMeasurement
    +    {
    +        // Used to optimally target latency probes only on nodes that are both requested for ranking
    +        // and are not being measured. For example with token aware clients a large portion of the cluster will never
    +        // be ranked at all and therefore we won't probe them.
    +        public AtomicLong millisSinceLastRequest = new AtomicLong(MAX_PROBE_INTERVAL_MS);
    +        public AtomicLong millisSinceLastMeasure = new AtomicLong(0);
    +        public volatile long nextProbeDelayMillis = 0;
    +        public volatile ScheduledFuture<?> probeFuture = null;
    +
    +        // The underlying measurement technique. E.g. a median filter (histogram) or an EMA filter, or ...
    +        public final ISnitchMeasurement measurement;
    +        public volatile double cachedMeasurement;
    +
    +        public AnnotatedMeasurement(ISnitchMeasurement measurement)
             {
    -            updateSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(update, dynamicUpdateInterval, dynamicUpdateInterval, TimeUnit.MILLISECONDS);
    -            resetSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(reset, dynamicResetInterval, dynamicResetInterval, TimeUnit.MILLISECONDS);
    -            registerMBean();
    +            this.measurement = measurement;
    +            this.cachedMeasurement = measurement.measure();
    +        }
    +
    +        @Override
    +        public String toString()
    +        {
    +            return "AnnotatedMeasurement{" +
    +                   "millisSinceLastRequest=" + millisSinceLastRequest +
    +                   ", millisSinceLastMeasure=" + millisSinceLastMeasure +
    +                   ", nextProbeDelayMillis=" + nextProbeDelayMillis +
    +                   ", probeFuturePending=" + (probeFuture != null && !probeFuture.isDone()) +
    +                   ", measurementClass=" + measurement.getClass().getSimpleName() +
    +                   ", cachedMeasurement=" + cachedMeasurement +
    +                   '}';
             }
         }
     
         /**
    -     * Update configuration from {@link DatabaseDescriptor} and estart the update-scheduler and reset-scheduler tasks
    -     * if the configured rates for these tasks have changed.
    +     * Allows the subclasses to inject ISnitchMeasurement instances back into this common class so that common
    +     * functionality can be handled here
    +     * @param initialValue The initial value of the measurement, some implementations may use this, others may not
    +     * @return a constructed instance of an ISnitchMeasurement interface
          */
    -    public void applyConfigChanges()
    +    abstract protected ISnitchMeasurement measurementImpl(long initialValue);
    +
    +    /**
    +     * Records a latency. This MUST be cheap as it is called in the fast path
    +     */
    +    public void receiveTiming(InetAddressAndPort address, long latencyMicros, LatencyMeasurementType measurementType)
    +    {
    +        if (measurementType == LatencyMeasurementType.IGNORE)
    +           return;
    +
    +        AnnotatedMeasurement sample = samples.get(address);
    +
    +        if (sample == null)
    +        {
    +            AnnotatedMeasurement maybeNewSample = new AnnotatedMeasurement(measurementImpl(latencyMicros));
    +            sample = samples.putIfAbsent(address, maybeNewSample);
    +            if (sample == null)
    +                sample = maybeNewSample;
    +        }
    +
    +        if (measurementType == LatencyMeasurementType.READ && sample.millisSinceLastMeasure.get() > 0)
    +            sample.millisSinceLastMeasure.lazySet(0);
    +
    +        sample.measurement.sample(latencyMicros);
    +    }
    +
    +    @VisibleForTesting
    +    protected void reset()
    +    {
    +        scores.clear();
    +        samples.clear();
    +    }
    +
    +    @VisibleForTesting
    +    protected void updateScores()
         {
    -        if (dynamicUpdateInterval != DatabaseDescriptor.getDynamicUpdateInterval())
    +        if (!StorageService.instance.isGossipActive())
    +            return;
    +
    +        if (!registered)
    --- End diff --
    
    So why do we register in updateScores? A bit weird no?
    
    Also maybe MessagingService should store a list of WeakReferences so we don't need to unregister? Or we should unregister.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r229486041
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -23,128 +23,300 @@
     import java.net.UnknownHostException;
     import java.util.*;
     import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ExecutionException;
     import java.util.concurrent.ScheduledFuture;
     import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
     import java.util.stream.Collectors;
     
    -import com.codahale.metrics.ExponentiallyDecayingReservoir;
     import javax.management.MBeanServer;
     import javax.management.ObjectName;
     
    -import com.codahale.metrics.Snapshot;
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.util.concurrent.Futures;
    +import com.google.common.util.concurrent.MoreExecutors;
    +import com.google.common.util.concurrent.RateLimiter;
    +import com.google.common.util.concurrent.SettableFuture;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
     import org.apache.cassandra.concurrent.ScheduledExecutors;
     import org.apache.cassandra.config.DatabaseDescriptor;
    +import org.apache.cassandra.exceptions.RequestFailureReason;
     import org.apache.cassandra.gms.ApplicationState;
     import org.apache.cassandra.gms.EndpointState;
     import org.apache.cassandra.gms.Gossiper;
     import org.apache.cassandra.gms.VersionedValue;
    +import org.apache.cassandra.locator.dynamicsnitch.DynamicEndpointSnitchHistogram;
    +import org.apache.cassandra.net.IAsyncCallbackWithFailure;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +import org.apache.cassandra.net.MessageIn;
    +import org.apache.cassandra.net.MessageOut;
     import org.apache.cassandra.net.MessagingService;
    +import org.apache.cassandra.net.PingMessage;
     import org.apache.cassandra.service.StorageService;
     import org.apache.cassandra.utils.FBUtilities;
     
    +import static org.apache.cassandra.net.MessagingService.Verb.PING;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.LARGE_MESSAGE;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.SMALL_MESSAGE;
    +
    +
     /**
      * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector
    + * Note that the subclasses (e.g. {@link DynamicEndpointSnitchHistogram}) are responsible for actually measuring
    + * latency and providing an ISnitchMeasurement implementation back to this class.
      */
    -public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
    +public abstract class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
     {
    -    private static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    -
    -    private static final double ALPHA = 0.75; // set to 0.75 to make EDS more biased to towards the newer values
    -    private static final int WINDOW_SIZE = 100;
    -
    -    private volatile int dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    -    private volatile int dynamicResetInterval = DatabaseDescriptor.getDynamicResetInterval();
    -    private volatile double dynamicBadnessThreshold = DatabaseDescriptor.getDynamicBadnessThreshold();
    +    private static final Logger logger = LoggerFactory.getLogger(DynamicEndpointSnitch.class);
    +
    +    // Latency measurement and ranking. The samples contain latency measurements and the scores are used for ranking
    +    protected boolean registered = false;
    +    protected static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    protected volatile Map<InetAddressAndPort, Double> scores = new HashMap<>();
    +    protected final Map<InetAddressAndPort, AnnotatedMeasurement> samples = new ConcurrentHashMap<>();
    +
    +    // Latency probe functionality for actively probing endpoints that we haven't measured recently but are ranking
    +    public static final long MAX_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_max_probe_interval_ms", 60 * 10 * 1000L);
    +    public static final long MIN_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_min_probe_interval_ms", 60 * 1000L) ;
    +    // The probe rate is set later when configuration is read
    +    protected static final RateLimiter probeRateLimiter = RateLimiter.create(1);
    +    protected static final DebuggableScheduledThreadPoolExecutor latencyProbeExecutor = new DebuggableScheduledThreadPoolExecutor(1, "LatencyProbes", Thread.MIN_PRIORITY);
    +
    +    // User configuration of the snitch tunables
    +    protected volatile int dynamicUpdateInterval = -1;
    +    protected volatile int dynamicLatencyProbeInterval = -1;
    +    protected volatile double dynamicBadnessThreshold = 0;
     
         // the score for a merged set of endpoints must be this much worse than the score for separate endpoints to
         // warrant not merging two ranges into a single range
         private static final double RANGE_MERGING_PREFERENCE = 1.5;
     
         private String mbeanName;
    -    private boolean registered = false;
    -
    -    private volatile HashMap<InetAddressAndPort, Double> scores = new HashMap<>();
    -    private final ConcurrentHashMap<InetAddressAndPort, ExponentiallyDecayingReservoir> samples = new ConcurrentHashMap<>();
    +    private boolean mbeanRegistered = false;
     
         public final IEndpointSnitch subsnitch;
     
    -    private volatile ScheduledFuture<?> updateSchedular;
    -    private volatile ScheduledFuture<?> resetSchedular;
    -
    -    private final Runnable update;
    -    private final Runnable reset;
    +    private volatile ScheduledFuture<?> updateScoresScheduler;
    +    private volatile ScheduledFuture<?> updateSamplesScheduler;
     
         public DynamicEndpointSnitch(IEndpointSnitch snitch)
         {
             this(snitch, null);
         }
     
    -    public DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
    +    protected DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
         {
             mbeanName = "org.apache.cassandra.db:type=DynamicEndpointSnitch";
             if (instance != null)
                 mbeanName += ",instance=" + instance;
             subsnitch = snitch;
    -        update = new Runnable()
    +
    +        if (DatabaseDescriptor.isDaemonInitialized())
             {
    -            public void run()
    -            {
    -                updateScores();
    -            }
    -        };
    -        reset = new Runnable()
    +            applyConfigChanges(DatabaseDescriptor.getDynamicUpdateInterval(),
    +                               DatabaseDescriptor.getDynamicSampleUpdateInterval(),
    +                               DatabaseDescriptor.getDynamicBadnessThreshold());
    +            open();
    +        }
    +    }
    +
    +    /**
    +     * Allows subclasses to inject new ways of measuring latency back to this abstract base class.
    +     */
    +    protected interface ISnitchMeasurement
    +    {
    +        void sample(long value);
    +        double measure();
    +        Iterable<Double> measurements();
    +    }
    +
    +    /**
    +     * Adds some boookeeping that the DES uses over top of the various metrics techniques used by the
    +     * implementations. This is used to allow CASSANDRA-14459 latency probes as well as further experimentation
    +     * on new latency measurement techniques in CASSANDRA-14817
    +     *
    +     * {@link AnnotatedMeasurement#millisSinceLastRequest} is set to zero through
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}. It defaults to
    +     * the maximum interval so that we only start probing once it has been ranked at least once
    +     * {@link AnnotatedMeasurement#millisSinceLastMeasure} is set to zero from
    +     * {@link DynamicEndpointSnitch#receiveTiming(InetAddressAndPort, long, LatencyMeasurementType)}.
    +     *
    +     * {@link AnnotatedMeasurement#millisSinceLastMeasure and {@link AnnotatedMeasurement#probeTimerMillis}
    +     * are incremented via {@link DynamicEndpointSnitch#updateSamples()}
    +     */
    +    protected static class AnnotatedMeasurement
    +    {
    +        // Used to optimally target latency probes only on nodes that are both requested for ranking
    +        // and are not being measured. For example with token aware clients a large portion of the cluster will never
    +        // be ranked at all and therefore we won't probe them.
    +        public AtomicLong millisSinceLastRequest = new AtomicLong(MAX_PROBE_INTERVAL_MS);
    +        public AtomicLong millisSinceLastMeasure = new AtomicLong(0);
    +        public long probeTimerMillis = 0;
    +        public ScheduledFuture<?> probeFuture = null;
    +
    +        // The underlying measurement technique. E.g. a median filter (histogram) or an EMA filter, or ...
    +        public final ISnitchMeasurement measurement;
    +
    +        public AnnotatedMeasurement(ISnitchMeasurement measurement)
    +        {
    +            this.measurement = measurement;
    +        }
    +
    +        @Override
    +        public String toString()
    +        {
    +            return "AnnotatedMeasurement{" +
    +                   "millisSinceLastRequest=" + millisSinceLastRequest +
    +                   ", millisSinceLastMeasure=" + millisSinceLastMeasure +
    +                   ", probeTimerMillis=" + probeTimerMillis +
    +                   ", probeFuturePending=" + (probeFuture != null && !probeFuture.isDone()) +
    +                   ", measurementClass=" + measurement.getClass().getSimpleName() +
    +                   '}';
    +        }
    +    }
    +
    +    /**
    +     * Allows the subclasses to inject ISnitchMeasurement instances back into this common class so that common
    +     * functionality can be handled here
    +     * @param initialValue The initial value of the measurement, some implementations may use this, others may not
    +     * @return a constructed instance of an ISnitchMeasurement interface
    +     */
    +    abstract protected ISnitchMeasurement measurementImpl(long initialValue);
    +
    +    /**
    +     * Records a latency. This MUST be cheap as it is called in the fast path
    +     */
    +    public void receiveTiming(InetAddressAndPort address, long latency, LatencyMeasurementType measurementType)
    +    {
    +        if (measurementType == LatencyMeasurementType.IGNORE)
    +           return;
    +
    +        AnnotatedMeasurement sample = samples.get(address);
    +
    +        if (sample == null)
    +        {
    +            AnnotatedMeasurement maybeNewSample = new AnnotatedMeasurement(measurementImpl(latency));
    +            sample = samples.putIfAbsent(address, maybeNewSample);
    +            if (sample == null)
    +                sample = maybeNewSample;
    +        }
    +
    +        if (measurementType == LatencyMeasurementType.READ && sample.millisSinceLastMeasure.get() > 0)
    +            sample.millisSinceLastMeasure.lazySet(0);
    +
    +        sample.measurement.sample(latency);
    +    }
    +
    +    @VisibleForTesting
    +    protected void reset()
    +    {
    +        scores.clear();
    +        samples.clear();
    +    }
    +
    +    @VisibleForTesting
    +    void updateScores()
    +    {
    +        if (!StorageService.instance.isGossipActive())
    +            return;
    +
    +        if (!registered)
             {
    -            public void run()
    +            if (MessagingService.instance() != null)
                 {
    -                // we do this so that a host considered bad has a chance to recover, otherwise would we never try
    -                // to read from it, which would cause its score to never change
    -                reset();
    +                MessagingService.instance().register(this);
    +                registered = true;
                 }
    -        };
    +        }
     
    -        if (DatabaseDescriptor.isDaemonInitialized())
    +        this.scores = calculateScores();
    +    }
    +
    +    /**
    +     * This is generally expensive and is called periodically (semi-frequently) not on the fast path.
    +     * @return a freshly constructed scores map.
    +     */
    +    public Map<InetAddressAndPort, Double> calculateScores()
    +    {
    +        double maxLatency = 1;
    +
    +        Map<InetAddressAndPort, Double> measurements = new HashMap<>(samples.size());
    +
    +        // We're going to weight the latency for each host against the worst one we see, to
    +        // arrive at sort of a 'badness percentage' for them. First, find the worst for each:
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry : samples.entrySet())
             {
    -            updateSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(update, dynamicUpdateInterval, dynamicUpdateInterval, TimeUnit.MILLISECONDS);
    -            resetSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(reset, dynamicResetInterval, dynamicResetInterval, TimeUnit.MILLISECONDS);
    -            registerMBean();
    +            // This is expensive for e.g. the Histogram, so do it once and cache the result
    +            double measure = entry.getValue().measurement.measure();
    +            if (measure > maxLatency)
    +                maxLatency = measure;
    +            measurements.put(entry.getKey(), measure);
             }
    +
    +        HashMap<InetAddressAndPort, Double> newScores = new HashMap<>(measurements.size());
    +
    +        // now make another pass to do the weighting based on the maximums we found before
    +        for (Map.Entry<InetAddressAndPort, Double> entry : measurements.entrySet())
    +        {
    +            double score = entry.getValue() / maxLatency;
    +            // finally, add the severity without any weighting, since hosts scale this relative to their own load and the size of the task causing the severity.
    +            // "Severity" is basically a measure of compaction activity (CASSANDRA-3722).
    +            if (USE_SEVERITY)
    +                score += getSeverity(entry.getKey());
    +            // lowest score (least amount of badness) wins.
    +            newScores.put(entry.getKey(), score);
    +        }
    +
    +        return newScores;
         }
     
         /**
    -     * Update configuration from {@link DatabaseDescriptor} and estart the update-scheduler and reset-scheduler tasks
    +     * Update configuration from {@link DatabaseDescriptor} and restart the various scheduler tasks
          * if the configured rates for these tasks have changed.
          */
    -    public void applyConfigChanges()
    +    public void applyConfigChanges(int newDynamicUpdateInternal, int newDynamicLatencyProbeInterval, double newDynamicBadnessThreshold)
         {
    -        if (dynamicUpdateInterval != DatabaseDescriptor.getDynamicUpdateInterval())
    +        if (dynamicUpdateInterval != newDynamicUpdateInternal)
             {
    -            dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    +            dynamicUpdateInterval = newDynamicUpdateInternal;
                 if (DatabaseDescriptor.isDaemonInitialized())
                 {
    -                updateSchedular.cancel(false);
    -                updateSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(update, dynamicUpdateInterval, dynamicUpdateInterval, TimeUnit.MILLISECONDS);
    +                if (updateScoresScheduler != null)
    +                    updateScoresScheduler.cancel(false);
    +                updateScoresScheduler = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(this::updateScores, dynamicUpdateInterval, dynamicUpdateInterval, TimeUnit.MILLISECONDS);
                 }
             }
     
    -        if (dynamicResetInterval != DatabaseDescriptor.getDynamicResetInterval())
    +        if (this.dynamicLatencyProbeInterval != newDynamicLatencyProbeInterval)
             {
    -            dynamicResetInterval = DatabaseDescriptor.getDynamicResetInterval();
    +            this.dynamicLatencyProbeInterval = newDynamicLatencyProbeInterval;
                 if (DatabaseDescriptor.isDaemonInitialized())
                 {
    -                resetSchedular.cancel(false);
    -                resetSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(reset, dynamicResetInterval, dynamicResetInterval, TimeUnit.MILLISECONDS);
    +                if (updateSamplesScheduler != null)
    +                    updateSamplesScheduler.cancel(false);
    +
    +                if (newDynamicLatencyProbeInterval > 0)
    +                {
    +                    probeRateLimiter.setRate(dynamicLatencyProbeInterval);
    +                    updateSamplesScheduler = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(this::updateSamples, dynamicLatencyProbeInterval, dynamicLatencyProbeInterval, TimeUnit.MILLISECONDS);
    --- End diff --
    
    Hmm... really I just wanted to not be blocked behind all the other things in that other thread pool. Where is an asynchronous rate limit when you need it. I suppose it's fine since we only need it to run on the order of seconds and we can stand to be delayed a little.



---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225361184
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -22,121 +22,194 @@
     import java.net.InetAddress;
     import java.net.UnknownHostException;
     import java.util.*;
    -import java.util.concurrent.ConcurrentHashMap;
     import java.util.concurrent.ScheduledFuture;
     import java.util.concurrent.TimeUnit;
     import java.util.stream.Collectors;
     
    -import com.codahale.metrics.ExponentiallyDecayingReservoir;
     import javax.management.MBeanServer;
     import javax.management.ObjectName;
     
    -import com.codahale.metrics.Snapshot;
    +import com.google.common.annotations.VisibleForTesting;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
     import org.apache.cassandra.concurrent.ScheduledExecutors;
     import org.apache.cassandra.config.DatabaseDescriptor;
     import org.apache.cassandra.gms.ApplicationState;
     import org.apache.cassandra.gms.EndpointState;
     import org.apache.cassandra.gms.Gossiper;
     import org.apache.cassandra.gms.VersionedValue;
    +import org.apache.cassandra.net.IAsyncCallback;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +import org.apache.cassandra.net.MessageIn;
    +import org.apache.cassandra.net.MessageOut;
     import org.apache.cassandra.net.MessagingService;
    +import org.apache.cassandra.net.PingMessage;
     import org.apache.cassandra.service.StorageService;
     import org.apache.cassandra.utils.FBUtilities;
     
    +import static org.apache.cassandra.net.MessagingService.Verb.PING;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.LARGE_MESSAGE;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.SMALL_MESSAGE;
    +
    +
     /**
      * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector
    + * Note that the subclasses (e.g. {@link DynamicEndpointSnitchHistogram}) are responsible for actually measuring
    + * latency and populating the {@link #scores} map.
      */
    -public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
    +public abstract class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
     {
    -    private static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    -
    -    private static final double ALPHA = 0.75; // set to 0.75 to make EDS more biased to towards the newer values
    -    private static final int WINDOW_SIZE = 100;
    -
    -    private volatile int dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    -    private volatile int dynamicResetInterval = DatabaseDescriptor.getDynamicResetInterval();
    -    private volatile double dynamicBadnessThreshold = DatabaseDescriptor.getDynamicBadnessThreshold();
    +    private static final Logger logger = LoggerFactory.getLogger(DynamicEndpointSnitch.class);
    +
    +    // Subclass specific functionality
    +    protected static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    protected boolean registered = false;
    +    // The scores map is updated via copy in updateScores
    +    // We keep it in the base class for performance reasons (so it can be easily aliased)
    +    protected volatile Map<InetAddressAndPort, Double> scores = new HashMap<>();
    +
    +    // Rate limit how often we generate latency probes
    +    protected long nextAllowedProbeGenerationTime;
    +    protected long nextProbeGenerationTime;
    +    protected int currentProbePosition = 0;
    +    protected final List<InetAddressAndPort> latencyProbeSequence = new ArrayList<>();
    +
    +    // DES general functionality
    +    protected volatile int dynamicUpdateInterval = -1;
    +    protected volatile int dynamicLatencyProbeInterval = -1;
    +    protected volatile double dynamicBadnessThreshold = 0;
     
         // the score for a merged set of endpoints must be this much worse than the score for separate endpoints to
         // warrant not merging two ranges into a single range
         private static final double RANGE_MERGING_PREFERENCE = 1.5;
     
         private String mbeanName;
    -    private boolean registered = false;
    -
    -    private volatile HashMap<InetAddressAndPort, Double> scores = new HashMap<>();
    -    private final ConcurrentHashMap<InetAddressAndPort, ExponentiallyDecayingReservoir> samples = new ConcurrentHashMap<>();
    +    private boolean mbeanRegistered = false;
     
         public final IEndpointSnitch subsnitch;
     
    -    private volatile ScheduledFuture<?> updateSchedular;
    -    private volatile ScheduledFuture<?> resetSchedular;
    +    private volatile ScheduledFuture<?> updateScheduler;
    +    private volatile ScheduledFuture<?> latencyProbeScheduler;
     
    -    private final Runnable update;
    -    private final Runnable reset;
    +    private final Runnable update = this::updateScores;
    +    private final Runnable latencyProbe = this::maybeSendLatencyProbe;
     
         public DynamicEndpointSnitch(IEndpointSnitch snitch)
         {
             this(snitch, null);
         }
     
    -    public DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
    +    protected DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
         {
             mbeanName = "org.apache.cassandra.db:type=DynamicEndpointSnitch";
             if (instance != null)
                 mbeanName += ",instance=" + instance;
             subsnitch = snitch;
    -        update = new Runnable()
    -        {
    -            public void run()
    -            {
    -                updateScores();
    -            }
    -        };
    -        reset = new Runnable()
    -        {
    -            public void run()
    -            {
    -                // we do this so that a host considered bad has a chance to recover, otherwise would we never try
    -                // to read from it, which would cause its score to never change
    -                reset();
    -            }
    -        };
     
             if (DatabaseDescriptor.isDaemonInitialized())
             {
    -            updateSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(update, dynamicUpdateInterval, dynamicUpdateInterval, TimeUnit.MILLISECONDS);
    -            resetSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(reset, dynamicResetInterval, dynamicResetInterval, TimeUnit.MILLISECONDS);
    +            applyConfigChanges();
                 registerMBean();
             }
    +        nextProbeGenerationTime = System.nanoTime();
    +    }
    +
    +    /**
    +     * Records a latency. This MUST be cheap as it is called in the fast path
    +     */
    +    abstract public void receiveTiming(InetAddressAndPort address, long latency, LatencyMeasurementType measurementType);
    +
    +    /**
    +     * Called periodically from {@link DynamicEndpointSnitch#updateScores()}. This method returns a new score map
    +     * that will be assigned to the {@link DynamicEndpointSnitch#scores} map.
    +     * This is generally more expensive than {@link DynamicEndpointSnitch#receiveTiming}
    +     */
    +    abstract public Map<InetAddressAndPort, Double> calculateScores();
    --- End diff --
    
    I was thinking that it would be pretty different in future implementations, but I can just make it protected and then subclasses can redefine it anyways :-)


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r238169787
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -154,31 +326,203 @@ private void registerMBean()
     
         public void close()
         {
    -        updateSchedular.cancel(false);
    -        resetSchedular.cancel(false);
    +        if (updateScoresScheduler != null)
    +            updateScoresScheduler.cancel(false);
    +        if (updateSamplesScheduler != null)
    +            updateSamplesScheduler.cancel(false);
    +
    +        for (AnnotatedMeasurement measurement : samples.values())
    +        {
    +            if (measurement.probeFuture != null)
    +                measurement.probeFuture.cancel(false);
    +
    +            measurement.millisSinceLastMeasure.set(0);
    +            measurement.millisSinceLastRequest.set(MAX_PROBE_INTERVAL_MS);
    +            measurement.probeTimerMillis = 0;
    +        }
     
             MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
             try
             {
    -            mbs.unregisterMBean(new ObjectName(mbeanName));
    +            if (mbeanRegistered)
    +                mbs.unregisterMBean(new ObjectName(mbeanName));
             }
             catch (Exception e)
             {
                 throw new RuntimeException(e);
             }
         }
     
    +    /**
    +     * Background task running on the samples dictionary. The default implementation sends latency probes (PING)
    +     * messages to explore nodes that we have not received timings for recently but have ranked in
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}.
    +     */
    +    protected void updateSamples()
    +    {
    +        // Split calculation of probe timers from sending probes for testability
    +        calculateProbes(samples, dynamicLatencyProbeInterval);
    +
    +        if (!StorageService.instance.isGossipActive())
    --- End diff --
    
    This needs to be here so that the bookeeping clocks keep ticking. Basically we do the calculation but don't send any probes (respecting a disabled gossip indicating we shouldn't be sending messages).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225329303
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -22,121 +22,194 @@
     import java.net.InetAddress;
     import java.net.UnknownHostException;
     import java.util.*;
    -import java.util.concurrent.ConcurrentHashMap;
     import java.util.concurrent.ScheduledFuture;
     import java.util.concurrent.TimeUnit;
     import java.util.stream.Collectors;
     
    -import com.codahale.metrics.ExponentiallyDecayingReservoir;
     import javax.management.MBeanServer;
     import javax.management.ObjectName;
     
    -import com.codahale.metrics.Snapshot;
    +import com.google.common.annotations.VisibleForTesting;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
     import org.apache.cassandra.concurrent.ScheduledExecutors;
     import org.apache.cassandra.config.DatabaseDescriptor;
     import org.apache.cassandra.gms.ApplicationState;
     import org.apache.cassandra.gms.EndpointState;
     import org.apache.cassandra.gms.Gossiper;
     import org.apache.cassandra.gms.VersionedValue;
    +import org.apache.cassandra.net.IAsyncCallback;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +import org.apache.cassandra.net.MessageIn;
    +import org.apache.cassandra.net.MessageOut;
     import org.apache.cassandra.net.MessagingService;
    +import org.apache.cassandra.net.PingMessage;
     import org.apache.cassandra.service.StorageService;
     import org.apache.cassandra.utils.FBUtilities;
     
    +import static org.apache.cassandra.net.MessagingService.Verb.PING;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.LARGE_MESSAGE;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.SMALL_MESSAGE;
    +
    +
     /**
      * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector
    + * Note that the subclasses (e.g. {@link DynamicEndpointSnitchHistogram}) are responsible for actually measuring
    + * latency and populating the {@link #scores} map.
      */
    -public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
    +public abstract class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
     {
    -    private static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    -
    -    private static final double ALPHA = 0.75; // set to 0.75 to make EDS more biased to towards the newer values
    -    private static final int WINDOW_SIZE = 100;
    -
    -    private volatile int dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    -    private volatile int dynamicResetInterval = DatabaseDescriptor.getDynamicResetInterval();
    -    private volatile double dynamicBadnessThreshold = DatabaseDescriptor.getDynamicBadnessThreshold();
    +    private static final Logger logger = LoggerFactory.getLogger(DynamicEndpointSnitch.class);
    +
    +    // Subclass specific functionality
    +    protected static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    protected boolean registered = false;
    +    // The scores map is updated via copy in updateScores
    +    // We keep it in the base class for performance reasons (so it can be easily aliased)
    +    protected volatile Map<InetAddressAndPort, Double> scores = new HashMap<>();
    +
    +    // Rate limit how often we generate latency probes
    +    protected long nextAllowedProbeGenerationTime;
    +    protected long nextProbeGenerationTime;
    +    protected int currentProbePosition = 0;
    +    protected final List<InetAddressAndPort> latencyProbeSequence = new ArrayList<>();
    +
    +    // DES general functionality
    +    protected volatile int dynamicUpdateInterval = -1;
    +    protected volatile int dynamicLatencyProbeInterval = -1;
    +    protected volatile double dynamicBadnessThreshold = 0;
     
         // the score for a merged set of endpoints must be this much worse than the score for separate endpoints to
         // warrant not merging two ranges into a single range
         private static final double RANGE_MERGING_PREFERENCE = 1.5;
     
         private String mbeanName;
    -    private boolean registered = false;
    -
    -    private volatile HashMap<InetAddressAndPort, Double> scores = new HashMap<>();
    -    private final ConcurrentHashMap<InetAddressAndPort, ExponentiallyDecayingReservoir> samples = new ConcurrentHashMap<>();
    +    private boolean mbeanRegistered = false;
     
         public final IEndpointSnitch subsnitch;
     
    -    private volatile ScheduledFuture<?> updateSchedular;
    -    private volatile ScheduledFuture<?> resetSchedular;
    +    private volatile ScheduledFuture<?> updateScheduler;
    +    private volatile ScheduledFuture<?> latencyProbeScheduler;
     
    -    private final Runnable update;
    -    private final Runnable reset;
    +    private final Runnable update = this::updateScores;
    +    private final Runnable latencyProbe = this::maybeSendLatencyProbe;
     
         public DynamicEndpointSnitch(IEndpointSnitch snitch)
         {
             this(snitch, null);
         }
     
    -    public DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
    +    protected DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
         {
             mbeanName = "org.apache.cassandra.db:type=DynamicEndpointSnitch";
             if (instance != null)
                 mbeanName += ",instance=" + instance;
             subsnitch = snitch;
    -        update = new Runnable()
    -        {
    -            public void run()
    -            {
    -                updateScores();
    -            }
    -        };
    -        reset = new Runnable()
    -        {
    -            public void run()
    -            {
    -                // we do this so that a host considered bad has a chance to recover, otherwise would we never try
    -                // to read from it, which would cause its score to never change
    -                reset();
    -            }
    -        };
     
             if (DatabaseDescriptor.isDaemonInitialized())
             {
    -            updateSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(update, dynamicUpdateInterval, dynamicUpdateInterval, TimeUnit.MILLISECONDS);
    -            resetSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(reset, dynamicResetInterval, dynamicResetInterval, TimeUnit.MILLISECONDS);
    +            applyConfigChanges();
                 registerMBean();
             }
    +        nextProbeGenerationTime = System.nanoTime();
    +    }
    +
    +    /**
    +     * Records a latency. This MUST be cheap as it is called in the fast path
    +     */
    +    abstract public void receiveTiming(InetAddressAndPort address, long latency, LatencyMeasurementType measurementType);
    +
    +    /**
    +     * Called periodically from {@link DynamicEndpointSnitch#updateScores()}. This method returns a new score map
    +     * that will be assigned to the {@link DynamicEndpointSnitch#scores} map.
    +     * This is generally more expensive than {@link DynamicEndpointSnitch#receiveTiming}
    +     */
    +    abstract public Map<InetAddressAndPort, Double> calculateScores();
    +
    +    /**
    +     * Signals that we actually tried to rank vs this host. This way any latency probing can just focus
    +     * on hosts that this peer would potentially talk to.
    +     * @param address
    +     */
    +    abstract protected void markRequested(InetAddressAndPort address);
    --- End diff --
    
    I don't get why mark requested is in the derived class? Seems like this is a property of the probing approach that is common to the different implementations.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r239897722
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -21,133 +21,505 @@
     import java.net.InetAddress;
     import java.net.UnknownHostException;
     import java.util.*;
    +import java.util.concurrent.CancellationException;
     import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ExecutionException;
     import java.util.concurrent.ScheduledFuture;
     import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
    +import java.util.concurrent.atomic.AtomicLong;
     import java.util.stream.Collectors;
     
    -import com.codahale.metrics.ExponentiallyDecayingReservoir;
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.util.concurrent.Futures;
    +import com.google.common.util.concurrent.MoreExecutors;
    +import com.google.common.util.concurrent.RateLimiter;
    +import com.google.common.util.concurrent.SettableFuture;
     
    -import com.codahale.metrics.Snapshot;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
     import org.apache.cassandra.concurrent.ScheduledExecutors;
     import org.apache.cassandra.config.DatabaseDescriptor;
    +import org.apache.cassandra.exceptions.RequestFailureReason;
     import org.apache.cassandra.gms.ApplicationState;
     import org.apache.cassandra.gms.EndpointState;
     import org.apache.cassandra.gms.Gossiper;
     import org.apache.cassandra.gms.VersionedValue;
    +import org.apache.cassandra.locator.dynamicsnitch.DynamicEndpointSnitchHistogram;
    +import org.apache.cassandra.net.IAsyncCallbackWithFailure;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +import org.apache.cassandra.net.MessageIn;
    +import org.apache.cassandra.net.MessageOut;
     import org.apache.cassandra.net.MessagingService;
    +import org.apache.cassandra.net.PingMessage;
     import org.apache.cassandra.service.StorageService;
     import org.apache.cassandra.utils.FBUtilities;
     import org.apache.cassandra.utils.MBeanWrapper;
     
    +import static org.apache.cassandra.net.MessagingService.Verb.PING;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.LARGE_MESSAGE;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.SMALL_MESSAGE;
    +
    +
     /**
      * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector
    + * Note that the subclasses (e.g. {@link DynamicEndpointSnitchHistogram}) are responsible for actually measuring
    + * latency and providing an ISnitchMeasurement implementation back to this class.
      */
    -public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
    +public abstract class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
     {
    -    private static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    private static final Logger logger = LoggerFactory.getLogger(DynamicEndpointSnitch.class);
    +
    +    // Latency measurement and ranking. The samples contain latency measurements and the scores are used for ranking
    +    protected boolean registered = false;
    --- End diff --
    
    Should probably have a more specific similar to how mbeanRegistered specifies what is registered.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r229486558
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -154,31 +326,203 @@ private void registerMBean()
     
         public void close()
         {
    -        updateSchedular.cancel(false);
    -        resetSchedular.cancel(false);
    +        if (updateScoresScheduler != null)
    +            updateScoresScheduler.cancel(false);
    +        if (updateSamplesScheduler != null)
    +            updateSamplesScheduler.cancel(false);
    +
    +        for (AnnotatedMeasurement measurement : samples.values())
    +        {
    +            if (measurement.probeFuture != null)
    +                measurement.probeFuture.cancel(false);
    +
    +            measurement.millisSinceLastMeasure.set(0);
    +            measurement.millisSinceLastRequest.set(MAX_PROBE_INTERVAL_MS);
    +            measurement.probeTimerMillis = 0;
    +        }
     
             MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
             try
             {
    -            mbs.unregisterMBean(new ObjectName(mbeanName));
    +            if (mbeanRegistered)
    +                mbs.unregisterMBean(new ObjectName(mbeanName));
             }
             catch (Exception e)
             {
                 throw new RuntimeException(e);
             }
         }
     
    +    /**
    +     * Background task running on the samples dictionary. The default implementation sends latency probes (PING)
    +     * messages to explore nodes that we have not received timings for recently but have ranked in
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}.
    +     */
    +    protected void updateSamples()
    +    {
    +        // Split calculation of probe timers from sending probes for testability
    +        calculateProbes(samples, dynamicLatencyProbeInterval);
    +
    +        if (!StorageService.instance.isGossipActive())
    +            return;
    +
    +        schedulePings(samples);
    +    }
    +
    +    /**
    +     * This method mutates the passed AnnotatedMeasurements to implement capped exponential backoff per endpoint.
    +     *
    +     * The algorithm is as follows:
    +     * 1. All samples get their millisSinceLastMeasure and millisSinceLastRequest fields
    +     *    incremented by the passed interval
    +     * 2. Any recently requested (ranked) endpoints that have not been measured recently (e.g. because the snitch
    +     *    has sent them no traffic) get probes with exponential backoff.
    +     *
    +     * The backoff is capped at MAX_PROBE_INTERVAL_MS. Furthermore the probes are stopped after
    +     * MAX_PROBE_INTERVAL_MS of no ranking requests as well.
    +     *
    +     * At the end of this method, any passed AnnotatedMeasurements that need latency probes will have non zero
    +     * probeTimerMillis members set.
    +     */
    +    @VisibleForTesting
    +    static void calculateProbes(Map<InetAddressAndPort, AnnotatedMeasurement> samples, long intervalMillis) {
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry: samples.entrySet())
    +        {
    +            if (entry.getKey().equals(FBUtilities.getBroadcastAddressAndPort()))
    +                continue;
    +
    +            AnnotatedMeasurement measurement = entry.getValue();
    +            long lastMeasure = measurement.millisSinceLastMeasure.getAndAdd(intervalMillis);
    +            long lastRequest = measurement.millisSinceLastRequest.getAndAdd(intervalMillis);
    +
    +            if (lastMeasure >= MIN_PROBE_INTERVAL_MS && lastRequest < MAX_PROBE_INTERVAL_MS)
    +            {
    +                if (measurement.probeTimerMillis == 0)
    +                {
    +                    measurement.probeTimerMillis = intervalMillis;
    +                }
    +                else if (measurement.probeFuture != null && measurement.probeFuture.isDone())
    +                {
    +                    measurement.probeTimerMillis = Math.min(MAX_PROBE_INTERVAL_MS, measurement.probeTimerMillis * 2);
    +                }
    +            }
    +            else
    +            {
    +                measurement.probeTimerMillis = 0;
    +            }
    +        }
    +    }
    +
    +    @VisibleForTesting
    +    void schedulePings(Map<InetAddressAndPort, AnnotatedMeasurement> samples)
    +    {
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry: samples.entrySet())
    +        {
    +            AnnotatedMeasurement measurement = entry.getValue();
    +            long delay = measurement.probeTimerMillis;
    +            long millisSinceLastRequest = measurement.millisSinceLastRequest.get();
    +
    +            if (millisSinceLastRequest > MAX_PROBE_INTERVAL_MS && !Gossiper.instance.isAlive(entry.getKey()))
    --- End diff --
    
    I think I like the simplicity of where things are right now and I want to take it back. It's still mostly bounded on how far off it can be from the configured value and it won't go too fast. That should be fine?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r229467720
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -154,31 +326,203 @@ private void registerMBean()
     
         public void close()
         {
    -        updateSchedular.cancel(false);
    -        resetSchedular.cancel(false);
    +        if (updateScoresScheduler != null)
    +            updateScoresScheduler.cancel(false);
    +        if (updateSamplesScheduler != null)
    +            updateSamplesScheduler.cancel(false);
    +
    +        for (AnnotatedMeasurement measurement : samples.values())
    +        {
    +            if (measurement.probeFuture != null)
    +                measurement.probeFuture.cancel(false);
    +
    +            measurement.millisSinceLastMeasure.set(0);
    +            measurement.millisSinceLastRequest.set(MAX_PROBE_INTERVAL_MS);
    +            measurement.probeTimerMillis = 0;
    +        }
     
             MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
             try
             {
    -            mbs.unregisterMBean(new ObjectName(mbeanName));
    +            if (mbeanRegistered)
    +                mbs.unregisterMBean(new ObjectName(mbeanName));
             }
             catch (Exception e)
             {
                 throw new RuntimeException(e);
             }
         }
     
    +    /**
    +     * Background task running on the samples dictionary. The default implementation sends latency probes (PING)
    +     * messages to explore nodes that we have not received timings for recently but have ranked in
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}.
    +     */
    +    protected void updateSamples()
    +    {
    +        // Split calculation of probe timers from sending probes for testability
    +        calculateProbes(samples, dynamicLatencyProbeInterval);
    +
    +        if (!StorageService.instance.isGossipActive())
    +            return;
    +
    +        schedulePings(samples);
    +    }
    +
    +    /**
    +     * This method mutates the passed AnnotatedMeasurements to implement capped exponential backoff per endpoint.
    +     *
    +     * The algorithm is as follows:
    +     * 1. All samples get their millisSinceLastMeasure and millisSinceLastRequest fields
    +     *    incremented by the passed interval
    +     * 2. Any recently requested (ranked) endpoints that have not been measured recently (e.g. because the snitch
    +     *    has sent them no traffic) get probes with exponential backoff.
    +     *
    +     * The backoff is capped at MAX_PROBE_INTERVAL_MS. Furthermore the probes are stopped after
    +     * MAX_PROBE_INTERVAL_MS of no ranking requests as well.
    +     *
    +     * At the end of this method, any passed AnnotatedMeasurements that need latency probes will have non zero
    +     * probeTimerMillis members set.
    +     */
    +    @VisibleForTesting
    +    static void calculateProbes(Map<InetAddressAndPort, AnnotatedMeasurement> samples, long intervalMillis) {
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry: samples.entrySet())
    +        {
    +            if (entry.getKey().equals(FBUtilities.getBroadcastAddressAndPort()))
    +                continue;
    +
    +            AnnotatedMeasurement measurement = entry.getValue();
    +            long lastMeasure = measurement.millisSinceLastMeasure.getAndAdd(intervalMillis);
    +            long lastRequest = measurement.millisSinceLastRequest.getAndAdd(intervalMillis);
    +
    +            if (lastMeasure >= MIN_PROBE_INTERVAL_MS && lastRequest < MAX_PROBE_INTERVAL_MS)
    +            {
    +                if (measurement.probeTimerMillis == 0)
    +                {
    +                    measurement.probeTimerMillis = intervalMillis;
    +                }
    +                else if (measurement.probeFuture != null && measurement.probeFuture.isDone())
    +                {
    +                    measurement.probeTimerMillis = Math.min(MAX_PROBE_INTERVAL_MS, measurement.probeTimerMillis * 2);
    +                }
    +            }
    +            else
    +            {
    +                measurement.probeTimerMillis = 0;
    +            }
    +        }
    +    }
    +
    +    @VisibleForTesting
    +    void schedulePings(Map<InetAddressAndPort, AnnotatedMeasurement> samples)
    +    {
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry: samples.entrySet())
    +        {
    +            AnnotatedMeasurement measurement = entry.getValue();
    +            long delay = measurement.probeTimerMillis;
    +            long millisSinceLastRequest = measurement.millisSinceLastRequest.get();
    +
    +            if (millisSinceLastRequest > MAX_PROBE_INTERVAL_MS && !Gossiper.instance.isAlive(entry.getKey()))
    +            {
    +                samples.remove(entry.getKey());
    +            }
    +
    +            if (delay > 0 && millisSinceLastRequest < MAX_PROBE_INTERVAL_MS &&
    +                (measurement.probeFuture == null || measurement.probeFuture.isDone()) &&
    +                !updateSamplesScheduler.isCancelled())
    --- End diff --
    
    The cancelled check can go? It's already pretty racy and redundant.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r239900862
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -21,133 +21,505 @@
     import java.net.InetAddress;
     import java.net.UnknownHostException;
     import java.util.*;
    +import java.util.concurrent.CancellationException;
     import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ExecutionException;
     import java.util.concurrent.ScheduledFuture;
     import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
    +import java.util.concurrent.atomic.AtomicLong;
     import java.util.stream.Collectors;
     
    -import com.codahale.metrics.ExponentiallyDecayingReservoir;
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.util.concurrent.Futures;
    +import com.google.common.util.concurrent.MoreExecutors;
    +import com.google.common.util.concurrent.RateLimiter;
    +import com.google.common.util.concurrent.SettableFuture;
     
    -import com.codahale.metrics.Snapshot;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
     import org.apache.cassandra.concurrent.ScheduledExecutors;
     import org.apache.cassandra.config.DatabaseDescriptor;
    +import org.apache.cassandra.exceptions.RequestFailureReason;
     import org.apache.cassandra.gms.ApplicationState;
     import org.apache.cassandra.gms.EndpointState;
     import org.apache.cassandra.gms.Gossiper;
     import org.apache.cassandra.gms.VersionedValue;
    +import org.apache.cassandra.locator.dynamicsnitch.DynamicEndpointSnitchHistogram;
    +import org.apache.cassandra.net.IAsyncCallbackWithFailure;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +import org.apache.cassandra.net.MessageIn;
    +import org.apache.cassandra.net.MessageOut;
     import org.apache.cassandra.net.MessagingService;
    +import org.apache.cassandra.net.PingMessage;
     import org.apache.cassandra.service.StorageService;
     import org.apache.cassandra.utils.FBUtilities;
     import org.apache.cassandra.utils.MBeanWrapper;
     
    +import static org.apache.cassandra.net.MessagingService.Verb.PING;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.LARGE_MESSAGE;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.SMALL_MESSAGE;
    +
    +
     /**
      * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector
    + * Note that the subclasses (e.g. {@link DynamicEndpointSnitchHistogram}) are responsible for actually measuring
    + * latency and providing an ISnitchMeasurement implementation back to this class.
      */
    -public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
    +public abstract class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
     {
    -    private static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    private static final Logger logger = LoggerFactory.getLogger(DynamicEndpointSnitch.class);
    +
    +    // Latency measurement and ranking. The samples contain latency measurements and the scores are used for ranking
    +    protected boolean registered = false;
    +    protected static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    protected volatile Map<InetAddressAndPort, Double> scores = new HashMap<>();
     
    -    private static final double ALPHA = 0.75; // set to 0.75 to make EDS more biased to towards the newer values
    -    private static final int WINDOW_SIZE = 100;
    +    protected final Map<InetAddressAndPort, AnnotatedMeasurement> samples = new ConcurrentHashMap<>();
     
    -    private volatile int dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    -    private volatile int dynamicResetInterval = DatabaseDescriptor.getDynamicResetInterval();
    -    private volatile double dynamicBadnessThreshold = DatabaseDescriptor.getDynamicBadnessThreshold();
    +    // Latency probe functionality for actively probing endpoints that we haven't measured recently but are ranking
    +    public static final long MAX_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_max_probe_interval_ms", 60 * 10 * 1000L);
    +    public static final long MIN_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_min_probe_interval_ms", 60 * 1000L) ;
    +    // The probe rate is set later when configuration is read in applyConfigChanges
    +    protected static final RateLimiter probeRateLimiter = RateLimiter.create(1);
    +    protected static final DebuggableScheduledThreadPoolExecutor latencyProbeExecutor = new DebuggableScheduledThreadPoolExecutor("LatencyProbes");
    +    private long lastUpdateSamplesNanos = System.nanoTime();
    +
    +    // User configuration of the snitch tunables
    +    protected volatile int dynamicUpdateInterval = -1;
    +    protected volatile int dynamicSampleUpdateInterval = -1;
    +    protected volatile double dynamicBadnessThreshold = 0;
     
         // the score for a merged set of endpoints must be this much worse than the score for separate endpoints to
         // warrant not merging two ranges into a single range
         private static final double RANGE_MERGING_PREFERENCE = 1.5;
     
         private String mbeanName;
    -    private boolean registered = false;
    -
    -    private volatile HashMap<InetAddressAndPort, Double> scores = new HashMap<>();
    -    private final ConcurrentHashMap<InetAddressAndPort, ExponentiallyDecayingReservoir> samples = new ConcurrentHashMap<>();
    +    private boolean mbeanRegistered = false;
     
         public final IEndpointSnitch subsnitch;
     
    -    private volatile ScheduledFuture<?> updateSchedular;
    -    private volatile ScheduledFuture<?> resetSchedular;
    -
    -    private final Runnable update;
    -    private final Runnable reset;
    +    private volatile ScheduledFuture<?> updateScoresScheduler = null;
    +    private volatile ScheduledFuture<?> updateSamplesScheduler = null;
     
         public DynamicEndpointSnitch(IEndpointSnitch snitch)
         {
             this(snitch, null);
         }
     
    -    public DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
    +    protected DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
         {
             mbeanName = "org.apache.cassandra.db:type=DynamicEndpointSnitch";
             if (instance != null)
                 mbeanName += ",instance=" + instance;
             subsnitch = snitch;
    -        update = new Runnable()
    +
    +        if (DatabaseDescriptor.isDaemonInitialized())
             {
    -            public void run()
    +            open();
    +        }
    +    }
    +
    +    /**
    +     * Update configuration of the background tasks and restart the various scheduler tasks
    +     * if the configured rates for these tasks have changed.
    +     */
    +    public void applyConfigChanges(int newDynamicUpdateInternal, int newDynamicSampleUpdateInterval, double newDynamicBadnessThreshold)
    +    {
    +        if (DatabaseDescriptor.isDaemonInitialized())
    +        {
    +            if (dynamicUpdateInterval != newDynamicUpdateInternal || updateScoresScheduler == null)
                 {
    -                updateScores();
    +                cancelAndWait(updateScoresScheduler, Math.max(1, dynamicUpdateInterval), TimeUnit.MILLISECONDS);
    +                updateScoresScheduler = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(this::updateScores, newDynamicUpdateInternal, newDynamicUpdateInternal, TimeUnit.MILLISECONDS);
                 }
    -        };
    -        reset = new Runnable()
    +
    +            if (dynamicSampleUpdateInterval != newDynamicSampleUpdateInterval || updateSamplesScheduler == null)
    +            {
    +                cancelAndWait(updateSamplesScheduler, Math.max(1, dynamicSampleUpdateInterval), TimeUnit.MILLISECONDS);
    +                if (newDynamicSampleUpdateInterval > 0)
    +                    updateSamplesScheduler = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(this::updateSamples, newDynamicSampleUpdateInterval, newDynamicSampleUpdateInterval, TimeUnit.MILLISECONDS);
    +            }
    +        }
    +
    +        dynamicUpdateInterval = newDynamicUpdateInternal;
    +        dynamicSampleUpdateInterval = newDynamicSampleUpdateInterval;
    +        dynamicBadnessThreshold = newDynamicBadnessThreshold;
    +
    +        if (dynamicSampleUpdateInterval > 0)
    +            probeRateLimiter.setRate(dynamicSampleUpdateInterval);
    +    }
    +
    +    public synchronized void open()
    +    {
    +        applyConfigChanges(DatabaseDescriptor.getDynamicUpdateInterval(),
    +                           DatabaseDescriptor.getDynamicSampleUpdateInterval(),
    +                           DatabaseDescriptor.getDynamicBadnessThreshold());
    +
    +        MBeanWrapper.instance.registerMBean(this, mbeanName);
    +        mbeanRegistered = true;
    +    }
    +
    +    public synchronized void close()
    +    {
    +        cancelAndWait(updateScoresScheduler, Math.max(1, dynamicUpdateInterval), TimeUnit.MILLISECONDS);
    +        cancelAndWait(updateSamplesScheduler, Math.max(1, dynamicSampleUpdateInterval), TimeUnit.MILLISECONDS);
    +        updateScoresScheduler = null;
    +        updateSamplesScheduler = null;
    +
    +        for (AnnotatedMeasurement measurement : samples.values())
             {
    -            public void run()
    +            cancelAndWait(measurement.probeFuture, PING.getTimeout(), TimeUnit.MILLISECONDS);
    +
    +            measurement.millisSinceLastMeasure.set(0);
    +            measurement.millisSinceLastRequest.set(MAX_PROBE_INTERVAL_MS);
    +            measurement.nextProbeDelayMillis = 0;
    +        }
    +
    +        if (mbeanRegistered)
    +            MBeanWrapper.instance.unregisterMBean(mbeanName);
    +
    +        mbeanRegistered = false;
    +    }
    +
    +    private static void cancelAndWait(ScheduledFuture future, long timeout, TimeUnit unit)
    +    {
    +        if (future != null)
    +        {
    +            future.cancel(false);
    +            try
                 {
    -                // we do this so that a host considered bad has a chance to recover, otherwise would we never try
    -                // to read from it, which would cause its score to never change
    -                reset();
    +                future.get(timeout, unit);
                 }
    -        };
    +            catch (CancellationException | InterruptedException | ExecutionException | TimeoutException ignored)
    +            {
    +                // Exception is expected to happen eventually due to the cancel -> get
    +            }
    +        }
    +    }
     
    -        if (DatabaseDescriptor.isDaemonInitialized())
    +    /**
    +     * Allows subclasses to inject new ways of measuring latency back to this abstract base class.
    +     */
    +    protected interface ISnitchMeasurement
    +    {
    +        void sample(long value);
    +        double measure();
    +        Iterable<Double> measurements();
    +    }
    +
    +    /**
    +     * Adds some boookeeping that the DES uses over top of the various metrics techniques used by the
    +     * implementations. This is used to allow CASSANDRA-14459 latency probes as well as further safe
    +     * experimentation on new latency measurement techniques in CASSANDRA-14817
    +     *
    +     * {@link AnnotatedMeasurement#millisSinceLastRequest} is set to zero through
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}. It defaults to
    +     * the maximum interval so that we only start probing once it has been ranked at least once
    +     * {@link AnnotatedMeasurement#millisSinceLastMeasure} is set to zero from
    +     * {@link DynamicEndpointSnitch#receiveTiming(InetAddressAndPort, long, LatencyMeasurementType)}.
    +     *
    +     * {@link AnnotatedMeasurement#millisSinceLastMeasure and {@link AnnotatedMeasurement#nextProbeDelayMillis }
    +     * are incremented via {@link DynamicEndpointSnitch#updateSamples()}
    +     */
    +    protected static class AnnotatedMeasurement
    +    {
    +        // Used to optimally target latency probes only on nodes that are both requested for ranking
    +        // and are not being measured. For example with token aware clients a large portion of the cluster will never
    +        // be ranked at all and therefore we won't probe them.
    +        public AtomicLong millisSinceLastRequest = new AtomicLong(MAX_PROBE_INTERVAL_MS);
    +        public AtomicLong millisSinceLastMeasure = new AtomicLong(0);
    +        public volatile long nextProbeDelayMillis = 0;
    +        public volatile ScheduledFuture<?> probeFuture = null;
    +
    +        // The underlying measurement technique. E.g. a median filter (histogram) or an EMA filter, or ...
    +        public final ISnitchMeasurement measurement;
    +        public volatile double cachedMeasurement;
    --- End diff --
    
    Does this need to be volatile? Isn't it all one thread?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225301460
  
    --- Diff: src/java/org/apache/cassandra/net/IAsyncCallback.java ---
    @@ -31,11 +31,23 @@
         void response(MessageIn<T> msg);
     
         /**
    +     * @deprecated Use {@link #latencyMeasurementType()}. This is left for backwards
    +     *             compatibility with existing IAsyncCallback implementations but will
    +     *             be removed at a future date.
          * @return true if this callback is on the read path and its latency should be
          * given as input to the dynamic snitch.
          */
    +    @Deprecated
    --- End diff --
    
    You don't need to deprecate you can just remove it? It's purely internal?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r229459445
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -154,31 +326,203 @@ private void registerMBean()
     
         public void close()
         {
    -        updateSchedular.cancel(false);
    -        resetSchedular.cancel(false);
    +        if (updateScoresScheduler != null)
    +            updateScoresScheduler.cancel(false);
    +        if (updateSamplesScheduler != null)
    +            updateSamplesScheduler.cancel(false);
    +
    +        for (AnnotatedMeasurement measurement : samples.values())
    +        {
    +            if (measurement.probeFuture != null)
    +                measurement.probeFuture.cancel(false);
    +
    +            measurement.millisSinceLastMeasure.set(0);
    +            measurement.millisSinceLastRequest.set(MAX_PROBE_INTERVAL_MS);
    +            measurement.probeTimerMillis = 0;
    +        }
     
             MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
             try
             {
    -            mbs.unregisterMBean(new ObjectName(mbeanName));
    +            if (mbeanRegistered)
    +                mbs.unregisterMBean(new ObjectName(mbeanName));
             }
             catch (Exception e)
             {
                 throw new RuntimeException(e);
             }
         }
     
    +    /**
    +     * Background task running on the samples dictionary. The default implementation sends latency probes (PING)
    +     * messages to explore nodes that we have not received timings for recently but have ranked in
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}.
    +     */
    +    protected void updateSamples()
    +    {
    +        // Split calculation of probe timers from sending probes for testability
    +        calculateProbes(samples, dynamicLatencyProbeInterval);
    +
    +        if (!StorageService.instance.isGossipActive())
    +            return;
    +
    +        schedulePings(samples);
    +    }
    +
    +    /**
    +     * This method mutates the passed AnnotatedMeasurements to implement capped exponential backoff per endpoint.
    +     *
    +     * The algorithm is as follows:
    +     * 1. All samples get their millisSinceLastMeasure and millisSinceLastRequest fields
    +     *    incremented by the passed interval
    +     * 2. Any recently requested (ranked) endpoints that have not been measured recently (e.g. because the snitch
    +     *    has sent them no traffic) get probes with exponential backoff.
    +     *
    +     * The backoff is capped at MAX_PROBE_INTERVAL_MS. Furthermore the probes are stopped after
    +     * MAX_PROBE_INTERVAL_MS of no ranking requests as well.
    +     *
    +     * At the end of this method, any passed AnnotatedMeasurements that need latency probes will have non zero
    +     * probeTimerMillis members set.
    +     */
    +    @VisibleForTesting
    +    static void calculateProbes(Map<InetAddressAndPort, AnnotatedMeasurement> samples, long intervalMillis) {
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry: samples.entrySet())
    +        {
    +            if (entry.getKey().equals(FBUtilities.getBroadcastAddressAndPort()))
    +                continue;
    +
    +            AnnotatedMeasurement measurement = entry.getValue();
    +            long lastMeasure = measurement.millisSinceLastMeasure.getAndAdd(intervalMillis);
    +            long lastRequest = measurement.millisSinceLastRequest.getAndAdd(intervalMillis);
    +
    +            if (lastMeasure >= MIN_PROBE_INTERVAL_MS && lastRequest < MAX_PROBE_INTERVAL_MS)
    +            {
    +                if (measurement.probeTimerMillis == 0)
    +                {
    +                    measurement.probeTimerMillis = intervalMillis;
    +                }
    +                else if (measurement.probeFuture != null && measurement.probeFuture.isDone())
    +                {
    +                    measurement.probeTimerMillis = Math.min(MAX_PROBE_INTERVAL_MS, measurement.probeTimerMillis * 2);
    +                }
    +            }
    +            else
    +            {
    +                measurement.probeTimerMillis = 0;
    +            }
    +        }
    +    }
    +
    +    @VisibleForTesting
    +    void schedulePings(Map<InetAddressAndPort, AnnotatedMeasurement> samples)
    +    {
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry: samples.entrySet())
    +        {
    +            AnnotatedMeasurement measurement = entry.getValue();
    +            long delay = measurement.probeTimerMillis;
    +            long millisSinceLastRequest = measurement.millisSinceLastRequest.get();
    +
    +            if (millisSinceLastRequest > MAX_PROBE_INTERVAL_MS && !Gossiper.instance.isAlive(entry.getKey()))
    +            {
    +                samples.remove(entry.getKey());
    +            }
    +
    +            if (delay > 0 && millisSinceLastRequest < MAX_PROBE_INTERVAL_MS &&
    +                (measurement.probeFuture == null || measurement.probeFuture.isDone()) &&
    +                !updateSamplesScheduler.isCancelled())
    +            {
    +                logger.trace("Scheduled latency probe against {} in {}ms", entry.getKey(), delay);
    +                measurement.probeFuture = latencyProbeExecutor.schedule(() -> sendPingMessageToPeer(entry.getKey()),
    +                                                                        delay, TimeUnit.MILLISECONDS);
    +            }
    +        }
    +    }
    +
    +    /**
    +     * Method that actually sends latency probes as PING messages. This is the only function in this class
    +     * that operates on the latencyProbeExecutor thread and it records the maximum latency between a small and large
    +     * message channel ping.
    +     */
    +    private void sendPingMessageToPeer(InetAddressAndPort to)
    +    {
    +        probeRateLimiter.acquire(dynamicLatencyProbeInterval);
    +
    +        if (!StorageService.instance.isGossipActive())
    --- End diff --
    
    Just curious why this check? Maybe acquire the rate limiter after this check?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r229488760
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -154,31 +326,203 @@ private void registerMBean()
     
         public void close()
         {
    -        updateSchedular.cancel(false);
    -        resetSchedular.cancel(false);
    +        if (updateScoresScheduler != null)
    +            updateScoresScheduler.cancel(false);
    +        if (updateSamplesScheduler != null)
    +            updateSamplesScheduler.cancel(false);
    +
    +        for (AnnotatedMeasurement measurement : samples.values())
    +        {
    +            if (measurement.probeFuture != null)
    +                measurement.probeFuture.cancel(false);
    +
    +            measurement.millisSinceLastMeasure.set(0);
    +            measurement.millisSinceLastRequest.set(MAX_PROBE_INTERVAL_MS);
    +            measurement.probeTimerMillis = 0;
    +        }
     
             MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
             try
             {
    -            mbs.unregisterMBean(new ObjectName(mbeanName));
    +            if (mbeanRegistered)
    +                mbs.unregisterMBean(new ObjectName(mbeanName));
             }
             catch (Exception e)
             {
                 throw new RuntimeException(e);
             }
         }
     
    +    /**
    +     * Background task running on the samples dictionary. The default implementation sends latency probes (PING)
    +     * messages to explore nodes that we have not received timings for recently but have ranked in
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}.
    +     */
    +    protected void updateSamples()
    +    {
    +        // Split calculation of probe timers from sending probes for testability
    +        calculateProbes(samples, dynamicLatencyProbeInterval);
    +
    +        if (!StorageService.instance.isGossipActive())
    +            return;
    +
    +        schedulePings(samples);
    +    }
    +
    +    /**
    +     * This method mutates the passed AnnotatedMeasurements to implement capped exponential backoff per endpoint.
    +     *
    +     * The algorithm is as follows:
    +     * 1. All samples get their millisSinceLastMeasure and millisSinceLastRequest fields
    +     *    incremented by the passed interval
    +     * 2. Any recently requested (ranked) endpoints that have not been measured recently (e.g. because the snitch
    +     *    has sent them no traffic) get probes with exponential backoff.
    +     *
    +     * The backoff is capped at MAX_PROBE_INTERVAL_MS. Furthermore the probes are stopped after
    +     * MAX_PROBE_INTERVAL_MS of no ranking requests as well.
    +     *
    +     * At the end of this method, any passed AnnotatedMeasurements that need latency probes will have non zero
    +     * probeTimerMillis members set.
    +     */
    +    @VisibleForTesting
    +    static void calculateProbes(Map<InetAddressAndPort, AnnotatedMeasurement> samples, long intervalMillis) {
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry: samples.entrySet())
    +        {
    +            if (entry.getKey().equals(FBUtilities.getBroadcastAddressAndPort()))
    +                continue;
    +
    +            AnnotatedMeasurement measurement = entry.getValue();
    +            long lastMeasure = measurement.millisSinceLastMeasure.getAndAdd(intervalMillis);
    --- End diff --
    
    Sorry that is a silly request. We hard rate limit it so there isn't much of a reason to monitor for how often we do it?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r239884316
  
    --- Diff: src/java/org/apache/cassandra/net/MessagingService.java ---
    @@ -729,20 +730,26 @@ void markTimeout(InetAddressAndPort addr)
         /**
          * Track latency information for the dynamic snitch
          *
    -     * @param cb      the callback associated with this message -- this lets us know if it's a message type we're interested in
    -     * @param address the host that replied to the message
    -     * @param latency
    +     * @param cb            the callback associated with this message -- this lets us know if it's a message type we're interested in
    +     * @param address       the host that replied to the message
    +     * @param latencyMicros the number of microseconds to record for this host
          */
    -    public void maybeAddLatency(IAsyncCallback cb, InetAddressAndPort address, long latency)
    +    public void maybeAddLatency(IAsyncCallback cb, InetAddressAndPort address, long latencyMicros)
         {
    -        if (cb.isLatencyForSnitch())
    -            addLatency(address, latency);
    +        if (cb.latencyMeasurementType() != LatencyMeasurementType.IGNORE)
    +            addLatency(address, latencyMicros, cb.latencyMeasurementType());
         }
     
    -    public void addLatency(InetAddressAndPort address, long latency)
    +    // Used on the local read path
    +    public void addLatency(InetAddressAndPort address, long latencyMicros)
    +    {
    +        addLatency(address, latencyMicros, LatencyMeasurementType.READ);
    +    }
    +
    +    private void addLatency(InetAddressAndPort address, long latencyMicros, LatencyMeasurementType usable)
    --- End diff --
    
    Is it usable or is it a type? It's not a boolean anymore.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r232436157
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -23,128 +23,300 @@
     import java.net.UnknownHostException;
     import java.util.*;
     import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ExecutionException;
     import java.util.concurrent.ScheduledFuture;
     import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
     import java.util.stream.Collectors;
     
    -import com.codahale.metrics.ExponentiallyDecayingReservoir;
     import javax.management.MBeanServer;
     import javax.management.ObjectName;
     
    -import com.codahale.metrics.Snapshot;
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.util.concurrent.Futures;
    +import com.google.common.util.concurrent.MoreExecutors;
    +import com.google.common.util.concurrent.RateLimiter;
    +import com.google.common.util.concurrent.SettableFuture;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
     import org.apache.cassandra.concurrent.ScheduledExecutors;
     import org.apache.cassandra.config.DatabaseDescriptor;
    +import org.apache.cassandra.exceptions.RequestFailureReason;
     import org.apache.cassandra.gms.ApplicationState;
     import org.apache.cassandra.gms.EndpointState;
     import org.apache.cassandra.gms.Gossiper;
     import org.apache.cassandra.gms.VersionedValue;
    +import org.apache.cassandra.locator.dynamicsnitch.DynamicEndpointSnitchHistogram;
    +import org.apache.cassandra.net.IAsyncCallbackWithFailure;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +import org.apache.cassandra.net.MessageIn;
    +import org.apache.cassandra.net.MessageOut;
     import org.apache.cassandra.net.MessagingService;
    +import org.apache.cassandra.net.PingMessage;
     import org.apache.cassandra.service.StorageService;
     import org.apache.cassandra.utils.FBUtilities;
     
    +import static org.apache.cassandra.net.MessagingService.Verb.PING;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.LARGE_MESSAGE;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.SMALL_MESSAGE;
    +
    +
     /**
      * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector
    + * Note that the subclasses (e.g. {@link DynamicEndpointSnitchHistogram}) are responsible for actually measuring
    + * latency and providing an ISnitchMeasurement implementation back to this class.
      */
    -public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
    +public abstract class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
     {
    -    private static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    -
    -    private static final double ALPHA = 0.75; // set to 0.75 to make EDS more biased to towards the newer values
    -    private static final int WINDOW_SIZE = 100;
    -
    -    private volatile int dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    -    private volatile int dynamicResetInterval = DatabaseDescriptor.getDynamicResetInterval();
    -    private volatile double dynamicBadnessThreshold = DatabaseDescriptor.getDynamicBadnessThreshold();
    +    private static final Logger logger = LoggerFactory.getLogger(DynamicEndpointSnitch.class);
    +
    +    // Latency measurement and ranking. The samples contain latency measurements and the scores are used for ranking
    +    protected boolean registered = false;
    +    protected static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    protected volatile Map<InetAddressAndPort, Double> scores = new HashMap<>();
    +    protected final Map<InetAddressAndPort, AnnotatedMeasurement> samples = new ConcurrentHashMap<>();
    +
    +    // Latency probe functionality for actively probing endpoints that we haven't measured recently but are ranking
    +    public static final long MAX_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_max_probe_interval_ms", 60 * 10 * 1000L);
    +    public static final long MIN_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_min_probe_interval_ms", 60 * 1000L) ;
    +    // The probe rate is set later when configuration is read
    +    protected static final RateLimiter probeRateLimiter = RateLimiter.create(1);
    +    protected static final DebuggableScheduledThreadPoolExecutor latencyProbeExecutor = new DebuggableScheduledThreadPoolExecutor(1, "LatencyProbes", Thread.MIN_PRIORITY);
    +
    +    // User configuration of the snitch tunables
    +    protected volatile int dynamicUpdateInterval = -1;
    +    protected volatile int dynamicLatencyProbeInterval = -1;
    +    protected volatile double dynamicBadnessThreshold = 0;
     
         // the score for a merged set of endpoints must be this much worse than the score for separate endpoints to
         // warrant not merging two ranges into a single range
         private static final double RANGE_MERGING_PREFERENCE = 1.5;
     
         private String mbeanName;
    -    private boolean registered = false;
    -
    -    private volatile HashMap<InetAddressAndPort, Double> scores = new HashMap<>();
    -    private final ConcurrentHashMap<InetAddressAndPort, ExponentiallyDecayingReservoir> samples = new ConcurrentHashMap<>();
    +    private boolean mbeanRegistered = false;
     
         public final IEndpointSnitch subsnitch;
     
    -    private volatile ScheduledFuture<?> updateSchedular;
    -    private volatile ScheduledFuture<?> resetSchedular;
    -
    -    private final Runnable update;
    -    private final Runnable reset;
    +    private volatile ScheduledFuture<?> updateScoresScheduler;
    +    private volatile ScheduledFuture<?> updateSamplesScheduler;
     
         public DynamicEndpointSnitch(IEndpointSnitch snitch)
         {
             this(snitch, null);
         }
     
    -    public DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
    +    protected DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
         {
             mbeanName = "org.apache.cassandra.db:type=DynamicEndpointSnitch";
             if (instance != null)
                 mbeanName += ",instance=" + instance;
             subsnitch = snitch;
    -        update = new Runnable()
    +
    +        if (DatabaseDescriptor.isDaemonInitialized())
             {
    -            public void run()
    -            {
    -                updateScores();
    -            }
    -        };
    -        reset = new Runnable()
    +            applyConfigChanges(DatabaseDescriptor.getDynamicUpdateInterval(),
    +                               DatabaseDescriptor.getDynamicSampleUpdateInterval(),
    +                               DatabaseDescriptor.getDynamicBadnessThreshold());
    +            open();
    +        }
    +    }
    +
    +    /**
    +     * Allows subclasses to inject new ways of measuring latency back to this abstract base class.
    +     */
    +    protected interface ISnitchMeasurement
    +    {
    +        void sample(long value);
    +        double measure();
    +        Iterable<Double> measurements();
    +    }
    +
    +    /**
    +     * Adds some boookeeping that the DES uses over top of the various metrics techniques used by the
    +     * implementations. This is used to allow CASSANDRA-14459 latency probes as well as further experimentation
    +     * on new latency measurement techniques in CASSANDRA-14817
    +     *
    +     * {@link AnnotatedMeasurement#millisSinceLastRequest} is set to zero through
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}. It defaults to
    +     * the maximum interval so that we only start probing once it has been ranked at least once
    +     * {@link AnnotatedMeasurement#millisSinceLastMeasure} is set to zero from
    +     * {@link DynamicEndpointSnitch#receiveTiming(InetAddressAndPort, long, LatencyMeasurementType)}.
    +     *
    +     * {@link AnnotatedMeasurement#millisSinceLastMeasure and {@link AnnotatedMeasurement#probeTimerMillis}
    +     * are incremented via {@link DynamicEndpointSnitch#updateSamples()}
    +     */
    +    protected static class AnnotatedMeasurement
    +    {
    +        // Used to optimally target latency probes only on nodes that are both requested for ranking
    +        // and are not being measured. For example with token aware clients a large portion of the cluster will never
    +        // be ranked at all and therefore we won't probe them.
    +        public AtomicLong millisSinceLastRequest = new AtomicLong(MAX_PROBE_INTERVAL_MS);
    +        public AtomicLong millisSinceLastMeasure = new AtomicLong(0);
    +        public long probeTimerMillis = 0;
    +        public ScheduledFuture<?> probeFuture = null;
    +
    +        // The underlying measurement technique. E.g. a median filter (histogram) or an EMA filter, or ...
    +        public final ISnitchMeasurement measurement;
    +
    +        public AnnotatedMeasurement(ISnitchMeasurement measurement)
    +        {
    +            this.measurement = measurement;
    +        }
    +
    +        @Override
    +        public String toString()
    +        {
    +            return "AnnotatedMeasurement{" +
    +                   "millisSinceLastRequest=" + millisSinceLastRequest +
    +                   ", millisSinceLastMeasure=" + millisSinceLastMeasure +
    +                   ", probeTimerMillis=" + probeTimerMillis +
    +                   ", probeFuturePending=" + (probeFuture != null && !probeFuture.isDone()) +
    +                   ", measurementClass=" + measurement.getClass().getSimpleName() +
    +                   '}';
    +        }
    +    }
    +
    +    /**
    +     * Allows the subclasses to inject ISnitchMeasurement instances back into this common class so that common
    +     * functionality can be handled here
    +     * @param initialValue The initial value of the measurement, some implementations may use this, others may not
    +     * @return a constructed instance of an ISnitchMeasurement interface
    +     */
    +    abstract protected ISnitchMeasurement measurementImpl(long initialValue);
    +
    +    /**
    +     * Records a latency. This MUST be cheap as it is called in the fast path
    +     */
    +    public void receiveTiming(InetAddressAndPort address, long latency, LatencyMeasurementType measurementType)
    +    {
    +        if (measurementType == LatencyMeasurementType.IGNORE)
    +           return;
    +
    +        AnnotatedMeasurement sample = samples.get(address);
    +
    +        if (sample == null)
    +        {
    +            AnnotatedMeasurement maybeNewSample = new AnnotatedMeasurement(measurementImpl(latency));
    +            sample = samples.putIfAbsent(address, maybeNewSample);
    +            if (sample == null)
    +                sample = maybeNewSample;
    +        }
    +
    +        if (measurementType == LatencyMeasurementType.READ && sample.millisSinceLastMeasure.get() > 0)
    +            sample.millisSinceLastMeasure.lazySet(0);
    +
    +        sample.measurement.sample(latency);
    +    }
    +
    +    @VisibleForTesting
    +    protected void reset()
    +    {
    +        scores.clear();
    +        samples.clear();
    +    }
    +
    +    @VisibleForTesting
    +    void updateScores()
    +    {
    +        if (!StorageService.instance.isGossipActive())
    +            return;
    +
    +        if (!registered)
             {
    -            public void run()
    +            if (MessagingService.instance() != null)
                 {
    -                // we do this so that a host considered bad has a chance to recover, otherwise would we never try
    -                // to read from it, which would cause its score to never change
    -                reset();
    +                MessagingService.instance().register(this);
    +                registered = true;
                 }
    -        };
    +        }
     
    -        if (DatabaseDescriptor.isDaemonInitialized())
    +        this.scores = calculateScores();
    +    }
    +
    +    /**
    +     * This is generally expensive and is called periodically (semi-frequently) not on the fast path.
    +     * @return a freshly constructed scores map.
    +     */
    +    public Map<InetAddressAndPort, Double> calculateScores()
    +    {
    +        double maxLatency = 1;
    +
    +        Map<InetAddressAndPort, Double> measurements = new HashMap<>(samples.size());
    +
    +        // We're going to weight the latency for each host against the worst one we see, to
    +        // arrive at sort of a 'badness percentage' for them. First, find the worst for each:
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry : samples.entrySet())
             {
    -            updateSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(update, dynamicUpdateInterval, dynamicUpdateInterval, TimeUnit.MILLISECONDS);
    -            resetSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(reset, dynamicResetInterval, dynamicResetInterval, TimeUnit.MILLISECONDS);
    -            registerMBean();
    +            // This is expensive for e.g. the Histogram, so do it once and cache the result
    +            double measure = entry.getValue().measurement.measure();
    +            if (measure > maxLatency)
    +                maxLatency = measure;
    +            measurements.put(entry.getKey(), measure);
             }
    +
    +        HashMap<InetAddressAndPort, Double> newScores = new HashMap<>(measurements.size());
    +
    +        // now make another pass to do the weighting based on the maximums we found before
    +        for (Map.Entry<InetAddressAndPort, Double> entry : measurements.entrySet())
    +        {
    +            double score = entry.getValue() / maxLatency;
    +            // finally, add the severity without any weighting, since hosts scale this relative to their own load and the size of the task causing the severity.
    +            // "Severity" is basically a measure of compaction activity (CASSANDRA-3722).
    +            if (USE_SEVERITY)
    +                score += getSeverity(entry.getKey());
    +            // lowest score (least amount of badness) wins.
    +            newScores.put(entry.getKey(), score);
    +        }
    +
    +        return newScores;
         }
     
         /**
    -     * Update configuration from {@link DatabaseDescriptor} and estart the update-scheduler and reset-scheduler tasks
    +     * Update configuration from {@link DatabaseDescriptor} and restart the various scheduler tasks
          * if the configured rates for these tasks have changed.
          */
    -    public void applyConfigChanges()
    +    public void applyConfigChanges(int newDynamicUpdateInternal, int newDynamicLatencyProbeInterval, double newDynamicBadnessThreshold)
         {
    -        if (dynamicUpdateInterval != DatabaseDescriptor.getDynamicUpdateInterval())
    +        if (dynamicUpdateInterval != newDynamicUpdateInternal)
             {
    -            dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    +            dynamicUpdateInterval = newDynamicUpdateInternal;
                 if (DatabaseDescriptor.isDaemonInitialized())
                 {
    -                updateSchedular.cancel(false);
    -                updateSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(update, dynamicUpdateInterval, dynamicUpdateInterval, TimeUnit.MILLISECONDS);
    +                if (updateScoresScheduler != null)
    +                    updateScoresScheduler.cancel(false);
    +                updateScoresScheduler = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(this::updateScores, dynamicUpdateInterval, dynamicUpdateInterval, TimeUnit.MILLISECONDS);
    --- End diff --
    
    Yea ... I can move it onto a dedicated STPE but I don't want to change too much behavior. Regarding the naming the LegacyHistogram doesn't send any probes, so I thought it could just be generically a `updateSamples` method similar to `updateScores` ... I don't care strongly one way or the other it's up to you :-)


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r232435703
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -23,128 +23,300 @@
     import java.net.UnknownHostException;
     import java.util.*;
     import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ExecutionException;
     import java.util.concurrent.ScheduledFuture;
     import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
     import java.util.stream.Collectors;
     
    -import com.codahale.metrics.ExponentiallyDecayingReservoir;
     import javax.management.MBeanServer;
     import javax.management.ObjectName;
     
    -import com.codahale.metrics.Snapshot;
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.util.concurrent.Futures;
    +import com.google.common.util.concurrent.MoreExecutors;
    +import com.google.common.util.concurrent.RateLimiter;
    +import com.google.common.util.concurrent.SettableFuture;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
     import org.apache.cassandra.concurrent.ScheduledExecutors;
     import org.apache.cassandra.config.DatabaseDescriptor;
    +import org.apache.cassandra.exceptions.RequestFailureReason;
     import org.apache.cassandra.gms.ApplicationState;
     import org.apache.cassandra.gms.EndpointState;
     import org.apache.cassandra.gms.Gossiper;
     import org.apache.cassandra.gms.VersionedValue;
    +import org.apache.cassandra.locator.dynamicsnitch.DynamicEndpointSnitchHistogram;
    +import org.apache.cassandra.net.IAsyncCallbackWithFailure;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +import org.apache.cassandra.net.MessageIn;
    +import org.apache.cassandra.net.MessageOut;
     import org.apache.cassandra.net.MessagingService;
    +import org.apache.cassandra.net.PingMessage;
     import org.apache.cassandra.service.StorageService;
     import org.apache.cassandra.utils.FBUtilities;
     
    +import static org.apache.cassandra.net.MessagingService.Verb.PING;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.LARGE_MESSAGE;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.SMALL_MESSAGE;
    +
    +
     /**
      * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector
    + * Note that the subclasses (e.g. {@link DynamicEndpointSnitchHistogram}) are responsible for actually measuring
    + * latency and providing an ISnitchMeasurement implementation back to this class.
      */
    -public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
    +public abstract class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
     {
    -    private static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    -
    -    private static final double ALPHA = 0.75; // set to 0.75 to make EDS more biased to towards the newer values
    -    private static final int WINDOW_SIZE = 100;
    -
    -    private volatile int dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    -    private volatile int dynamicResetInterval = DatabaseDescriptor.getDynamicResetInterval();
    -    private volatile double dynamicBadnessThreshold = DatabaseDescriptor.getDynamicBadnessThreshold();
    +    private static final Logger logger = LoggerFactory.getLogger(DynamicEndpointSnitch.class);
    +
    +    // Latency measurement and ranking. The samples contain latency measurements and the scores are used for ranking
    +    protected boolean registered = false;
    +    protected static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    protected volatile Map<InetAddressAndPort, Double> scores = new HashMap<>();
    +    protected final Map<InetAddressAndPort, AnnotatedMeasurement> samples = new ConcurrentHashMap<>();
    +
    +    // Latency probe functionality for actively probing endpoints that we haven't measured recently but are ranking
    +    public static final long MAX_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_max_probe_interval_ms", 60 * 10 * 1000L);
    +    public static final long MIN_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_min_probe_interval_ms", 60 * 1000L) ;
    +    // The probe rate is set later when configuration is read
    +    protected static final RateLimiter probeRateLimiter = RateLimiter.create(1);
    +    protected static final DebuggableScheduledThreadPoolExecutor latencyProbeExecutor = new DebuggableScheduledThreadPoolExecutor(1, "LatencyProbes", Thread.MIN_PRIORITY);
    +
    +    // User configuration of the snitch tunables
    +    protected volatile int dynamicUpdateInterval = -1;
    +    protected volatile int dynamicLatencyProbeInterval = -1;
    +    protected volatile double dynamicBadnessThreshold = 0;
     
         // the score for a merged set of endpoints must be this much worse than the score for separate endpoints to
         // warrant not merging two ranges into a single range
         private static final double RANGE_MERGING_PREFERENCE = 1.5;
     
         private String mbeanName;
    -    private boolean registered = false;
    -
    -    private volatile HashMap<InetAddressAndPort, Double> scores = new HashMap<>();
    -    private final ConcurrentHashMap<InetAddressAndPort, ExponentiallyDecayingReservoir> samples = new ConcurrentHashMap<>();
    +    private boolean mbeanRegistered = false;
     
         public final IEndpointSnitch subsnitch;
     
    -    private volatile ScheduledFuture<?> updateSchedular;
    -    private volatile ScheduledFuture<?> resetSchedular;
    -
    -    private final Runnable update;
    -    private final Runnable reset;
    +    private volatile ScheduledFuture<?> updateScoresScheduler;
    +    private volatile ScheduledFuture<?> updateSamplesScheduler;
     
         public DynamicEndpointSnitch(IEndpointSnitch snitch)
         {
             this(snitch, null);
         }
     
    -    public DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
    +    protected DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
         {
             mbeanName = "org.apache.cassandra.db:type=DynamicEndpointSnitch";
             if (instance != null)
                 mbeanName += ",instance=" + instance;
             subsnitch = snitch;
    -        update = new Runnable()
    +
    +        if (DatabaseDescriptor.isDaemonInitialized())
             {
    -            public void run()
    -            {
    -                updateScores();
    -            }
    -        };
    -        reset = new Runnable()
    +            applyConfigChanges(DatabaseDescriptor.getDynamicUpdateInterval(),
    +                               DatabaseDescriptor.getDynamicSampleUpdateInterval(),
    +                               DatabaseDescriptor.getDynamicBadnessThreshold());
    +            open();
    +        }
    +    }
    +
    +    /**
    +     * Allows subclasses to inject new ways of measuring latency back to this abstract base class.
    +     */
    +    protected interface ISnitchMeasurement
    +    {
    +        void sample(long value);
    +        double measure();
    +        Iterable<Double> measurements();
    +    }
    +
    +    /**
    +     * Adds some boookeeping that the DES uses over top of the various metrics techniques used by the
    +     * implementations. This is used to allow CASSANDRA-14459 latency probes as well as further experimentation
    +     * on new latency measurement techniques in CASSANDRA-14817
    +     *
    +     * {@link AnnotatedMeasurement#millisSinceLastRequest} is set to zero through
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}. It defaults to
    +     * the maximum interval so that we only start probing once it has been ranked at least once
    +     * {@link AnnotatedMeasurement#millisSinceLastMeasure} is set to zero from
    +     * {@link DynamicEndpointSnitch#receiveTiming(InetAddressAndPort, long, LatencyMeasurementType)}.
    +     *
    +     * {@link AnnotatedMeasurement#millisSinceLastMeasure and {@link AnnotatedMeasurement#probeTimerMillis}
    +     * are incremented via {@link DynamicEndpointSnitch#updateSamples()}
    +     */
    +    protected static class AnnotatedMeasurement
    +    {
    +        // Used to optimally target latency probes only on nodes that are both requested for ranking
    +        // and are not being measured. For example with token aware clients a large portion of the cluster will never
    +        // be ranked at all and therefore we won't probe them.
    +        public AtomicLong millisSinceLastRequest = new AtomicLong(MAX_PROBE_INTERVAL_MS);
    +        public AtomicLong millisSinceLastMeasure = new AtomicLong(0);
    +        public long probeTimerMillis = 0;
    --- End diff --
    
    Done.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r229505922
  
    --- Diff: test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java ---
    @@ -60,57 +130,229 @@ private static EndpointsForRange full(InetAddressAndPort... endpoints)
         }
     
         @Test
    -    public void testSnitch() throws InterruptedException, IOException, ConfigurationException
    +    public void testSortedByProximity() throws InterruptedException, IOException, ConfigurationException
         {
    -        // do this because SS needs to be initialized before DES can work properly.
    -        StorageService.instance.unsafeInitialize();
    -        SimpleSnitch ss = new SimpleSnitch();
    -        DynamicEndpointSnitch dsnitch = new DynamicEndpointSnitch(ss, String.valueOf(ss.hashCode()));
    -        InetAddressAndPort self = FBUtilities.getBroadcastAddressAndPort();
    -        InetAddressAndPort host1 = InetAddressAndPort.getByName("127.0.0.2");
    -        InetAddressAndPort host2 = InetAddressAndPort.getByName("127.0.0.3");
    -        InetAddressAndPort host3 = InetAddressAndPort.getByName("127.0.0.4");
    -        InetAddressAndPort host4 = InetAddressAndPort.getByName("127.0.0.5");
    -        List<InetAddressAndPort> hosts = Arrays.asList(host1, host2, host3);
    +        InetAddressAndPort self = hosts[0];
    +        List<InetAddressAndPort> allHosts = Arrays.asList(hosts[1], hosts[2], hosts[3]);
     
             // first, make all hosts equal
    -        setScores(dsnitch, 1, hosts, 10, 10, 10);
    -        EndpointsForRange order = full(host1, host2, host3);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    -
    -        // make host1 a little worse
    -        setScores(dsnitch, 1, hosts, 20, 10, 10);
    -        order = full(host2, host3, host1);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    -
    -        // make host2 as bad as host1
    -        setScores(dsnitch, 2, hosts, 15, 20, 10);
    -        order = full(host3, host1, host2);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    -
    -        // make host3 the worst
    -        setScores(dsnitch, 3, hosts, 10, 10, 30);
    -        order = full(host1, host2, host3);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    -
    -        // make host3 equal to the others
    -        setScores(dsnitch, 5, hosts, 10, 10, 10);
    -        order = full(host1, host2, host3);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    +        setScores(dsnitch, 1, allHosts, 10, 10, 10);
    +        EndpointsForRange order = full(hosts[1], hosts[2], hosts[3]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        // make hosts[1] a little worse
    +        setScores(dsnitch, 2, allHosts, 20, 10, 10);
    +        order = full(hosts[2], hosts[3], hosts[1]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        // make hosts[2] as bad as hosts[1]
    +        setScores(dsnitch, 4, allHosts, 15, 20, 10);
    +        order = full(hosts[3], hosts[1], hosts[2]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        // make hosts[3] the worst
    +        setScores(dsnitch, 10, allHosts, 10, 10, 30);
    +        order = full(hosts[1], hosts[2], hosts[3]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        // make hosts[3] equal to the others
    +        setScores(dsnitch, 15, allHosts, 10, 10, 10);
    +        order = full(hosts[1], hosts[2], hosts[3]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
     
             /// Tests CASSANDRA-6683 improvements
             // make the scores differ enough from the ideal order that we sort by score; under the old
             // dynamic snitch behavior (where we only compared neighbors), these wouldn't get sorted
    -        setScores(dsnitch, 20, hosts, 10, 70, 20);
    -        order = full(host1, host3, host2);
    -        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
    +        dsnitch.reset();
    +        setScores(dsnitch, 20, allHosts, 10, 70, 20);
    +        order = full(hosts[1], hosts[3], hosts[2]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3])));
    +
    +        order = full(hosts[4], hosts[1], hosts[3], hosts[2]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3], hosts[4])));
    +
    +        setScores(dsnitch, 40, allHosts, 10, 10, 11);
    +        order = full(hosts[4], hosts[1], hosts[2], hosts[3]);
    +        Util.assertRCEquals(order, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3], hosts[4])));
    +    }
    +
    +    // CASSANDRA-14459 improvements to add latency probes instead of resets
    +    @Test
    +    public void testLatencyProbeNeeded()
    +    {
    +        InetAddressAndPort self = hosts[0];
    +
    +        // Four hosts, two are very latent, but all are requested for ranking
    +        dsnitch.receiveTiming(hosts[1], 20, LatencyMeasurementType.READ);
    +        dsnitch.receiveTiming(hosts[2], 10, LatencyMeasurementType.READ);
    +        dsnitch.receiveTiming(hosts[3], 1000, LatencyMeasurementType.READ);
    +        dsnitch.receiveTiming(hosts[4], 1000, LatencyMeasurementType.READ);
    +        dsnitch.updateScores();
    +
    +        EndpointsForRange orderBefore = full(hosts[2], hosts[1], hosts[3], hosts[4]);
    +
    +        Map<InetAddressAndPort, DynamicEndpointSnitch.AnnotatedMeasurement> measurements = dsnitch.getMeasurementsWithPort();
    +        // At this point we haven't ranked any of hosts so we should need no probes even
    +        DynamicEndpointSnitch.calculateProbes(measurements, dsnitch.dynamicLatencyProbeInterval);
    +        assertFalse(measurements.values().stream().anyMatch(m -> m.probeTimerMillis > 0));
    +        assertTrue(measurements.values().stream().allMatch(m -> m.probeFuture == null));
    +
    +        // Two hosts continue receiving traffic but the last two are always ranked by the snitch
    +        for (int i = 0; i < 10; i++)
    +        {
    +            dsnitch.receiveTiming(hosts[1], 20, LatencyMeasurementType.READ);
    +            dsnitch.receiveTiming(hosts[2], 10, LatencyMeasurementType.READ);
    +            Util.assertRCEquals(orderBefore, dsnitch.sortedByProximity(self, full(hosts[1], hosts[2], hosts[3], hosts[4])));
    --- End diff --
    
    Why would the sort order change if we haven't updated the scores?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225332967
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -188,6 +343,9 @@ public String getDatacenter(InetAddressAndPort endpoint)
         public <C extends ReplicaCollection<? extends C>> C sortedByProximity(final InetAddressAndPort address, C unsortedAddresses)
         {
             assert address.equals(FBUtilities.getBroadcastAddressAndPort()); // we only know about ourself
    +        for (Replica unsortedAddress : unsortedAddresses)
    --- End diff --
    
    We shouldn't need to fetch each endpoint from the map a second time.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225698605
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -210,8 +368,10 @@ public String getDatacenter(InetAddressAndPort endpoint)
     
             // TODO: avoid copy
             replicas = subsnitch.sortedByProximity(address, replicas);
    -        HashMap<InetAddressAndPort, Double> scores = this.scores; // Make sure the score don't change in the middle of the loop below
    -                                                           // (which wouldn't really matter here but its cleaner that way).
    +        // Make sure the score don't change in the middle of the loop below
    +        // (which wouldn't really matter here but its cleaner that way).
    +        final Map<InetAddressAndPort, Double> scores = this.scores;
    --- End diff --
    
    I asked and yes we do strongly discourage final on the stack and on method parameters.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225329497
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -22,121 +22,194 @@
     import java.net.InetAddress;
     import java.net.UnknownHostException;
     import java.util.*;
    -import java.util.concurrent.ConcurrentHashMap;
     import java.util.concurrent.ScheduledFuture;
     import java.util.concurrent.TimeUnit;
     import java.util.stream.Collectors;
     
    -import com.codahale.metrics.ExponentiallyDecayingReservoir;
     import javax.management.MBeanServer;
     import javax.management.ObjectName;
     
    -import com.codahale.metrics.Snapshot;
    +import com.google.common.annotations.VisibleForTesting;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
     import org.apache.cassandra.concurrent.ScheduledExecutors;
     import org.apache.cassandra.config.DatabaseDescriptor;
     import org.apache.cassandra.gms.ApplicationState;
     import org.apache.cassandra.gms.EndpointState;
     import org.apache.cassandra.gms.Gossiper;
     import org.apache.cassandra.gms.VersionedValue;
    +import org.apache.cassandra.net.IAsyncCallback;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +import org.apache.cassandra.net.MessageIn;
    +import org.apache.cassandra.net.MessageOut;
     import org.apache.cassandra.net.MessagingService;
    +import org.apache.cassandra.net.PingMessage;
     import org.apache.cassandra.service.StorageService;
     import org.apache.cassandra.utils.FBUtilities;
     
    +import static org.apache.cassandra.net.MessagingService.Verb.PING;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.LARGE_MESSAGE;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.SMALL_MESSAGE;
    +
    +
     /**
      * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector
    + * Note that the subclasses (e.g. {@link DynamicEndpointSnitchHistogram}) are responsible for actually measuring
    + * latency and populating the {@link #scores} map.
      */
    -public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
    +public abstract class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
     {
    -    private static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    -
    -    private static final double ALPHA = 0.75; // set to 0.75 to make EDS more biased to towards the newer values
    -    private static final int WINDOW_SIZE = 100;
    -
    -    private volatile int dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    -    private volatile int dynamicResetInterval = DatabaseDescriptor.getDynamicResetInterval();
    -    private volatile double dynamicBadnessThreshold = DatabaseDescriptor.getDynamicBadnessThreshold();
    +    private static final Logger logger = LoggerFactory.getLogger(DynamicEndpointSnitch.class);
    +
    +    // Subclass specific functionality
    +    protected static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    protected boolean registered = false;
    +    // The scores map is updated via copy in updateScores
    +    // We keep it in the base class for performance reasons (so it can be easily aliased)
    +    protected volatile Map<InetAddressAndPort, Double> scores = new HashMap<>();
    +
    +    // Rate limit how often we generate latency probes
    +    protected long nextAllowedProbeGenerationTime;
    +    protected long nextProbeGenerationTime;
    +    protected int currentProbePosition = 0;
    +    protected final List<InetAddressAndPort> latencyProbeSequence = new ArrayList<>();
    +
    +    // DES general functionality
    +    protected volatile int dynamicUpdateInterval = -1;
    +    protected volatile int dynamicLatencyProbeInterval = -1;
    +    protected volatile double dynamicBadnessThreshold = 0;
     
         // the score for a merged set of endpoints must be this much worse than the score for separate endpoints to
         // warrant not merging two ranges into a single range
         private static final double RANGE_MERGING_PREFERENCE = 1.5;
     
         private String mbeanName;
    -    private boolean registered = false;
    -
    -    private volatile HashMap<InetAddressAndPort, Double> scores = new HashMap<>();
    -    private final ConcurrentHashMap<InetAddressAndPort, ExponentiallyDecayingReservoir> samples = new ConcurrentHashMap<>();
    +    private boolean mbeanRegistered = false;
     
         public final IEndpointSnitch subsnitch;
     
    -    private volatile ScheduledFuture<?> updateSchedular;
    -    private volatile ScheduledFuture<?> resetSchedular;
    +    private volatile ScheduledFuture<?> updateScheduler;
    +    private volatile ScheduledFuture<?> latencyProbeScheduler;
     
    -    private final Runnable update;
    -    private final Runnable reset;
    +    private final Runnable update = this::updateScores;
    +    private final Runnable latencyProbe = this::maybeSendLatencyProbe;
     
         public DynamicEndpointSnitch(IEndpointSnitch snitch)
         {
             this(snitch, null);
         }
     
    -    public DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
    +    protected DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
         {
             mbeanName = "org.apache.cassandra.db:type=DynamicEndpointSnitch";
             if (instance != null)
                 mbeanName += ",instance=" + instance;
             subsnitch = snitch;
    -        update = new Runnable()
    -        {
    -            public void run()
    -            {
    -                updateScores();
    -            }
    -        };
    -        reset = new Runnable()
    -        {
    -            public void run()
    -            {
    -                // we do this so that a host considered bad has a chance to recover, otherwise would we never try
    -                // to read from it, which would cause its score to never change
    -                reset();
    -            }
    -        };
     
             if (DatabaseDescriptor.isDaemonInitialized())
             {
    -            updateSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(update, dynamicUpdateInterval, dynamicUpdateInterval, TimeUnit.MILLISECONDS);
    -            resetSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(reset, dynamicResetInterval, dynamicResetInterval, TimeUnit.MILLISECONDS);
    +            applyConfigChanges();
                 registerMBean();
             }
    +        nextProbeGenerationTime = System.nanoTime();
    +    }
    +
    +    /**
    +     * Records a latency. This MUST be cheap as it is called in the fast path
    +     */
    +    abstract public void receiveTiming(InetAddressAndPort address, long latency, LatencyMeasurementType measurementType);
    --- End diff --
    
    This should also be common across derived classes except for the how it stores the latency value in the end?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r227617037
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitchEMA.java ---
    @@ -0,0 +1,181 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.cassandra.locator;
    +
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.concurrent.ConcurrentHashMap;
    +
    +import org.apache.cassandra.gms.Gossiper;
    +import org.apache.cassandra.metrics.ExponentialMovingAverage;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +
    +
    +/**
    + * A dynamic snitching implementation that uses Exponentially Decaying Histograms to prefer or
    + * de-prefer hosts
    + *
    + * This was the default implementation prior to Cassandra 4.0 and is being left as the default
    + * in 4.0
    + */
    +public class DynamicEndpointSnitchEMA extends DynamicEndpointSnitch
    +{
    +    // A ~10 sample EMA
    +    private static final double EMA_ALPHA = 0.10;
    +
    +    private final ConcurrentHashMap<InetAddressAndPort, AnnotatedEMA> samples = new ConcurrentHashMap<>();
    +
    +    /**
    +     * Adds two boolean markers to the ExponentialMovingAverage for telling if the data has been
    +     * updated or requested recently.
    +     *
    +     * recentlyMeasured is updated through {@link AnnotatedEMA#update(long, boolean)}
    +     * recentlyRequested is updated through {@link DynamicEndpointSnitch#markRequested}
    +     *
    +     * Both markers are periodically reset via {@link DynamicEndpointSnitch#latencyProbeNeeded(long)}
    +     */
    +    private static class AnnotatedEMA extends ExponentialMovingAverage
    +    {
    +        volatile boolean recentlyRequested = false;
    +        volatile boolean recentlyMeasured = false;
    +
    +        AnnotatedEMA(double alpha, double initialValue)
    +        {
    +            super(alpha, initialValue);
    +        }
    +
    +        public void update(long value, boolean isRealRead)
    +        {
    +            recentlyMeasured = recentlyMeasured || isRealRead;
    +            super.update(value);
    +        }
    +    }
    +
    +    // Called via reflection
    +    public DynamicEndpointSnitchEMA(IEndpointSnitch snitch)
    +    {
    +        this(snitch, "ema");
    +    }
    +
    +    protected DynamicEndpointSnitchEMA(IEndpointSnitch snitch, String instance)
    +    {
    +        super(snitch, instance);
    +    }
    +
    +    private void receiveTiming(InetAddressAndPort host, long latency, boolean isRealRead) // this is cheap
    +    {
    +        AnnotatedEMA sample = samples.get(host);
    +        if (sample == null)
    +        {
    +            AnnotatedEMA maybeNewSample = new AnnotatedEMA(EMA_ALPHA, latency);
    +            sample = samples.putIfAbsent(host, maybeNewSample);
    +            if (sample == null)
    +                sample = maybeNewSample;
    +        }
    +        sample.update(latency, isRealRead);
    +    }
    +
    +    public void receiveTiming(InetAddressAndPort address, long latency, LatencyMeasurementType measurementType)
    +    {
    +        if (measurementType == LatencyMeasurementType.READ)
    +            receiveTiming(address, latency, true);
    +        else if (measurementType == LatencyMeasurementType.PROBE)
    +            receiveTiming(address, latency, false);
    +    }
    +
    +    @Override
    +    protected void reset() {
    +        this.samples.clear();
    +        super.reset();
    +    }
    +
    +    @Override
    +    public Map<InetAddressAndPort, Double> calculateScores()
    +    {
    +        // We're going to weight the latency for each host against the worst one we see, to
    +        // arrive at sort of a 'badness percentage' for them. First, find the worst for each:
    +        HashMap<InetAddressAndPort, Double> newScores = new HashMap<>(samples.size());
    +        Optional<Double> maxObservedAvgLatency = samples.values().stream()
    +                                                        .map(ExponentialMovingAverage::getAvg)
    +                                                        .max(Double::compare);
    +
    +        final double maxAvgLatency = maxObservedAvgLatency.isPresent() ? maxObservedAvgLatency.get() : 1;
    +
    +        // now make another pass to do the weighting based on the maximums we found before
    +        for (Map.Entry<InetAddressAndPort, AnnotatedEMA> entry : samples.entrySet())
    +        {
    +            // Samples may have changed but rather than creating garbage by copying samples we just ensure
    +            // that all scores are less than 1.0
    +            double addrAvg = entry.getValue().getAvg();
    +            double score = addrAvg / Math.max(addrAvg, maxAvgLatency);
    +            // finally, add the severity without any weighting, since hosts scale this relative to their own load
    +            // and the size of the task causing the severity.
    +            // "Severity" is basically a measure of compaction activity (CASSANDRA-3722).
    +            if (USE_SEVERITY)
    +                score += getSeverity(entry.getKey());
    +            // lowest score (least amount of badness) wins.
    +            newScores.put(entry.getKey(), score);
    +        }
    +        return newScores;
    +    }
    +
    +    @Override
    +    public List<Double> dumpTimings(String hostname) throws UnknownHostException
    +    {
    +        InetAddressAndPort host = InetAddressAndPort.getByName(hostname);
    +        ArrayList<Double> timings = new ArrayList<>();
    +        ExponentialMovingAverage avg = samples.get(host);
    +        if (avg != null)
    +        {
    +            timings.add(avg.getAvg());
    +        }
    +        return timings;
    +    }
    +
    +    @Override
    +    protected void updateLatencyProbeSequence(List<InetAddressAndPort> probeSequence)
    +    {
    +        samples.keySet().retainAll(Gossiper.instance.getLiveMembers());
    +
    +        for (Map.Entry<InetAddressAndPort, AnnotatedEMA> entry: samples.entrySet())
    +        {
    +            // We only send latency probes to nodes that we may plausibly talk to (requested is true)
    +            // but we have not talked to since the last reset of this information (recentlyMeasured is false)
    +            if (entry.getValue().recentlyRequested && !entry.getValue().recentlyMeasured)
    +            {
    +                probeSequence.add(entry.getKey());
    +            }
    +
    +            entry.getValue().recentlyMeasured = false;
    +            entry.getValue().recentlyRequested = false;
    +        }
    +    }
    +
    +    @Override
    +    protected void markRequested(InetAddressAndPort address)
    +    {
    +        AnnotatedEMA ema = samples.get(address);
    +        if (ema != null)
    +            ema.recentlyRequested = true;
    --- End diff --
    
    With the latest change I do a `get` before I do a `lazySet` on the timekeeping state. This way within a given second we only do a single `lazySet` operation (allowing gets to hit the cache basically always).
    
    I also did a quick JMH run to confirm that the load barrier is cheaper than the unconditional store-store barrier (`lazySet`). It is indeed better by a decent amount.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r232436329
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -154,31 +326,203 @@ private void registerMBean()
     
         public void close()
         {
    -        updateSchedular.cancel(false);
    -        resetSchedular.cancel(false);
    +        if (updateScoresScheduler != null)
    +            updateScoresScheduler.cancel(false);
    +        if (updateSamplesScheduler != null)
    +            updateSamplesScheduler.cancel(false);
    +
    +        for (AnnotatedMeasurement measurement : samples.values())
    +        {
    +            if (measurement.probeFuture != null)
    +                measurement.probeFuture.cancel(false);
    +
    +            measurement.millisSinceLastMeasure.set(0);
    +            measurement.millisSinceLastRequest.set(MAX_PROBE_INTERVAL_MS);
    +            measurement.probeTimerMillis = 0;
    +        }
     
             MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
             try
             {
    -            mbs.unregisterMBean(new ObjectName(mbeanName));
    +            if (mbeanRegistered)
    +                mbs.unregisterMBean(new ObjectName(mbeanName));
             }
             catch (Exception e)
             {
                 throw new RuntimeException(e);
             }
         }
     
    +    /**
    +     * Background task running on the samples dictionary. The default implementation sends latency probes (PING)
    +     * messages to explore nodes that we have not received timings for recently but have ranked in
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}.
    +     */
    +    protected void updateSamples()
    +    {
    +        // Split calculation of probe timers from sending probes for testability
    +        calculateProbes(samples, dynamicLatencyProbeInterval);
    +
    +        if (!StorageService.instance.isGossipActive())
    +            return;
    +
    +        schedulePings(samples);
    --- End diff --
    
    Sure, done.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225622741
  
    --- Diff: doc/source/operating/snitch.rst ---
    @@ -28,16 +28,18 @@ In cassandra, the snitch has two functions:
     
     Dynamic snitching
     ^^^^^^^^^^^^^^^^^
    -
     The dynamic snitch monitor read latencies to avoid reading from hosts that have slowed down. The dynamic snitch is
     configured with the following properties on ``cassandra.yaml``:
     
     - ``dynamic_snitch``: whether the dynamic snitch should be enabled or disabled.
     - ``dynamic_snitch_update_interval_in_ms``: controls how often to perform the more expensive part of host score
       calculation.
    -- ``dynamic_snitch_reset_interval_in_ms``: if set greater than zero, this will allow 'pinning' of replicas to hosts
    -  in order to increase cache capacity.
    -- ``dynamic_snitch_badness_threshold:``: The badness threshold will control how much worse the pinned host has to be
    +- ``dynamic_snitch_latency_probe_interval_in_ms``: controls how often this node
    --- End diff --
    
    OK. I agree we don't want to encourage people to implement their own dynamic snitch.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r240311277
  
    --- Diff: test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java ---
    @@ -18,35 +18,99 @@
     
     package org.apache.cassandra.locator;
     
    -import java.io.IOException;
    +import java.net.UnknownHostException;
    --- End diff --
    
    It looks like shutting down the DES is covered a little in the settings test, but it would be good if it validated somewhere the futures are cleaned up and there are no scheduled/running tasks in the executors.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225348901
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitchEMA.java ---
    @@ -0,0 +1,181 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.cassandra.locator;
    +
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.concurrent.ConcurrentHashMap;
    +
    +import org.apache.cassandra.gms.Gossiper;
    +import org.apache.cassandra.metrics.ExponentialMovingAverage;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +
    +
    +/**
    + * A dynamic snitching implementation that uses Exponentially Decaying Histograms to prefer or
    + * de-prefer hosts
    + *
    + * This was the default implementation prior to Cassandra 4.0 and is being left as the default
    + * in 4.0
    + */
    +public class DynamicEndpointSnitchEMA extends DynamicEndpointSnitch
    +{
    +    // A ~10 sample EMA
    +    private static final double EMA_ALPHA = 0.10;
    +
    +    private final ConcurrentHashMap<InetAddressAndPort, AnnotatedEMA> samples = new ConcurrentHashMap<>();
    +
    +    /**
    +     * Adds two boolean markers to the ExponentialMovingAverage for telling if the data has been
    +     * updated or requested recently.
    +     *
    +     * recentlyMeasured is updated through {@link AnnotatedEMA#update(long, boolean)}
    +     * recentlyRequested is updated through {@link DynamicEndpointSnitch#markRequested}
    +     *
    +     * Both markers are periodically reset via {@link DynamicEndpointSnitch#latencyProbeNeeded(long)}
    +     */
    +    private static class AnnotatedEMA extends ExponentialMovingAverage
    +    {
    +        volatile boolean recentlyRequested = false;
    +        volatile boolean recentlyMeasured = false;
    +
    +        AnnotatedEMA(double alpha, double initialValue)
    +        {
    +            super(alpha, initialValue);
    +        }
    +
    +        public void update(long value, boolean isRealRead)
    +        {
    +            recentlyMeasured = recentlyMeasured || isRealRead;
    +            super.update(value);
    +        }
    +    }
    +
    +    // Called via reflection
    +    public DynamicEndpointSnitchEMA(IEndpointSnitch snitch)
    +    {
    +        this(snitch, "ema");
    +    }
    +
    +    protected DynamicEndpointSnitchEMA(IEndpointSnitch snitch, String instance)
    +    {
    +        super(snitch, instance);
    +    }
    +
    +    private void receiveTiming(InetAddressAndPort host, long latency, boolean isRealRead) // this is cheap
    +    {
    +        AnnotatedEMA sample = samples.get(host);
    +        if (sample == null)
    +        {
    +            AnnotatedEMA maybeNewSample = new AnnotatedEMA(EMA_ALPHA, latency);
    +            sample = samples.putIfAbsent(host, maybeNewSample);
    +            if (sample == null)
    +                sample = maybeNewSample;
    +        }
    +        sample.update(latency, isRealRead);
    +    }
    +
    +    public void receiveTiming(InetAddressAndPort address, long latency, LatencyMeasurementType measurementType)
    +    {
    +        if (measurementType == LatencyMeasurementType.READ)
    +            receiveTiming(address, latency, true);
    +        else if (measurementType == LatencyMeasurementType.PROBE)
    +            receiveTiming(address, latency, false);
    +    }
    +
    +    @Override
    +    protected void reset() {
    +        this.samples.clear();
    +        super.reset();
    +    }
    +
    +    @Override
    +    public Map<InetAddressAndPort, Double> calculateScores()
    +    {
    +        // We're going to weight the latency for each host against the worst one we see, to
    +        // arrive at sort of a 'badness percentage' for them. First, find the worst for each:
    +        HashMap<InetAddressAndPort, Double> newScores = new HashMap<>(samples.size());
    +        Optional<Double> maxObservedAvgLatency = samples.values().stream()
    +                                                        .map(ExponentialMovingAverage::getAvg)
    +                                                        .max(Double::compare);
    +
    +        final double maxAvgLatency = maxObservedAvgLatency.isPresent() ? maxObservedAvgLatency.get() : 1;
    +
    +        // now make another pass to do the weighting based on the maximums we found before
    +        for (Map.Entry<InetAddressAndPort, AnnotatedEMA> entry : samples.entrySet())
    +        {
    +            // Samples may have changed but rather than creating garbage by copying samples we just ensure
    +            // that all scores are less than 1.0
    +            double addrAvg = entry.getValue().getAvg();
    +            double score = addrAvg / Math.max(addrAvg, maxAvgLatency);
    +            // finally, add the severity without any weighting, since hosts scale this relative to their own load
    +            // and the size of the task causing the severity.
    +            // "Severity" is basically a measure of compaction activity (CASSANDRA-3722).
    +            if (USE_SEVERITY)
    +                score += getSeverity(entry.getKey());
    +            // lowest score (least amount of badness) wins.
    +            newScores.put(entry.getKey(), score);
    +        }
    +        return newScores;
    +    }
    +
    +    @Override
    +    public List<Double> dumpTimings(String hostname) throws UnknownHostException
    +    {
    +        InetAddressAndPort host = InetAddressAndPort.getByName(hostname);
    +        ArrayList<Double> timings = new ArrayList<>();
    +        ExponentialMovingAverage avg = samples.get(host);
    +        if (avg != null)
    +        {
    +            timings.add(avg.getAvg());
    +        }
    +        return timings;
    +    }
    +
    +    @Override
    +    protected void updateLatencyProbeSequence(List<InetAddressAndPort> probeSequence)
    +    {
    +        samples.keySet().retainAll(Gossiper.instance.getLiveMembers());
    +
    +        for (Map.Entry<InetAddressAndPort, AnnotatedEMA> entry: samples.entrySet())
    +        {
    +            // We only send latency probes to nodes that we may plausibly talk to (requested is true)
    +            // but we have not talked to since the last reset of this information (recentlyMeasured is false)
    +            if (entry.getValue().recentlyRequested && !entry.getValue().recentlyMeasured)
    +            {
    +                probeSequence.add(entry.getKey());
    +            }
    +
    +            entry.getValue().recentlyMeasured = false;
    +            entry.getValue().recentlyRequested = false;
    +        }
    +    }
    +
    +    @Override
    +    protected void markRequested(InetAddressAndPort address)
    +    {
    +        AnnotatedEMA ema = samples.get(address);
    +        if (ema != null)
    +            ema.recentlyRequested = true;
    --- End diff --
    
    I was curious about this if a load before the set would be faster or slower than just a set. I guess I decided that the performance difference was probably so negligible I should just go for most readable. I don't care though heh, changing with the flag update.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r239939483
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -21,133 +21,505 @@
     import java.net.InetAddress;
     import java.net.UnknownHostException;
     import java.util.*;
    +import java.util.concurrent.CancellationException;
     import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ExecutionException;
     import java.util.concurrent.ScheduledFuture;
     import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
    +import java.util.concurrent.atomic.AtomicLong;
     import java.util.stream.Collectors;
     
    -import com.codahale.metrics.ExponentiallyDecayingReservoir;
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.util.concurrent.Futures;
    +import com.google.common.util.concurrent.MoreExecutors;
    +import com.google.common.util.concurrent.RateLimiter;
    +import com.google.common.util.concurrent.SettableFuture;
     
    -import com.codahale.metrics.Snapshot;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
     import org.apache.cassandra.concurrent.ScheduledExecutors;
     import org.apache.cassandra.config.DatabaseDescriptor;
    +import org.apache.cassandra.exceptions.RequestFailureReason;
     import org.apache.cassandra.gms.ApplicationState;
     import org.apache.cassandra.gms.EndpointState;
     import org.apache.cassandra.gms.Gossiper;
     import org.apache.cassandra.gms.VersionedValue;
    +import org.apache.cassandra.locator.dynamicsnitch.DynamicEndpointSnitchHistogram;
    +import org.apache.cassandra.net.IAsyncCallbackWithFailure;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +import org.apache.cassandra.net.MessageIn;
    +import org.apache.cassandra.net.MessageOut;
     import org.apache.cassandra.net.MessagingService;
    +import org.apache.cassandra.net.PingMessage;
     import org.apache.cassandra.service.StorageService;
     import org.apache.cassandra.utils.FBUtilities;
     import org.apache.cassandra.utils.MBeanWrapper;
     
    +import static org.apache.cassandra.net.MessagingService.Verb.PING;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.LARGE_MESSAGE;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.SMALL_MESSAGE;
    +
    +
     /**
      * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector
    + * Note that the subclasses (e.g. {@link DynamicEndpointSnitchHistogram}) are responsible for actually measuring
    + * latency and providing an ISnitchMeasurement implementation back to this class.
      */
    -public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
    +public abstract class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
     {
    -    private static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    private static final Logger logger = LoggerFactory.getLogger(DynamicEndpointSnitch.class);
    +
    +    // Latency measurement and ranking. The samples contain latency measurements and the scores are used for ranking
    +    protected boolean registered = false;
    +    protected static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    protected volatile Map<InetAddressAndPort, Double> scores = new HashMap<>();
     
    -    private static final double ALPHA = 0.75; // set to 0.75 to make EDS more biased to towards the newer values
    -    private static final int WINDOW_SIZE = 100;
    +    protected final Map<InetAddressAndPort, AnnotatedMeasurement> samples = new ConcurrentHashMap<>();
     
    -    private volatile int dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    -    private volatile int dynamicResetInterval = DatabaseDescriptor.getDynamicResetInterval();
    -    private volatile double dynamicBadnessThreshold = DatabaseDescriptor.getDynamicBadnessThreshold();
    +    // Latency probe functionality for actively probing endpoints that we haven't measured recently but are ranking
    +    public static final long MAX_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_max_probe_interval_ms", 60 * 10 * 1000L);
    +    public static final long MIN_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_min_probe_interval_ms", 60 * 1000L) ;
    +    // The probe rate is set later when configuration is read in applyConfigChanges
    +    protected static final RateLimiter probeRateLimiter = RateLimiter.create(1);
    +    protected static final DebuggableScheduledThreadPoolExecutor latencyProbeExecutor = new DebuggableScheduledThreadPoolExecutor("LatencyProbes");
    +    private long lastUpdateSamplesNanos = System.nanoTime();
    +
    +    // User configuration of the snitch tunables
    +    protected volatile int dynamicUpdateInterval = -1;
    +    protected volatile int dynamicSampleUpdateInterval = -1;
    +    protected volatile double dynamicBadnessThreshold = 0;
     
         // the score for a merged set of endpoints must be this much worse than the score for separate endpoints to
         // warrant not merging two ranges into a single range
         private static final double RANGE_MERGING_PREFERENCE = 1.5;
     
         private String mbeanName;
    -    private boolean registered = false;
    -
    -    private volatile HashMap<InetAddressAndPort, Double> scores = new HashMap<>();
    -    private final ConcurrentHashMap<InetAddressAndPort, ExponentiallyDecayingReservoir> samples = new ConcurrentHashMap<>();
    +    private boolean mbeanRegistered = false;
     
         public final IEndpointSnitch subsnitch;
     
    -    private volatile ScheduledFuture<?> updateSchedular;
    -    private volatile ScheduledFuture<?> resetSchedular;
    -
    -    private final Runnable update;
    -    private final Runnable reset;
    +    private volatile ScheduledFuture<?> updateScoresScheduler = null;
    +    private volatile ScheduledFuture<?> updateSamplesScheduler = null;
     
         public DynamicEndpointSnitch(IEndpointSnitch snitch)
         {
             this(snitch, null);
         }
     
    -    public DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
    +    protected DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
         {
             mbeanName = "org.apache.cassandra.db:type=DynamicEndpointSnitch";
             if (instance != null)
                 mbeanName += ",instance=" + instance;
             subsnitch = snitch;
    -        update = new Runnable()
    +
    +        if (DatabaseDescriptor.isDaemonInitialized())
             {
    -            public void run()
    +            open();
    +        }
    +    }
    +
    +    /**
    +     * Update configuration of the background tasks and restart the various scheduler tasks
    +     * if the configured rates for these tasks have changed.
    +     */
    +    public void applyConfigChanges(int newDynamicUpdateInternal, int newDynamicSampleUpdateInterval, double newDynamicBadnessThreshold)
    +    {
    +        if (DatabaseDescriptor.isDaemonInitialized())
    +        {
    +            if (dynamicUpdateInterval != newDynamicUpdateInternal || updateScoresScheduler == null)
                 {
    -                updateScores();
    +                cancelAndWait(updateScoresScheduler, Math.max(1, dynamicUpdateInterval), TimeUnit.MILLISECONDS);
    +                updateScoresScheduler = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(this::updateScores, newDynamicUpdateInternal, newDynamicUpdateInternal, TimeUnit.MILLISECONDS);
                 }
    -        };
    -        reset = new Runnable()
    +
    +            if (dynamicSampleUpdateInterval != newDynamicSampleUpdateInterval || updateSamplesScheduler == null)
    +            {
    +                cancelAndWait(updateSamplesScheduler, Math.max(1, dynamicSampleUpdateInterval), TimeUnit.MILLISECONDS);
    +                if (newDynamicSampleUpdateInterval > 0)
    +                    updateSamplesScheduler = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(this::updateSamples, newDynamicSampleUpdateInterval, newDynamicSampleUpdateInterval, TimeUnit.MILLISECONDS);
    +            }
    +        }
    +
    +        dynamicUpdateInterval = newDynamicUpdateInternal;
    +        dynamicSampleUpdateInterval = newDynamicSampleUpdateInterval;
    +        dynamicBadnessThreshold = newDynamicBadnessThreshold;
    +
    +        if (dynamicSampleUpdateInterval > 0)
    +            probeRateLimiter.setRate(dynamicSampleUpdateInterval);
    +    }
    +
    +    public synchronized void open()
    +    {
    +        applyConfigChanges(DatabaseDescriptor.getDynamicUpdateInterval(),
    +                           DatabaseDescriptor.getDynamicSampleUpdateInterval(),
    +                           DatabaseDescriptor.getDynamicBadnessThreshold());
    +
    +        MBeanWrapper.instance.registerMBean(this, mbeanName);
    +        mbeanRegistered = true;
    +    }
    +
    +    public synchronized void close()
    +    {
    +        cancelAndWait(updateScoresScheduler, Math.max(1, dynamicUpdateInterval), TimeUnit.MILLISECONDS);
    +        cancelAndWait(updateSamplesScheduler, Math.max(1, dynamicSampleUpdateInterval), TimeUnit.MILLISECONDS);
    +        updateScoresScheduler = null;
    +        updateSamplesScheduler = null;
    +
    +        for (AnnotatedMeasurement measurement : samples.values())
             {
    -            public void run()
    +            cancelAndWait(measurement.probeFuture, PING.getTimeout(), TimeUnit.MILLISECONDS);
    +
    +            measurement.millisSinceLastMeasure.set(0);
    +            measurement.millisSinceLastRequest.set(MAX_PROBE_INTERVAL_MS);
    +            measurement.nextProbeDelayMillis = 0;
    +        }
    +
    +        if (mbeanRegistered)
    +            MBeanWrapper.instance.unregisterMBean(mbeanName);
    +
    +        mbeanRegistered = false;
    +    }
    +
    +    private static void cancelAndWait(ScheduledFuture future, long timeout, TimeUnit unit)
    +    {
    +        if (future != null)
    +        {
    +            future.cancel(false);
    +            try
                 {
    -                // we do this so that a host considered bad has a chance to recover, otherwise would we never try
    -                // to read from it, which would cause its score to never change
    -                reset();
    +                future.get(timeout, unit);
                 }
    -        };
    +            catch (CancellationException | InterruptedException | ExecutionException | TimeoutException ignored)
    +            {
    +                // Exception is expected to happen eventually due to the cancel -> get
    --- End diff --
    
    It seems like only CanellationException should be ignored? All the others shouldn't happen?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by jolynch <gi...@git.apache.org>.
Github user jolynch commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r227617118
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitchEMA.java ---
    @@ -0,0 +1,181 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.cassandra.locator;
    +
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.concurrent.ConcurrentHashMap;
    +
    +import org.apache.cassandra.gms.Gossiper;
    +import org.apache.cassandra.metrics.ExponentialMovingAverage;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +
    +
    +/**
    + * A dynamic snitching implementation that uses Exponentially Decaying Histograms to prefer or
    + * de-prefer hosts
    + *
    + * This was the default implementation prior to Cassandra 4.0 and is being left as the default
    + * in 4.0
    + */
    +public class DynamicEndpointSnitchEMA extends DynamicEndpointSnitch
    +{
    +    // A ~10 sample EMA
    +    private static final double EMA_ALPHA = 0.10;
    +
    +    private final ConcurrentHashMap<InetAddressAndPort, AnnotatedEMA> samples = new ConcurrentHashMap<>();
    +
    +    /**
    +     * Adds two boolean markers to the ExponentialMovingAverage for telling if the data has been
    +     * updated or requested recently.
    +     *
    +     * recentlyMeasured is updated through {@link AnnotatedEMA#update(long, boolean)}
    +     * recentlyRequested is updated through {@link DynamicEndpointSnitch#markRequested}
    +     *
    +     * Both markers are periodically reset via {@link DynamicEndpointSnitch#latencyProbeNeeded(long)}
    +     */
    +    private static class AnnotatedEMA extends ExponentialMovingAverage
    +    {
    +        volatile boolean recentlyRequested = false;
    +        volatile boolean recentlyMeasured = false;
    +
    +        AnnotatedEMA(double alpha, double initialValue)
    +        {
    +            super(alpha, initialValue);
    +        }
    +
    +        public void update(long value, boolean isRealRead)
    +        {
    +            recentlyMeasured = recentlyMeasured || isRealRead;
    --- End diff --
    
    I ended up making these two `AtomicIntegers` to make the state really clear and the code easier to understand per your recommendations to use a STPE instead of doing it all on the one thread.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r229487068
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -154,31 +326,203 @@ private void registerMBean()
     
         public void close()
         {
    -        updateSchedular.cancel(false);
    -        resetSchedular.cancel(false);
    +        if (updateScoresScheduler != null)
    +            updateScoresScheduler.cancel(false);
    +        if (updateSamplesScheduler != null)
    +            updateSamplesScheduler.cancel(false);
    +
    +        for (AnnotatedMeasurement measurement : samples.values())
    +        {
    +            if (measurement.probeFuture != null)
    +                measurement.probeFuture.cancel(false);
    +
    +            measurement.millisSinceLastMeasure.set(0);
    +            measurement.millisSinceLastRequest.set(MAX_PROBE_INTERVAL_MS);
    +            measurement.probeTimerMillis = 0;
    +        }
     
             MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
             try
             {
    -            mbs.unregisterMBean(new ObjectName(mbeanName));
    +            if (mbeanRegistered)
    +                mbs.unregisterMBean(new ObjectName(mbeanName));
             }
             catch (Exception e)
             {
                 throw new RuntimeException(e);
             }
         }
     
    +    /**
    +     * Background task running on the samples dictionary. The default implementation sends latency probes (PING)
    +     * messages to explore nodes that we have not received timings for recently but have ranked in
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}.
    +     */
    +    protected void updateSamples()
    +    {
    +        // Split calculation of probe timers from sending probes for testability
    +        calculateProbes(samples, dynamicLatencyProbeInterval);
    +
    +        if (!StorageService.instance.isGossipActive())
    +            return;
    +
    +        schedulePings(samples);
    +    }
    +
    +    /**
    +     * This method mutates the passed AnnotatedMeasurements to implement capped exponential backoff per endpoint.
    +     *
    +     * The algorithm is as follows:
    +     * 1. All samples get their millisSinceLastMeasure and millisSinceLastRequest fields
    +     *    incremented by the passed interval
    +     * 2. Any recently requested (ranked) endpoints that have not been measured recently (e.g. because the snitch
    +     *    has sent them no traffic) get probes with exponential backoff.
    +     *
    +     * The backoff is capped at MAX_PROBE_INTERVAL_MS. Furthermore the probes are stopped after
    +     * MAX_PROBE_INTERVAL_MS of no ranking requests as well.
    +     *
    +     * At the end of this method, any passed AnnotatedMeasurements that need latency probes will have non zero
    +     * probeTimerMillis members set.
    +     */
    +    @VisibleForTesting
    +    static void calculateProbes(Map<InetAddressAndPort, AnnotatedMeasurement> samples, long intervalMillis) {
    +        for (Map.Entry<InetAddressAndPort, AnnotatedMeasurement> entry: samples.entrySet())
    +        {
    +            if (entry.getKey().equals(FBUtilities.getBroadcastAddressAndPort()))
    +                continue;
    +
    +            AnnotatedMeasurement measurement = entry.getValue();
    +            long lastMeasure = measurement.millisSinceLastMeasure.getAndAdd(intervalMillis);
    --- End diff --
    
    Hmm that got me thinking, wouldn't a useful JMX gauge be probes sent per second? I am wondering if that is a nice to have just so you can monitor how much work is being done by the snitch? WDYT?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r229411406
  
    --- Diff: test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java ---
    @@ -19,34 +19,104 @@
     package org.apache.cassandra.locator;
     
     import java.io.IOException;
    +import java.net.UnknownHostException;
     import java.util.*;
     
    -import org.junit.BeforeClass;
    +import org.junit.After;
    +import org.junit.Before;
     import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
     
     import org.apache.cassandra.Util;
     import org.apache.cassandra.config.DatabaseDescriptor;
     import org.apache.cassandra.exceptions.ConfigurationException;
    +import org.apache.cassandra.gms.Gossiper;
    +import org.apache.cassandra.locator.dynamicsnitch.DynamicEndpointSnitchEMA;
    +import org.apache.cassandra.locator.dynamicsnitch.DynamicEndpointSnitchHistogram;
    +import org.apache.cassandra.locator.dynamicsnitch.DynamicEndpointSnitchLegacyHistogram;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +import org.apache.cassandra.net.async.TestScheduledFuture;
     import org.apache.cassandra.service.StorageService;
     import org.apache.cassandra.utils.FBUtilities;
     
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertFalse;
    +import static org.junit.Assert.assertTrue;
    +
    +@RunWith(Parameterized.class)
     public class DynamicEndpointSnitchTest
     {
    +    private static InetAddressAndPort[] hosts;
    +    // Reduce the update interval significantly so that tests run quickly
    +    private static final long UPDATE_INTERVAL_MS = 10;
    +    // Intentially 31 and a little bit instead of 30 seconds flat so this doesn't divide evenly into the default
    +    // MAX_PROBE_INTERVAL_MS. Also pretty high so latency probes don't interfere with the unit tests
    +    private static final long PING_INTERVAL_MS = 31 * 1003;
    +
    +    private final DynamicEndpointSnitch dsnitch;
    +
    +    public DynamicEndpointSnitchTest(DynamicEndpointSnitch dsnitch)
    +    {
    +        this.dsnitch = dsnitch;
    +    }
    +
    +    @Before
    +    public void prepareDES()
    +    {
    +        for (InetAddressAndPort host : hosts)
    +        {
    +            Gossiper.instance.initializeNodeUnsafe(host, UUID.randomUUID(), 1);
    +            Gossiper.instance.realMarkAlive(host, Gossiper.instance.getEndpointStateForEndpoint(host));
    +        }
    +        dsnitch.reset();
    +    }
     
    -    @BeforeClass
    -    public static void setupDD()
    +    @Parameterized.Parameters(name="{index}: {0}")
    +    public static Iterable<?> getDESImplementation() throws UnknownHostException
         {
             DatabaseDescriptor.daemonInitialization();
    +        // do this because SS needs to be initialized before DES can work properly.
    +        StorageService.instance.unsafeInitialize();
    +
    +        hosts = new InetAddressAndPort[] {
    +            FBUtilities.getBroadcastAddressAndPort(),
    +            InetAddressAndPort.getByName("127.0.0.2"),
    +            InetAddressAndPort.getByName("127.0.0.3"),
    +            InetAddressAndPort.getByName("127.0.0.4"),
    +            InetAddressAndPort.getByName("127.0.0.5"),
    +        };
    +
    +        SimpleSnitch ss1 = new SimpleSnitch();
    +        DynamicEndpointSnitch probeDES = new DynamicEndpointSnitchHistogram(ss1, String.valueOf(ss1.hashCode()));
    +        probeDES.applyConfigChanges((int) UPDATE_INTERVAL_MS, (int) PING_INTERVAL_MS, DatabaseDescriptor.getDynamicBadnessThreshold());
    +
    +        SimpleSnitch ss2 = new SimpleSnitch();
    +        DynamicEndpointSnitch oldDES = new DynamicEndpointSnitchLegacyHistogram(ss2, String.valueOf(ss2.hashCode()));
    +        oldDES.applyConfigChanges((int) UPDATE_INTERVAL_MS, (int) PING_INTERVAL_MS, DatabaseDescriptor.getDynamicBadnessThreshold());
    +
    +        SimpleSnitch ss3 = new SimpleSnitch();
    +        DynamicEndpointSnitch emaDES = new DynamicEndpointSnitchEMA(ss3, String.valueOf(ss3.hashCode()));
    +        emaDES.applyConfigChanges((int) UPDATE_INTERVAL_MS, (int) PING_INTERVAL_MS, DatabaseDescriptor.getDynamicBadnessThreshold());
    +
    +        return Arrays.asList(probeDES, oldDES, emaDES);
    +    }
    +
    +    @After
    +    public void resetDES()
    +    {
    +        dsnitch.reset();
         }
     
         private static void setScores(DynamicEndpointSnitch dsnitch, int rounds, List<InetAddressAndPort> hosts, Integer... scores) throws InterruptedException
         {
             for (int round = 0; round < rounds; round++)
             {
                 for (int i = 0; i < hosts.size(); i++)
    -                dsnitch.receiveTiming(hosts.get(i), scores[i]);
    +                dsnitch.receiveTiming(hosts.get(i), scores[i], LatencyMeasurementType.READ);
             }
    -        Thread.sleep(150);
    +        // Slightly higher than the update interval to allow scores to propagate
    +        Thread.sleep(UPDATE_INTERVAL_MS + 10);
    --- End diff --
    
    This could end up being a source of flakiness. A test method to force it to recalculate in the other thread and wait on the resulting future.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r229469414
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -23,128 +23,300 @@
     import java.net.UnknownHostException;
     import java.util.*;
     import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ExecutionException;
     import java.util.concurrent.ScheduledFuture;
     import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
     import java.util.stream.Collectors;
     
    -import com.codahale.metrics.ExponentiallyDecayingReservoir;
     import javax.management.MBeanServer;
     import javax.management.ObjectName;
     
    -import com.codahale.metrics.Snapshot;
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.util.concurrent.Futures;
    +import com.google.common.util.concurrent.MoreExecutors;
    +import com.google.common.util.concurrent.RateLimiter;
    +import com.google.common.util.concurrent.SettableFuture;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
     import org.apache.cassandra.concurrent.ScheduledExecutors;
     import org.apache.cassandra.config.DatabaseDescriptor;
    +import org.apache.cassandra.exceptions.RequestFailureReason;
     import org.apache.cassandra.gms.ApplicationState;
     import org.apache.cassandra.gms.EndpointState;
     import org.apache.cassandra.gms.Gossiper;
     import org.apache.cassandra.gms.VersionedValue;
    +import org.apache.cassandra.locator.dynamicsnitch.DynamicEndpointSnitchHistogram;
    +import org.apache.cassandra.net.IAsyncCallbackWithFailure;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +import org.apache.cassandra.net.MessageIn;
    +import org.apache.cassandra.net.MessageOut;
     import org.apache.cassandra.net.MessagingService;
    +import org.apache.cassandra.net.PingMessage;
     import org.apache.cassandra.service.StorageService;
     import org.apache.cassandra.utils.FBUtilities;
     
    +import static org.apache.cassandra.net.MessagingService.Verb.PING;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.LARGE_MESSAGE;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.SMALL_MESSAGE;
    +
    +
     /**
      * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector
    + * Note that the subclasses (e.g. {@link DynamicEndpointSnitchHistogram}) are responsible for actually measuring
    + * latency and providing an ISnitchMeasurement implementation back to this class.
      */
    -public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
    +public abstract class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
     {
    -    private static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    -
    -    private static final double ALPHA = 0.75; // set to 0.75 to make EDS more biased to towards the newer values
    -    private static final int WINDOW_SIZE = 100;
    -
    -    private volatile int dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    -    private volatile int dynamicResetInterval = DatabaseDescriptor.getDynamicResetInterval();
    -    private volatile double dynamicBadnessThreshold = DatabaseDescriptor.getDynamicBadnessThreshold();
    +    private static final Logger logger = LoggerFactory.getLogger(DynamicEndpointSnitch.class);
    +
    +    // Latency measurement and ranking. The samples contain latency measurements and the scores are used for ranking
    +    protected boolean registered = false;
    +    protected static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    protected volatile Map<InetAddressAndPort, Double> scores = new HashMap<>();
    +    protected final Map<InetAddressAndPort, AnnotatedMeasurement> samples = new ConcurrentHashMap<>();
    +
    +    // Latency probe functionality for actively probing endpoints that we haven't measured recently but are ranking
    +    public static final long MAX_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_max_probe_interval_ms", 60 * 10 * 1000L);
    +    public static final long MIN_PROBE_INTERVAL_MS = Long.getLong("cassandra.dynamic_snitch_min_probe_interval_ms", 60 * 1000L) ;
    +    // The probe rate is set later when configuration is read
    +    protected static final RateLimiter probeRateLimiter = RateLimiter.create(1);
    +    protected static final DebuggableScheduledThreadPoolExecutor latencyProbeExecutor = new DebuggableScheduledThreadPoolExecutor(1, "LatencyProbes", Thread.MIN_PRIORITY);
    +
    +    // User configuration of the snitch tunables
    +    protected volatile int dynamicUpdateInterval = -1;
    +    protected volatile int dynamicLatencyProbeInterval = -1;
    +    protected volatile double dynamicBadnessThreshold = 0;
     
         // the score for a merged set of endpoints must be this much worse than the score for separate endpoints to
         // warrant not merging two ranges into a single range
         private static final double RANGE_MERGING_PREFERENCE = 1.5;
     
         private String mbeanName;
    -    private boolean registered = false;
    -
    -    private volatile HashMap<InetAddressAndPort, Double> scores = new HashMap<>();
    -    private final ConcurrentHashMap<InetAddressAndPort, ExponentiallyDecayingReservoir> samples = new ConcurrentHashMap<>();
    +    private boolean mbeanRegistered = false;
     
         public final IEndpointSnitch subsnitch;
     
    -    private volatile ScheduledFuture<?> updateSchedular;
    -    private volatile ScheduledFuture<?> resetSchedular;
    -
    -    private final Runnable update;
    -    private final Runnable reset;
    +    private volatile ScheduledFuture<?> updateScoresScheduler;
    +    private volatile ScheduledFuture<?> updateSamplesScheduler;
     
         public DynamicEndpointSnitch(IEndpointSnitch snitch)
         {
             this(snitch, null);
         }
     
    -    public DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
    +    protected DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
         {
             mbeanName = "org.apache.cassandra.db:type=DynamicEndpointSnitch";
             if (instance != null)
                 mbeanName += ",instance=" + instance;
             subsnitch = snitch;
    -        update = new Runnable()
    +
    +        if (DatabaseDescriptor.isDaemonInitialized())
             {
    -            public void run()
    -            {
    -                updateScores();
    -            }
    -        };
    -        reset = new Runnable()
    +            applyConfigChanges(DatabaseDescriptor.getDynamicUpdateInterval(),
    +                               DatabaseDescriptor.getDynamicSampleUpdateInterval(),
    +                               DatabaseDescriptor.getDynamicBadnessThreshold());
    +            open();
    +        }
    +    }
    +
    +    /**
    +     * Allows subclasses to inject new ways of measuring latency back to this abstract base class.
    +     */
    +    protected interface ISnitchMeasurement
    +    {
    +        void sample(long value);
    +        double measure();
    +        Iterable<Double> measurements();
    +    }
    +
    +    /**
    +     * Adds some boookeeping that the DES uses over top of the various metrics techniques used by the
    +     * implementations. This is used to allow CASSANDRA-14459 latency probes as well as further experimentation
    +     * on new latency measurement techniques in CASSANDRA-14817
    +     *
    +     * {@link AnnotatedMeasurement#millisSinceLastRequest} is set to zero through
    +     * {@link DynamicEndpointSnitch#sortedByProximity(InetAddressAndPort, ReplicaCollection)}. It defaults to
    +     * the maximum interval so that we only start probing once it has been ranked at least once
    +     * {@link AnnotatedMeasurement#millisSinceLastMeasure} is set to zero from
    +     * {@link DynamicEndpointSnitch#receiveTiming(InetAddressAndPort, long, LatencyMeasurementType)}.
    +     *
    +     * {@link AnnotatedMeasurement#millisSinceLastMeasure and {@link AnnotatedMeasurement#probeTimerMillis}
    +     * are incremented via {@link DynamicEndpointSnitch#updateSamples()}
    +     */
    +    protected static class AnnotatedMeasurement
    +    {
    +        // Used to optimally target latency probes only on nodes that are both requested for ranking
    +        // and are not being measured. For example with token aware clients a large portion of the cluster will never
    +        // be ranked at all and therefore we won't probe them.
    +        public AtomicLong millisSinceLastRequest = new AtomicLong(MAX_PROBE_INTERVAL_MS);
    +        public AtomicLong millisSinceLastMeasure = new AtomicLong(0);
    +        public long probeTimerMillis = 0;
    --- End diff --
    
    This would make a lot more sense to me as nextProbeDelay? Since we are tracking how far into the future to delay the next probe?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #283: CASSANDRA-14459: DynamicEndpointSnitch should n...

Posted by aweisberg <gi...@git.apache.org>.
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/283#discussion_r225330113
  
    --- Diff: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java ---
    @@ -22,121 +22,194 @@
     import java.net.InetAddress;
     import java.net.UnknownHostException;
     import java.util.*;
    -import java.util.concurrent.ConcurrentHashMap;
     import java.util.concurrent.ScheduledFuture;
     import java.util.concurrent.TimeUnit;
     import java.util.stream.Collectors;
     
    -import com.codahale.metrics.ExponentiallyDecayingReservoir;
     import javax.management.MBeanServer;
     import javax.management.ObjectName;
     
    -import com.codahale.metrics.Snapshot;
    +import com.google.common.annotations.VisibleForTesting;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
     import org.apache.cassandra.concurrent.ScheduledExecutors;
     import org.apache.cassandra.config.DatabaseDescriptor;
     import org.apache.cassandra.gms.ApplicationState;
     import org.apache.cassandra.gms.EndpointState;
     import org.apache.cassandra.gms.Gossiper;
     import org.apache.cassandra.gms.VersionedValue;
    +import org.apache.cassandra.net.IAsyncCallback;
    +import org.apache.cassandra.net.LatencyMeasurementType;
    +import org.apache.cassandra.net.MessageIn;
    +import org.apache.cassandra.net.MessageOut;
     import org.apache.cassandra.net.MessagingService;
    +import org.apache.cassandra.net.PingMessage;
     import org.apache.cassandra.service.StorageService;
     import org.apache.cassandra.utils.FBUtilities;
     
    +import static org.apache.cassandra.net.MessagingService.Verb.PING;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.LARGE_MESSAGE;
    +import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.SMALL_MESSAGE;
    +
    +
     /**
      * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector
    + * Note that the subclasses (e.g. {@link DynamicEndpointSnitchHistogram}) are responsible for actually measuring
    + * latency and populating the {@link #scores} map.
      */
    -public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
    +public abstract class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean
     {
    -    private static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    -
    -    private static final double ALPHA = 0.75; // set to 0.75 to make EDS more biased to towards the newer values
    -    private static final int WINDOW_SIZE = 100;
    -
    -    private volatile int dynamicUpdateInterval = DatabaseDescriptor.getDynamicUpdateInterval();
    -    private volatile int dynamicResetInterval = DatabaseDescriptor.getDynamicResetInterval();
    -    private volatile double dynamicBadnessThreshold = DatabaseDescriptor.getDynamicBadnessThreshold();
    +    private static final Logger logger = LoggerFactory.getLogger(DynamicEndpointSnitch.class);
    +
    +    // Subclass specific functionality
    +    protected static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity");
    +    protected boolean registered = false;
    +    // The scores map is updated via copy in updateScores
    +    // We keep it in the base class for performance reasons (so it can be easily aliased)
    +    protected volatile Map<InetAddressAndPort, Double> scores = new HashMap<>();
    +
    +    // Rate limit how often we generate latency probes
    +    protected long nextAllowedProbeGenerationTime;
    +    protected long nextProbeGenerationTime;
    +    protected int currentProbePosition = 0;
    +    protected final List<InetAddressAndPort> latencyProbeSequence = new ArrayList<>();
    +
    +    // DES general functionality
    +    protected volatile int dynamicUpdateInterval = -1;
    +    protected volatile int dynamicLatencyProbeInterval = -1;
    +    protected volatile double dynamicBadnessThreshold = 0;
     
         // the score for a merged set of endpoints must be this much worse than the score for separate endpoints to
         // warrant not merging two ranges into a single range
         private static final double RANGE_MERGING_PREFERENCE = 1.5;
     
         private String mbeanName;
    -    private boolean registered = false;
    -
    -    private volatile HashMap<InetAddressAndPort, Double> scores = new HashMap<>();
    -    private final ConcurrentHashMap<InetAddressAndPort, ExponentiallyDecayingReservoir> samples = new ConcurrentHashMap<>();
    +    private boolean mbeanRegistered = false;
     
         public final IEndpointSnitch subsnitch;
     
    -    private volatile ScheduledFuture<?> updateSchedular;
    -    private volatile ScheduledFuture<?> resetSchedular;
    +    private volatile ScheduledFuture<?> updateScheduler;
    +    private volatile ScheduledFuture<?> latencyProbeScheduler;
     
    -    private final Runnable update;
    -    private final Runnable reset;
    +    private final Runnable update = this::updateScores;
    +    private final Runnable latencyProbe = this::maybeSendLatencyProbe;
     
         public DynamicEndpointSnitch(IEndpointSnitch snitch)
         {
             this(snitch, null);
         }
     
    -    public DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
    +    protected DynamicEndpointSnitch(IEndpointSnitch snitch, String instance)
         {
             mbeanName = "org.apache.cassandra.db:type=DynamicEndpointSnitch";
             if (instance != null)
                 mbeanName += ",instance=" + instance;
             subsnitch = snitch;
    -        update = new Runnable()
    -        {
    -            public void run()
    -            {
    -                updateScores();
    -            }
    -        };
    -        reset = new Runnable()
    -        {
    -            public void run()
    -            {
    -                // we do this so that a host considered bad has a chance to recover, otherwise would we never try
    -                // to read from it, which would cause its score to never change
    -                reset();
    -            }
    -        };
     
             if (DatabaseDescriptor.isDaemonInitialized())
             {
    -            updateSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(update, dynamicUpdateInterval, dynamicUpdateInterval, TimeUnit.MILLISECONDS);
    -            resetSchedular = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(reset, dynamicResetInterval, dynamicResetInterval, TimeUnit.MILLISECONDS);
    +            applyConfigChanges();
                 registerMBean();
             }
    +        nextProbeGenerationTime = System.nanoTime();
    +    }
    +
    +    /**
    +     * Records a latency. This MUST be cheap as it is called in the fast path
    +     */
    +    abstract public void receiveTiming(InetAddressAndPort address, long latency, LatencyMeasurementType measurementType);
    +
    +    /**
    +     * Called periodically from {@link DynamicEndpointSnitch#updateScores()}. This method returns a new score map
    +     * that will be assigned to the {@link DynamicEndpointSnitch#scores} map.
    +     * This is generally more expensive than {@link DynamicEndpointSnitch#receiveTiming}
    +     */
    +    abstract public Map<InetAddressAndPort, Double> calculateScores();
    +
    +    /**
    +     * Signals that we actually tried to rank vs this host. This way any latency probing can just focus
    +     * on hosts that this peer would potentially talk to.
    +     * @param address
    +     */
    +    abstract protected void markRequested(InetAddressAndPort address);
    +
    +    /**
    +     * Dump the underlying metrics backing the DES's decisions for a given host
    +     */
    +    abstract public List<Double> dumpTimings(String hostname) throws UnknownHostException;
    +
    +    /**
    +     * Populates the provided probe sequence using the underlying metrics
    +     */
    +    abstract void updateLatencyProbeSequence(List<InetAddressAndPort> sequence);
    --- End diff --
    
    Again this seems like a super class concern.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org