From f627170e752a26bcbb395b4f3024f702d00522d1 Mon Sep 17 00:00:00 2001 From: Terry Wilson Date: Fri, 5 Aug 2022 12:38:15 -0700 Subject: [PATCH 01/18] First cut outlier detection implementation --- .../java/io/grpc/SynchronizationContext.java | 30 + .../util/OutlierDetectionLoadBalancer.java | 714 ++++++++++++++++++ .../OutlierDetectionLoadBalancerTest.java | 15 + 3 files changed, 759 insertions(+) create mode 100644 core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java create mode 100644 core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java diff --git a/api/src/main/java/io/grpc/SynchronizationContext.java b/api/src/main/java/io/grpc/SynchronizationContext.java index 03d26b55f0a..7dbd6e5b724 100644 --- a/api/src/main/java/io/grpc/SynchronizationContext.java +++ b/api/src/main/java/io/grpc/SynchronizationContext.java @@ -163,6 +163,36 @@ public String toString() { return new ScheduledHandle(runnable, future); } + /** + * Schedules a task to be added and run via {@link #execute} after an inital delay and then + * repeated after the delay until cancelled. + * + * @param task the task being scheduled + * @param initialDelay the delay before the first run + * @param delay the delay after the first run. + * @param unit the time unit for the delay + * @param timerService the {@code ScheduledExecutorService} that provides delayed execution + * + * @return an object for checking the status and/or cancel the scheduled task + */ + public final ScheduledHandle scheduleWithFixedDelay( + final Runnable task, long initialDelay, long delay, TimeUnit unit, ScheduledExecutorService timerService) { + final ManagedRunnable runnable = new ManagedRunnable(task); + ScheduledFuture future = timerService.scheduleWithFixedDelay(new Runnable() { + @Override + public void run() { + execute(runnable); + } + + @Override + public String toString() { + return task.toString() + "(scheduled in SynchronizationContext)"; + } + }, initialDelay, delay, unit); + return new ScheduledHandle(runnable, future); + } + + private static class ManagedRunnable implements Runnable { final Runnable task; boolean isCancelled; diff --git a/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java b/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java new file mode 100644 index 00000000000..df2bc75f4b6 --- /dev/null +++ b/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java @@ -0,0 +1,714 @@ +/* + * Copyright 2022 The gRPC Authors + * + * Licensed 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 io.grpc.util; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.base.Preconditions.checkState; + +import com.google.common.collect.Iterables; +import io.grpc.ClientStreamTracer; +import io.grpc.ClientStreamTracer.StreamInfo; +import io.grpc.ConnectivityStateInfo; +import io.grpc.EquivalentAddressGroup; +import io.grpc.LoadBalancer; +import io.grpc.Metadata; +import io.grpc.Status; +import io.grpc.SynchronizationContext; +import io.grpc.SynchronizationContext.ScheduledHandle; +import io.grpc.internal.ServiceConfigUtil.PolicySelection; +import java.time.Clock; +import java.time.Duration; +import java.time.Instant; +import java.time.temporal.ChronoUnit; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; +import javax.annotation.Nullable; + +/** + * Wraps a child {@code LoadBalancer} while monitoring for outliers backends and removing them from + * use by the child LB. + * + *

This implements the outlier detection gRFC: + * https://github.com/grpc/proposal/blob/master/A50-xds-outlier-detection.md + */ +public class OutlierDetectionLoadBalancer extends LoadBalancer { + + private final Helper helper; + private final SynchronizationContext syncContext; + private final Helper childHelper; + private final GracefulSwitchLoadBalancer switchLb; + private final Map eagTrackerMap; + private Clock clock; + private final ScheduledExecutorService timeService; + private ScheduledHandle detectionTimerHandle; + private Instant detectionTimerStartInstant; + + public OutlierDetectionLoadBalancer(Helper helper) { + this.helper = checkNotNull(helper, "helper"); + childHelper = new ChildHelper(helper); + switchLb = new GracefulSwitchLoadBalancer(childHelper); + eagTrackerMap = new HashMap(); + this.syncContext = checkNotNull(helper.getSynchronizationContext(), "syncContext"); + this.timeService = checkNotNull(helper.getScheduledExecutorService(), "timeService"); + clock = Clock.systemDefaultZone(); + } + + @Override + public boolean acceptResolvedAddresses(ResolvedAddresses resolvedAddresses) { + OutlierDetectionLoadBalancerConfig config + = (OutlierDetectionLoadBalancerConfig) resolvedAddresses.getLoadBalancingPolicyConfig(); + + // The map should only retain entries for addresses in this latest update. + eagTrackerMap.keySet().retainAll(resolvedAddresses.getAddresses()); + + // Add any new ones. + for (EquivalentAddressGroup eag : resolvedAddresses.getAddresses()) { + eagTrackerMap.putIfAbsent(eag, new EquivalentAddressGroupTracker(config)); + } + + switchLb.switchTo(config.childPolicy.getProvider()); + + // If outlier detection is actually configured, start a timer that will periodically try to + // detect outliers. + if (config.outlierDetectionEnabled()) { + Duration initialDelay; + + if (detectionTimerHandle == null) { + // On the first go we use the configured interval. + initialDelay = config.interval; + + // When starting the timer for the first time we reset all call counters for a clean start. + eagTrackerMap.values().forEach(EquivalentAddressGroupTracker::clearCallCounters); + } else { + // If a timer has been started earlier we cancel it and use the difference between the start + // time and now as the interval. + detectionTimerHandle.cancel(); + initialDelay = Duration.ofMillis(Math.max(0L, + config.interval.minus(Duration.between(detectionTimerStartInstant, clock.instant())) + .toMillis())); + } + + detectionTimerHandle = syncContext.scheduleWithFixedDelay(new DetectionTimer(config), + initialDelay.toMillis(), config.interval.toMillis(), + TimeUnit.MILLISECONDS, timeService); + } else if (detectionTimerHandle != null) { + // Outlier detection is not configured, but we have a lingering timer. Let's cancel it and + // uneject any addresses we may have ejected. + detectionTimerHandle.cancel(); + detectionTimerStartInstant = null; + for (EquivalentAddressGroupTracker tracker : eagTrackerMap.values()) { + if (tracker.isEjected()) { + tracker.uneject(); + } + tracker.resetEjectionTimeMultiplier(); + } + } + + return switchLb.acceptResolvedAddresses(resolvedAddresses); + } + + @Override + public void handleNameResolutionError(Status error) { + switchLb.handleNameResolutionError(error); + } + + @Override + public void shutdown() { + switchLb.shutdown(); + } + + /** + * This timer will be invoked periodically, according to configuration, and it will look for any + * outlier subchannels. + */ + class DetectionTimer implements Runnable { + + OutlierDetectionLoadBalancerConfig config; + + DetectionTimer(OutlierDetectionLoadBalancerConfig config) { + this.config = config; + } + + @Override + public void run() { + detectionTimerStartInstant = clock.instant(); + + eagTrackerMap.values().forEach(EquivalentAddressGroupTracker::swapCounters); + + OutlierEjectionAlgorithm.forConfig(config) + .ejectOutliers(eagTrackerMap, detectionTimerStartInstant); + + for (EquivalentAddressGroupTracker tracker : eagTrackerMap.values()) { + if (!tracker.isEjected() && tracker.ejectionTimeMultiplier.get() > 0) { + tracker.decrementEjectionTimeMultiplier(); + } + + if (tracker.isEjected() && tracker.maxEjectionTimeElapsed(detectionTimerStartInstant)) { + tracker.uneject(); + } + } + } + } + + /** + * This child helper wraps the provided helper so that is can hand out wrapped {@link + * OutlierDetectionSubchannel}s and manage the address info map. + */ + class ChildHelper extends ForwardingLoadBalancerHelper { + + private Helper delegate; + + ChildHelper(Helper delegate) { + this.delegate = delegate; + } + + @Override + protected Helper delegate() { + return delegate; + } + + @Override + public Subchannel createSubchannel(CreateSubchannelArgs args) { + // Subchannels are wrapped so that we can monitor call results and to trigger failures when + // we decide to eject the subchannel. + OutlierDetectionSubchannel subchannel = new OutlierDetectionSubchannel( + delegate.createSubchannel(args)); + + // If the subchannel is associated with a single address that is also already in the map + // the subchannel will be added to the map and be included in outlier detection. + List allAddresses = subchannel.getAllAddresses(); + if (allAddresses.size() == 1 && eagTrackerMap.containsKey(allAddresses.get(0))) { + EquivalentAddressGroupTracker eagInfo = eagTrackerMap.get(allAddresses.get(0)); + subchannel.setEquivalentAddressGroupInfo(eagInfo); + eagInfo.addSubchannel(subchannel); + + // If this address has already been ejected, we need to immediately eject this Subchannel. + if (eagInfo.ejectionInstant != null) { + subchannel.eject(); + } + } + + return subchannel; + } + } + + class OutlierDetectionSubchannel extends ForwardingSubchannel { + + private final Subchannel delegate; + private EquivalentAddressGroupTracker eagInfo; + private boolean ejected; + private ConnectivityStateInfo lastSubchannelState; + private OutlierDetectionSubchannelStateListener subchannelStateListener; + + OutlierDetectionSubchannel(Subchannel delegate) { + this.delegate = delegate; + } + + @Override + public void start(SubchannelStateListener listener) { + subchannelStateListener = new OutlierDetectionSubchannelStateListener(listener); + super.start(subchannelStateListener); + } + + @Override + public void updateAddresses(List addresses) { + // Outlier detection only supports subchannels with a single address, but the list of + // addresses associated with a subchannel can change at any time, so we need to react to + // changes in the address list plurality. + + // No change in address plurality, we replace the single one with a new one. + if (getAllAddresses().size() == 1 && addresses.size() == 1) { + // Remove the current subchannel from the old address it is associated with in the map. + if (eagTrackerMap.containsKey(getAddresses())) { + eagTrackerMap.get(getAddresses()).removeSubchannel(this); + } + + // If the map has an entry for the new address, we associate this subchannel with it. + EquivalentAddressGroup newAddress = Iterables.getOnlyElement(addresses); + if (eagTrackerMap.containsKey(newAddress)) { + EquivalentAddressGroupTracker tracker = eagTrackerMap.get(newAddress); + tracker.addSubchannel(this); + + // Make sure that the subchannel is in the same ejection state as the new tracker it is + // associated with. + if (tracker.isEjected() && !ejected) { + eject(); + } else if (!tracker.isEjected() && ejected) { + uneject(); + } + } + } else if (getAllAddresses().size() == 1 && addresses.size() > 1) { + // We go from a single address to having multiple, making this subchannel uneligible for + // outlier detection. Remove it from all trackers and reset the call counters of all the + // associated trackers. + // Remove the current subchannel from the old address it is associated with in the map. + if (eagTrackerMap.containsKey(getAddresses())) { + EquivalentAddressGroupTracker tracker = eagTrackerMap.get(getAddresses()); + tracker.removeSubchannel(this); + tracker.clearCallCounters(); + } + } else if (getAllAddresses().size() > 1 && addresses.size() == 1) { + // If the map has an entry for the new address, we associate this subchannel with it. + EquivalentAddressGroup eag = Iterables.getOnlyElement(addresses); + if (eagTrackerMap.containsKey(eag)) { + EquivalentAddressGroupTracker tracker = eagTrackerMap.get(eag); + tracker.addSubchannel(this); + + // If the new address is already in the ejected state, we should also eject this + // subchannel. + if (tracker.isEjected()) { + eject(); + } + } + } + + // We could also have multiple addresses and get an update for multiple new ones. This is + // a no-op as we will just continue to ignore multiple address subchannels. + + super.updateAddresses(addresses); + } + + /** + * If the {@link Subchannel} is considered for outlier detection the associated {@link + * EquivalentAddressGroupTracker} should be set. + */ + void setEquivalentAddressGroupInfo(EquivalentAddressGroupTracker eagInfo) { + this.eagInfo = eagInfo; + } + + void eject() { + ejected = true; + subchannelStateListener.onSubchannelState( + ConnectivityStateInfo.forTransientFailure(Status.UNAVAILABLE)); + } + + void uneject() { + ejected = false; + if (lastSubchannelState != null) { + subchannelStateListener.onSubchannelState(lastSubchannelState); + } + } + + @Override + protected Subchannel delegate() { + return delegate; + } + + /** + * Wraps the actual listener so that state changes from the actual one can be intercepted. + */ + class OutlierDetectionSubchannelStateListener implements SubchannelStateListener { + + private final SubchannelStateListener delegate; + + OutlierDetectionSubchannelStateListener(SubchannelStateListener delegate) { + this.delegate = delegate; + } + + @Override + public void onSubchannelState(ConnectivityStateInfo newState) { + if (!ejected) { + lastSubchannelState = newState; + delegate.onSubchannelState(newState); + } + } + } + } + + + /** + * This picker delegates the actual picking logic to a wrapped delegate, but associates a {@link + * ClientStreamTracer} with each pick to track the results of each subchannel stream. + */ + class OutlierDetectionPicker extends SubchannelPicker { + + private final SubchannelPicker delegate; + + OutlierDetectionPicker(SubchannelPicker delegate) { + this.delegate = delegate; + } + + @Override + public PickResult pickSubchannel(PickSubchannelArgs args) { + PickResult pickResult = delegate.pickSubchannel(args); + + // Because we wrap the helper used by the delegate LB we are assured that the subchannel + // picked here will be an instance of our OutlierDetectionSubchannel. + OutlierDetectionSubchannel subchannel = (OutlierDetectionSubchannel) pickResult.getSubchannel(); + + // The subchannel wrapper has served its purpose, we can pass on the wrapped delegate on + // in case another layer of wrapping assumes a particular subchannel sub-type. + return PickResult.withSubchannel(subchannel.delegate(), + new ResultCountingClientStreamTracerFactory(subchannel)); + } + + /** + * Builds instances of {@link ResultCountingClientStreamTracer}. + */ + class ResultCountingClientStreamTracerFactory extends ClientStreamTracer.Factory { + + private final OutlierDetectionSubchannel subchannel; + + ResultCountingClientStreamTracerFactory(OutlierDetectionSubchannel subchannel) { + this.subchannel = subchannel; + } + + @Override + public ClientStreamTracer newClientStreamTracer(StreamInfo info, Metadata headers) { + return new ResultCountingClientStreamTracer(subchannel); + } + } + + /** + * Counts the results (successful/unsuccessful) of a particular {@link + * OutlierDetectionSubchannel}s streams and increments the counter in the associated {@link + * EquivalentAddressGroupTracker} + */ + class ResultCountingClientStreamTracer extends ClientStreamTracer { + + private final OutlierDetectionSubchannel subchannel; + + public ResultCountingClientStreamTracer(OutlierDetectionSubchannel subchannel) { + this.subchannel = subchannel; + } + + @Override + public void streamClosed(Status status) { + subchannel.eagInfo.incrementCallCount(status.isOk()); + } + } + } + + /** + * Tracks additional information about a set of equivalent addresses needed for outlier + * detection. + */ + class EquivalentAddressGroupTracker { + + private final OutlierDetectionLoadBalancerConfig config; + private CallCounter activeCallCounter = new CallCounter(); + private CallCounter inactiveCallCounter = new CallCounter(); + private Instant ejectionInstant; + private AtomicInteger ejectionTimeMultiplier; + private final Set subchannels = new HashSet(); + + EquivalentAddressGroupTracker(OutlierDetectionLoadBalancerConfig config) { + this.config = config; + } + + boolean addSubchannel(OutlierDetectionSubchannel subchannel) { + return subchannels.add(subchannel); + } + + boolean removeSubchannel(OutlierDetectionSubchannel subchannel) { + return subchannels.remove(subchannel); + } + + boolean containsSubchannel(OutlierDetectionSubchannel subchannel) { + return subchannels.contains(subchannel); + } + + void incrementCallCount(boolean success) { + // If neither algorithm is configured, no point in incrementing counters. + if (config.successRateEjection == null && config.failurePercentageEjection == null) { + return; + } + + if (success) { + activeCallCounter.successCount.getAndIncrement(); + } else { + activeCallCounter.failureCount.getAndIncrement(); + } + } + + /** + * The total number of calls in the active call counter. + */ + long volume() { + return activeCallCounter.successCount.get() + activeCallCounter.failureCount.get(); + } + + void clearCallCounters() { + activeCallCounter.successCount.set(0); + activeCallCounter.failureCount.set(0); + inactiveCallCounter.successCount.set(0); + inactiveCallCounter.failureCount.set(0); + } + + void decrementEjectionTimeMultiplier() { + // The multiplier should not go negative. + ejectionTimeMultiplier.updateAndGet(value -> value > 0 ? value : 0); + } + + void resetEjectionTimeMultiplier() { + ejectionTimeMultiplier.set(0); + } + + void swapCounters() { + CallCounter tempCounter = activeCallCounter; + activeCallCounter = inactiveCallCounter; + inactiveCallCounter = activeCallCounter; + } + + /** + * Ejects the address from use. + */ + void eject(Instant ejectionInstant) { + this.ejectionInstant = ejectionInstant; + ejectionTimeMultiplier.getAndIncrement(); + subchannels.forEach(OutlierDetectionSubchannel::eject); + } + + /** + * Uneject a currently ejected address. + */ + void uneject() { + checkState(ejectionInstant == null, "not currently ejected"); + ejectionInstant = null; + subchannels.forEach(OutlierDetectionSubchannel::uneject); + } + + boolean isEjected() { + return ejectionInstant != null; + } + + public boolean maxEjectionTimeElapsed(Instant now) { + // The instant in time beyond which the address should no longer be ejected. Also making sure + // we honor any maximum ejection time setting. + Instant maxEjectionInstant = ejectionInstant.plus( + Math.min(config.baseEjectionTime.multipliedBy( + ejectionTimeMultiplier.get()).toMillis(), + Math.max(config.baseEjectionTime.toMillis(), config.maxEjectionTime.toMillis())), + ChronoUnit.MILLIS); + return now.isAfter(maxEjectionInstant); + } + + private class CallCounter { + + AtomicInteger successCount; + AtomicInteger failureCount; + } + } + + /** + * Implementations provide different ways of ejecting outlier addresses.. + */ + interface OutlierEjectionAlgorithm { + + /** + * Is the given {@link EquivalentAddressGroup} an outlier based on the past call results stored + * in {@link EquivalentAddressGroupTracker}. + */ + void ejectOutliers( + Map eagTrackerMap, + Instant ejectionInstant); + + @Nullable + static OutlierEjectionAlgorithm forConfig(OutlierDetectionLoadBalancerConfig config) { + if (config.successRateEjection != null) { + return new SuccessRateOutlierEjectionAlgorithm(); + } else if (config.failurePercentageEjection != null) { + return new FailurePercentageOutlierEjectionAlgorithm(); + } else { + return null; + } + } + } + + static class SuccessRateOutlierEjectionAlgorithm implements OutlierEjectionAlgorithm { + + private final OutlierDetectionLoadBalancerConfig config; + + SuccessRateOutlierEjectionAlgorithm(OutlierDetectionLoadBalancerConfig config) { + checkArgument(config.successRateEjection != null, "success rate ejection config is null"); + this.config = config; + } + + @Override + public void ejectOutliers( + Map eagTrackerMap, + Instant ejectionInstant) { + + // Only consider addresses that have the minimum request volume specified in the config. + List trackersWithVolume = eagTrackerMap.values().stream() + .filter(tracker -> tracker.volume() >= config.successRateEjection.requestVolume) + .collect(Collectors.toList()); + + // If we don't have enough addresses with significant volume then there's nothing to do. + if (trackersWithVolume.size() < config.successRateEjection.minimumHosts + || trackersWithVolume.size() == 0) { + return; + } + + // Calculate mean and standard deviation of the successful calls. + double mean = trackersWithVolume.stream() + .mapToInt(tracker -> tracker.activeCallCounter.successCount.get()) + .average() + .getAsDouble(); + double variance = trackersWithVolume.stream() + .map(tracker -> tracker.activeCallCounter.successCount.get() - mean) + .map(difference -> difference * difference).mapToDouble(difference -> difference) + .average() + .getAsDouble(); + double stdev = Math.sqrt(variance); + + for (EquivalentAddressGroupTracker tracker : eagTrackerMap.values()) { + // If we have already ejected addresses past the max percentage, stop here + double ejectedPercentage = eagTrackerMap.values().stream() + .mapToInt(t -> t.isEjected() ? 1 : 0).summaryStatistics().getAverage(); + if (ejectedPercentage > config.maxEjectionPercent) { + return; + } + + // If this address does not have enough volume to be considered, skip to the next one. + if (tracker.volume() < config.successRateEjection.requestVolume) { + continue; + } + + // If success rate is below the threshold, eject the address. + double successRate = tracker.activeCallCounter.successCount.get() / tracker.volume(); + if (successRate < mean - stdev * (config.successRateEjection.stdevFactor / 1000)) { + // Only eject some addresses based on the enforcement percentage. + if (new Random().nextInt(100) < config.successRateEjection.enforcementPercentage) { + tracker.eject(ejectionInstant); + } + } + } + } + } + + static class FailurePercentageOutlierEjectionAlgorithm implements OutlierEjectionAlgorithm { + + private final OutlierDetectionLoadBalancerConfig config; + + FailurePercentageOutlierEjectionAlgorithm(OutlierDetectionLoadBalancerConfig config) { + this.config = config; + } + + @Override + public void ejectOutliers( + Map eagTrackerMap, + Instant ejectionInstant) { + + // If we don't have the minimum amount of addresses the config calls for, then return. + if (eagTrackerMap.size() < config.failurePercentageEjection.minimumHosts) { + return; + } + + // If this address does not have enough volume to be considered, skip to the next one. + for (EquivalentAddressGroupTracker tracker : eagTrackerMap.values()) { + // If we have already ejected addresses past the max percentage, stop here. + double ejectedPercentage = eagTrackerMap.values().stream() + .mapToInt(t -> t.isEjected() ? 1 : 0).summaryStatistics().getAverage(); + if (ejectedPercentage > config.maxEjectionPercent) { + return; + } + + if (tracker.volume() < config.failurePercentageEjection.requestVolume) { + continue; + } + + // If the failure percentage is above the threshold. + long failurePercentage = + (tracker.activeCallCounter.failureCount.get() / tracker.volume()) * 100; + if (failurePercentage > config.failurePercentageEjection.threshold) { + // Only eject some addresses based on the enforcement percentage. + if (new Random().nextInt(100) < config.failurePercentageEjection.enforcementPercentage) { + tracker.eject(ejectionInstant); + } + } + } + } + } + + + /** + * The configuration for {@link OutlierDetectionLoadBalancer}. + */ + static final class OutlierDetectionLoadBalancerConfig { + + final Duration interval; + final Duration baseEjectionTime; + final Duration maxEjectionTime; + final Integer maxEjectionPercent; + final SuccessRateEjection successRateEjection; + final FailurePercentageEjection failurePercentageEjection; + final PolicySelection childPolicy; + + OutlierDetectionLoadBalancerConfig(Duration interval, Duration baseEjectionTime, + Duration maxEjectionTime, Integer maxEjectionPercent, PolicySelection childPolicy, + SuccessRateEjection successRateEjection, + FailurePercentageEjection failurePercentageEjection) { + this.interval = interval != null ? interval : Duration.ofSeconds(10); + this.baseEjectionTime = baseEjectionTime != null ? baseEjectionTime : Duration.ofSeconds(30); + this.maxEjectionTime = maxEjectionTime != null ? maxEjectionTime : Duration.ofSeconds(30); + this.maxEjectionPercent = maxEjectionPercent != null ? maxEjectionPercent : 10; + this.successRateEjection = successRateEjection; + this.failurePercentageEjection = failurePercentageEjection; + this.childPolicy = childPolicy; + } + + class SuccessRateEjection { + + final Integer stdevFactor; + final Integer enforcementPercentage; + final Integer minimumHosts; + final Integer requestVolume; + + SuccessRateEjection(Integer stdevFactor, Integer enforcementPercentage, Integer minimumHosts, + Integer requestVolume) { + this.stdevFactor = stdevFactor != null ? stdevFactor : 1900; + this.enforcementPercentage = enforcementPercentage != null ? enforcementPercentage : 100; + this.minimumHosts = minimumHosts != null ? minimumHosts : 5; + this.requestVolume = requestVolume != null ? requestVolume : 100; + } + } + + class FailurePercentageEjection { + + final Integer threshold; + final Integer enforcementPercentage; + final Integer minimumHosts; + final Integer requestVolume; + + FailurePercentageEjection(Integer threshold, Integer enforcementPercentage, + Integer minimumHosts, Integer requestVolume) { + this.threshold = threshold != null ? threshold : 85; + this.enforcementPercentage = enforcementPercentage != null ? enforcementPercentage : 100; + this.minimumHosts = minimumHosts != null ? minimumHosts : 5; + this.requestVolume = requestVolume != null ? requestVolume : 50; + } + } + + /** + * Determine if outlier detection is at all enabled in this config. + */ + boolean outlierDetectionEnabled() { + // One of the two supported algorithms needs to be configured. + return successRateEjection != null || failurePercentageEjection != null; + } + } +} diff --git a/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java b/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java new file mode 100644 index 00000000000..22ed652fdad --- /dev/null +++ b/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java @@ -0,0 +1,15 @@ +package io.grpc.util; + +import static org.mockito.Mockito.mock; + +import io.grpc.LoadBalancer; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Unit tests for {@link OutlierDetectionLoadBalancer}. */ +@RunWith(JUnit4.class) +public class OutlierDetectionLoadBalancerTest { + + private final LoadBalancer mockDelegate = mock(LoadBalancer.class); + +} From 54ed07aeb59260c7c8943c76192f497ed4c4359d Mon Sep 17 00:00:00 2001 From: Terry Wilson Date: Wed, 10 Aug 2022 13:59:52 -0700 Subject: [PATCH 02/18] Tests for address updates. --- .../java/io/grpc/SynchronizationContext.java | 3 +- .../util/OutlierDetectionLoadBalancer.java | 230 ++++++----- .../test/java/io/grpc/internal/FakeClock.java | 25 +- .../OutlierDetectionLoadBalancerTest.java | 363 +++++++++++++++++- 4 files changed, 517 insertions(+), 104 deletions(-) diff --git a/api/src/main/java/io/grpc/SynchronizationContext.java b/api/src/main/java/io/grpc/SynchronizationContext.java index 7dbd6e5b724..5fb59f14fe9 100644 --- a/api/src/main/java/io/grpc/SynchronizationContext.java +++ b/api/src/main/java/io/grpc/SynchronizationContext.java @@ -176,7 +176,8 @@ public String toString() { * @return an object for checking the status and/or cancel the scheduled task */ public final ScheduledHandle scheduleWithFixedDelay( - final Runnable task, long initialDelay, long delay, TimeUnit unit, ScheduledExecutorService timerService) { + final Runnable task, long initialDelay, long delay, TimeUnit unit, + ScheduledExecutorService timerService) { final ManagedRunnable runnable = new ManagedRunnable(task); ScheduledFuture future = timerService.scheduleWithFixedDelay(new Runnable() { @Override diff --git a/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java b/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java index df2bc75f4b6..6d3549a0a87 100644 --- a/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java +++ b/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java @@ -19,6 +19,8 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; +import static java.util.concurrent.TimeUnit.NANOSECONDS; +import static java.util.concurrent.TimeUnit.SECONDS; import com.google.common.collect.Iterables; import io.grpc.ClientStreamTracer; @@ -31,10 +33,8 @@ import io.grpc.SynchronizationContext; import io.grpc.SynchronizationContext.ScheduledHandle; import io.grpc.internal.ServiceConfigUtil.PolicySelection; -import java.time.Clock; -import java.time.Duration; -import java.time.Instant; -import java.time.temporal.ChronoUnit; +import io.grpc.internal.TimeProvider; +import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -44,7 +44,6 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; -import java.util.stream.Collectors; import javax.annotation.Nullable; /** @@ -56,28 +55,29 @@ */ public class OutlierDetectionLoadBalancer extends LoadBalancer { - private final Helper helper; private final SynchronizationContext syncContext; private final Helper childHelper; private final GracefulSwitchLoadBalancer switchLb; private final Map eagTrackerMap; - private Clock clock; + private TimeProvider timeProvider; private final ScheduledExecutorService timeService; private ScheduledHandle detectionTimerHandle; - private Instant detectionTimerStartInstant; + private Long detectionTimerStartNanos; - public OutlierDetectionLoadBalancer(Helper helper) { - this.helper = checkNotNull(helper, "helper"); - childHelper = new ChildHelper(helper); + /** + * Creates a new instance of {@link OutlierDetectionLoadBalancer}. + */ + public OutlierDetectionLoadBalancer(Helper helper, TimeProvider timeProvider) { + childHelper = new ChildHelper(checkNotNull(helper, "helper")); switchLb = new GracefulSwitchLoadBalancer(childHelper); - eagTrackerMap = new HashMap(); + eagTrackerMap = new HashMap<>(); this.syncContext = checkNotNull(helper.getSynchronizationContext(), "syncContext"); this.timeService = checkNotNull(helper.getScheduledExecutorService(), "timeService"); - clock = Clock.systemDefaultZone(); + this.timeProvider = timeProvider; } @Override - public boolean acceptResolvedAddresses(ResolvedAddresses resolvedAddresses) { + public void handleResolvedAddresses(ResolvedAddresses resolvedAddresses) { OutlierDetectionLoadBalancerConfig config = (OutlierDetectionLoadBalancerConfig) resolvedAddresses.getLoadBalancingPolicyConfig(); @@ -86,7 +86,9 @@ public boolean acceptResolvedAddresses(ResolvedAddresses resolvedAddresses) { // Add any new ones. for (EquivalentAddressGroup eag : resolvedAddresses.getAddresses()) { - eagTrackerMap.putIfAbsent(eag, new EquivalentAddressGroupTracker(config)); + if (!eagTrackerMap.containsKey(eag)) { + eagTrackerMap.put(eag, new EquivalentAddressGroupTracker(config)); + } } switchLb.switchTo(config.childPolicy.getProvider()); @@ -94,31 +96,33 @@ public boolean acceptResolvedAddresses(ResolvedAddresses resolvedAddresses) { // If outlier detection is actually configured, start a timer that will periodically try to // detect outliers. if (config.outlierDetectionEnabled()) { - Duration initialDelay; + Long initialDelayNanos; - if (detectionTimerHandle == null) { + if (detectionTimerStartNanos == null) { // On the first go we use the configured interval. - initialDelay = config.interval; - - // When starting the timer for the first time we reset all call counters for a clean start. - eagTrackerMap.values().forEach(EquivalentAddressGroupTracker::clearCallCounters); + initialDelayNanos = TimeUnit.SECONDS.toNanos(config.intervalSecs); } else { - // If a timer has been started earlier we cancel it and use the difference between the start + // If a timer has started earlier we cancel it and use the difference between the start // time and now as the interval. - detectionTimerHandle.cancel(); - initialDelay = Duration.ofMillis(Math.max(0L, - config.interval.minus(Duration.between(detectionTimerStartInstant, clock.instant())) - .toMillis())); + initialDelayNanos = Math.max(0L, + TimeUnit.SECONDS.toNanos(config.intervalSecs) - (timeProvider.currentTimeNanos() + - detectionTimerStartNanos)); } + if (detectionTimerHandle != null) { + detectionTimerHandle.cancel(); + // When starting the timer for the first time we reset all call counters for a clean start. + for (EquivalentAddressGroupTracker tracker : eagTrackerMap.values()) { + tracker.clearCallCounters(); + } + } detectionTimerHandle = syncContext.scheduleWithFixedDelay(new DetectionTimer(config), - initialDelay.toMillis(), config.interval.toMillis(), - TimeUnit.MILLISECONDS, timeService); + initialDelayNanos, SECONDS.toNanos(config.intervalSecs), NANOSECONDS, timeService); } else if (detectionTimerHandle != null) { // Outlier detection is not configured, but we have a lingering timer. Let's cancel it and // uneject any addresses we may have ejected. detectionTimerHandle.cancel(); - detectionTimerStartInstant = null; + detectionTimerStartNanos = null; for (EquivalentAddressGroupTracker tracker : eagTrackerMap.values()) { if (tracker.isEjected()) { tracker.uneject(); @@ -127,7 +131,7 @@ public boolean acceptResolvedAddresses(ResolvedAddresses resolvedAddresses) { } } - return switchLb.acceptResolvedAddresses(resolvedAddresses); + switchLb.handleResolvedAddresses(resolvedAddresses); } @Override @@ -154,19 +158,21 @@ class DetectionTimer implements Runnable { @Override public void run() { - detectionTimerStartInstant = clock.instant(); + detectionTimerStartNanos = timeProvider.currentTimeNanos(); - eagTrackerMap.values().forEach(EquivalentAddressGroupTracker::swapCounters); + for (EquivalentAddressGroupTracker tracker : eagTrackerMap.values()) { + tracker.swapCounters(); + } OutlierEjectionAlgorithm.forConfig(config) - .ejectOutliers(eagTrackerMap, detectionTimerStartInstant); + .ejectOutliers(eagTrackerMap, detectionTimerStartNanos); for (EquivalentAddressGroupTracker tracker : eagTrackerMap.values()) { - if (!tracker.isEjected() && tracker.ejectionTimeMultiplier.get() > 0) { + if (!tracker.isEjected()) { tracker.decrementEjectionTimeMultiplier(); } - if (tracker.isEjected() && tracker.maxEjectionTimeElapsed(detectionTimerStartInstant)) { + if (tracker.isEjected() && tracker.maxEjectionTimeElapsed(detectionTimerStartNanos)) { tracker.uneject(); } } @@ -206,7 +212,7 @@ public Subchannel createSubchannel(CreateSubchannelArgs args) { eagInfo.addSubchannel(subchannel); // If this address has already been ejected, we need to immediately eject this Subchannel. - if (eagInfo.ejectionInstant != null) { + if (eagInfo.ejectionTimeNanos != null) { subchannel.eject(); } } @@ -357,7 +363,8 @@ public PickResult pickSubchannel(PickSubchannelArgs args) { // Because we wrap the helper used by the delegate LB we are assured that the subchannel // picked here will be an instance of our OutlierDetectionSubchannel. - OutlierDetectionSubchannel subchannel = (OutlierDetectionSubchannel) pickResult.getSubchannel(); + OutlierDetectionSubchannel subchannel + = (OutlierDetectionSubchannel) pickResult.getSubchannel(); // The subchannel wrapper has served its purpose, we can pass on the wrapped delegate on // in case another layer of wrapping assumes a particular subchannel sub-type. @@ -385,7 +392,7 @@ public ClientStreamTracer newClientStreamTracer(StreamInfo info, Metadata header /** * Counts the results (successful/unsuccessful) of a particular {@link * OutlierDetectionSubchannel}s streams and increments the counter in the associated {@link - * EquivalentAddressGroupTracker} + * EquivalentAddressGroupTracker}. */ class ResultCountingClientStreamTracer extends ClientStreamTracer { @@ -406,14 +413,14 @@ public void streamClosed(Status status) { * Tracks additional information about a set of equivalent addresses needed for outlier * detection. */ - class EquivalentAddressGroupTracker { + static class EquivalentAddressGroupTracker { private final OutlierDetectionLoadBalancerConfig config; private CallCounter activeCallCounter = new CallCounter(); private CallCounter inactiveCallCounter = new CallCounter(); - private Instant ejectionInstant; - private AtomicInteger ejectionTimeMultiplier; - private final Set subchannels = new HashSet(); + private Long ejectionTimeNanos; + private int ejectionTimeMultiplier; + private final Set subchannels = new HashSet<>(); EquivalentAddressGroupTracker(OutlierDetectionLoadBalancerConfig config) { this.config = config; @@ -448,7 +455,7 @@ void incrementCallCount(boolean success) { * The total number of calls in the active call counter. */ long volume() { - return activeCallCounter.successCount.get() + activeCallCounter.failureCount.get(); + return (long) activeCallCounter.successCount.get() + activeCallCounter.failureCount.get(); } void clearCallCounters() { @@ -460,56 +467,59 @@ void clearCallCounters() { void decrementEjectionTimeMultiplier() { // The multiplier should not go negative. - ejectionTimeMultiplier.updateAndGet(value -> value > 0 ? value : 0); + ejectionTimeMultiplier = ejectionTimeMultiplier == 0 ? 0 : ejectionTimeMultiplier - 1; } void resetEjectionTimeMultiplier() { - ejectionTimeMultiplier.set(0); + ejectionTimeMultiplier = 0; } void swapCounters() { CallCounter tempCounter = activeCallCounter; activeCallCounter = inactiveCallCounter; - inactiveCallCounter = activeCallCounter; + inactiveCallCounter = tempCounter; } /** * Ejects the address from use. */ - void eject(Instant ejectionInstant) { - this.ejectionInstant = ejectionInstant; - ejectionTimeMultiplier.getAndIncrement(); - subchannels.forEach(OutlierDetectionSubchannel::eject); + void eject(long ejectionTimeNanos) { + this.ejectionTimeNanos = ejectionTimeNanos; + ejectionTimeMultiplier++; + for (OutlierDetectionSubchannel subchannel : subchannels) { + subchannel.eject(); + } } /** * Uneject a currently ejected address. */ void uneject() { - checkState(ejectionInstant == null, "not currently ejected"); - ejectionInstant = null; - subchannels.forEach(OutlierDetectionSubchannel::uneject); + checkState(ejectionTimeNanos == null, "not currently ejected"); + ejectionTimeNanos = null; + for (OutlierDetectionSubchannel subchannel : subchannels) { + subchannel.uneject(); + } } boolean isEjected() { - return ejectionInstant != null; + return ejectionTimeNanos != null; } - public boolean maxEjectionTimeElapsed(Instant now) { + public boolean maxEjectionTimeElapsed(long currentTimeNanos) { // The instant in time beyond which the address should no longer be ejected. Also making sure // we honor any maximum ejection time setting. - Instant maxEjectionInstant = ejectionInstant.plus( - Math.min(config.baseEjectionTime.multipliedBy( - ejectionTimeMultiplier.get()).toMillis(), - Math.max(config.baseEjectionTime.toMillis(), config.maxEjectionTime.toMillis())), - ChronoUnit.MILLIS); - return now.isAfter(maxEjectionInstant); - } + long maxEjectionTimeNanos = ejectionTimeNanos + Math.min( + SECONDS.toNanos(config.baseEjectionTimeSecs) * ejectionTimeMultiplier, + Math.max(SECONDS.toNanos(config.baseEjectionTimeSecs), + SECONDS.toNanos(config.maxEjectionTimeSecs))); - private class CallCounter { + return currentTimeNanos > maxEjectionTimeNanos; + } - AtomicInteger successCount; - AtomicInteger failureCount; + private static class CallCounter { + AtomicInteger successCount = new AtomicInteger(); + AtomicInteger failureCount = new AtomicInteger(); } } @@ -524,14 +534,14 @@ interface OutlierEjectionAlgorithm { */ void ejectOutliers( Map eagTrackerMap, - Instant ejectionInstant); + long ejectionTimeMillis); @Nullable static OutlierEjectionAlgorithm forConfig(OutlierDetectionLoadBalancerConfig config) { if (config.successRateEjection != null) { - return new SuccessRateOutlierEjectionAlgorithm(); + return new SuccessRateOutlierEjectionAlgorithm(config); } else if (config.failurePercentageEjection != null) { - return new FailurePercentageOutlierEjectionAlgorithm(); + return new FailurePercentageOutlierEjectionAlgorithm(config); } else { return null; } @@ -550,13 +560,15 @@ static class SuccessRateOutlierEjectionAlgorithm implements OutlierEjectionAlgor @Override public void ejectOutliers( Map eagTrackerMap, - Instant ejectionInstant) { + long ejectionTimeNanos) { // Only consider addresses that have the minimum request volume specified in the config. - List trackersWithVolume = eagTrackerMap.values().stream() - .filter(tracker -> tracker.volume() >= config.successRateEjection.requestVolume) - .collect(Collectors.toList()); - + List trackersWithVolume = new ArrayList<>(); + for (EquivalentAddressGroupTracker tracker : eagTrackerMap.values()) { + if (tracker.volume() >= config.successRateEjection.requestVolume) { + trackersWithVolume.add(tracker); + } + } // If we don't have enough addresses with significant volume then there's nothing to do. if (trackersWithVolume.size() < config.successRateEjection.minimumHosts || trackersWithVolume.size() == 0) { @@ -564,21 +576,32 @@ public void ejectOutliers( } // Calculate mean and standard deviation of the successful calls. - double mean = trackersWithVolume.stream() - .mapToInt(tracker -> tracker.activeCallCounter.successCount.get()) - .average() - .getAsDouble(); - double variance = trackersWithVolume.stream() - .map(tracker -> tracker.activeCallCounter.successCount.get() - mean) - .map(difference -> difference * difference).mapToDouble(difference -> difference) - .average() - .getAsDouble(); + int totalSuccessCount = 0; + for (EquivalentAddressGroupTracker tracker : trackersWithVolume) { + totalSuccessCount += tracker.activeCallCounter.successCount.get(); + } + double mean = totalSuccessCount / trackersWithVolume.size(); + + double squaredDifferenceSum = 0; + for (EquivalentAddressGroupTracker tracker : trackersWithVolume) { + double difference = tracker.activeCallCounter.successCount.get() - mean; + squaredDifferenceSum += difference * difference; + } + double variance = squaredDifferenceSum / trackersWithVolume.size(); + double stdev = Math.sqrt(variance); for (EquivalentAddressGroupTracker tracker : eagTrackerMap.values()) { // If we have already ejected addresses past the max percentage, stop here - double ejectedPercentage = eagTrackerMap.values().stream() - .mapToInt(t -> t.isEjected() ? 1 : 0).summaryStatistics().getAverage(); + int totalAddresses = 0; + int ejectedAddresses = 0; + for (EquivalentAddressGroupTracker t : eagTrackerMap.values()) { + totalAddresses++; + if (t.isEjected()) { + ejectedAddresses++; + } + } + double ejectedPercentage = (ejectedAddresses / totalAddresses) * 100; if (ejectedPercentage > config.maxEjectionPercent) { return; } @@ -593,7 +616,7 @@ public void ejectOutliers( if (successRate < mean - stdev * (config.successRateEjection.stdevFactor / 1000)) { // Only eject some addresses based on the enforcement percentage. if (new Random().nextInt(100) < config.successRateEjection.enforcementPercentage) { - tracker.eject(ejectionInstant); + tracker.eject(ejectionTimeNanos); } } } @@ -611,7 +634,7 @@ static class FailurePercentageOutlierEjectionAlgorithm implements OutlierEjectio @Override public void ejectOutliers( Map eagTrackerMap, - Instant ejectionInstant) { + long ejectionTimeMillis) { // If we don't have the minimum amount of addresses the config calls for, then return. if (eagTrackerMap.size() < config.failurePercentageEjection.minimumHosts) { @@ -621,8 +644,16 @@ public void ejectOutliers( // If this address does not have enough volume to be considered, skip to the next one. for (EquivalentAddressGroupTracker tracker : eagTrackerMap.values()) { // If we have already ejected addresses past the max percentage, stop here. - double ejectedPercentage = eagTrackerMap.values().stream() - .mapToInt(t -> t.isEjected() ? 1 : 0).summaryStatistics().getAverage(); + int totalAddresses = 0; + int ejectedAddresses = 0; + for (EquivalentAddressGroupTracker t : eagTrackerMap.values()) { + totalAddresses++; + if (t.isEjected()) { + ejectedAddresses++; + } + } + double ejectedPercentage = (ejectedAddresses / totalAddresses) * 100; + if (ejectedPercentage > config.maxEjectionPercent) { return; } @@ -637,7 +668,7 @@ public void ejectOutliers( if (failurePercentage > config.failurePercentageEjection.threshold) { // Only eject some addresses based on the enforcement percentage. if (new Random().nextInt(100) < config.failurePercentageEjection.enforcementPercentage) { - tracker.eject(ejectionInstant); + tracker.eject(ejectionTimeMillis); } } } @@ -650,28 +681,29 @@ public void ejectOutliers( */ static final class OutlierDetectionLoadBalancerConfig { - final Duration interval; - final Duration baseEjectionTime; - final Duration maxEjectionTime; + final long intervalSecs; + final long baseEjectionTimeSecs; + final long maxEjectionTimeSecs; final Integer maxEjectionPercent; final SuccessRateEjection successRateEjection; final FailurePercentageEjection failurePercentageEjection; final PolicySelection childPolicy; - OutlierDetectionLoadBalancerConfig(Duration interval, Duration baseEjectionTime, - Duration maxEjectionTime, Integer maxEjectionPercent, PolicySelection childPolicy, + OutlierDetectionLoadBalancerConfig(PolicySelection childPolicy, Long intervalSecs, + Long baseEjectionTimeSecs, + Long maxEjectionTimeSecs, Integer maxEjectionPercent, SuccessRateEjection successRateEjection, FailurePercentageEjection failurePercentageEjection) { - this.interval = interval != null ? interval : Duration.ofSeconds(10); - this.baseEjectionTime = baseEjectionTime != null ? baseEjectionTime : Duration.ofSeconds(30); - this.maxEjectionTime = maxEjectionTime != null ? maxEjectionTime : Duration.ofSeconds(30); + this.intervalSecs = intervalSecs != null ? intervalSecs : 10; + this.baseEjectionTimeSecs = baseEjectionTimeSecs != null ? baseEjectionTimeSecs : 30; + this.maxEjectionTimeSecs = maxEjectionTimeSecs != null ? maxEjectionTimeSecs : 30; this.maxEjectionPercent = maxEjectionPercent != null ? maxEjectionPercent : 10; this.successRateEjection = successRateEjection; this.failurePercentageEjection = failurePercentageEjection; this.childPolicy = childPolicy; } - class SuccessRateEjection { + static class SuccessRateEjection { final Integer stdevFactor; final Integer enforcementPercentage; @@ -687,7 +719,7 @@ class SuccessRateEjection { } } - class FailurePercentageEjection { + static class FailurePercentageEjection { final Integer threshold; final Integer enforcementPercentage; diff --git a/core/src/test/java/io/grpc/internal/FakeClock.java b/core/src/test/java/io/grpc/internal/FakeClock.java index f5d22651271..b6a88de00ba 100644 --- a/core/src/test/java/io/grpc/internal/FakeClock.java +++ b/core/src/test/java/io/grpc/internal/FakeClock.java @@ -159,8 +159,10 @@ private void schedule(ScheduledTask task, long delay, TimeUnit unit) { } @Override public ScheduledFuture scheduleWithFixedDelay( - Runnable command, long initialDelay, long delay, TimeUnit unit) { - throw new UnsupportedOperationException(); + Runnable cmd, long initialDelay, long delay, TimeUnit unit) { + ScheduledTask task = new ScheduleWithFixedDelayTask(cmd, delay, unit); + schedule(task, initialDelay, unit); + return task; } @Override public boolean awaitTermination(long timeout, TimeUnit unit) { @@ -234,6 +236,25 @@ public ScheduleAtFixedRateTask(Runnable command, long period, TimeUnit unit) { } } } + + class ScheduleWithFixedDelayTask extends ScheduledTask { + + final long delayNanos; + + ScheduleWithFixedDelayTask(Runnable command, long delay, TimeUnit unit) { + super(command); + this.delayNanos = unit.toNanos(delay); + } + + @Override + void run() { + long startTimeNanos = currentTimeNanos; + command.run(); + if (!isCancelled()) { + schedule(this, delayNanos, TimeUnit.NANOSECONDS); + } + } + } } /** diff --git a/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java b/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java index 22ed652fdad..c26dd0ce9d1 100644 --- a/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java +++ b/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java @@ -1,15 +1,374 @@ +/* + * Copyright 2022 The gRPC Authors + * + * Licensed 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 io.grpc.util; +import static com.google.common.truth.Truth.assertThat; +import static io.grpc.ConnectivityState.CONNECTING; +import static io.grpc.ConnectivityState.READY; +import static org.junit.Assert.assertEquals; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoMoreInteractions; +import static org.mockito.Mockito.when; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import io.grpc.ConnectivityState; +import io.grpc.ConnectivityStateInfo; +import io.grpc.EquivalentAddressGroup; import io.grpc.LoadBalancer; +import io.grpc.LoadBalancer.CreateSubchannelArgs; +import io.grpc.LoadBalancer.Helper; +import io.grpc.LoadBalancer.ResolvedAddresses; +import io.grpc.LoadBalancer.Subchannel; +import io.grpc.LoadBalancer.SubchannelPicker; +import io.grpc.LoadBalancer.SubchannelStateListener; +import io.grpc.LoadBalancerProvider; +import io.grpc.LoadBalancerRegistry; +import io.grpc.Status; +import io.grpc.SynchronizationContext; +import io.grpc.internal.FakeClock; +import io.grpc.internal.FakeClock.ScheduledTask; +import io.grpc.internal.ServiceConfigUtil.PolicySelection; +import io.grpc.internal.TestUtils.StandardLoadBalancerProvider; +import io.grpc.util.OutlierDetectionLoadBalancer.OutlierDetectionLoadBalancerConfig; +import io.grpc.util.OutlierDetectionLoadBalancer.OutlierDetectionLoadBalancerConfig.SuccessRateEjection; +import io.grpc.util.RoundRobinLoadBalancer.ReadyPicker; +import java.net.SocketAddress; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; +import org.mockito.ArgumentCaptor; +import org.mockito.Captor; +import org.mockito.Mock; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.junit.MockitoJUnit; +import org.mockito.junit.MockitoRule; +import org.mockito.stubbing.Answer; -/** Unit tests for {@link OutlierDetectionLoadBalancer}. */ +/** + * Unit tests for {@link OutlierDetectionLoadBalancer}. + */ @RunWith(JUnit4.class) public class OutlierDetectionLoadBalancerTest { - private final LoadBalancer mockDelegate = mock(LoadBalancer.class); + @Rule + public final MockitoRule mockitoRule = MockitoJUnit.rule(); + + @Mock + private Helper helper; + @Mock + private LoadBalancer mockChildLb; + @Mock + private Helper mockHelper; + @Mock + private SocketAddress mockSocketAddress; + + @Captor + private ArgumentCaptor connectivityStateCaptor; + @Captor + private ArgumentCaptor errorPickerCaptor; + @Captor + private ArgumentCaptor pickerCaptor; + @Captor + private ArgumentCaptor stateCaptor; + @Captor + private ArgumentCaptor createArgsCaptor; + + private final LoadBalancerProvider mockChildLbProvider = new StandardLoadBalancerProvider( + "foo_policy") { + @Override + public LoadBalancer newLoadBalancer(Helper helper) { + return mockChildLb; + } + }; + private final LoadBalancerProvider roundRobinLbProvider = new StandardLoadBalancerProvider( + "round_robin") { + @Override + public LoadBalancer newLoadBalancer(Helper helper) { + return roundRobinLoadBalancer; + } + }; + private RoundRobinLoadBalancer roundRobinLoadBalancer; + + private final FakeClock fakeClock = new FakeClock(); + private final SynchronizationContext syncContext = new SynchronizationContext( + new Thread.UncaughtExceptionHandler() { + @Override + public void uncaughtException(Thread t, Throwable e) { + throw new AssertionError(e); + } + }); + private LoadBalancerRegistry lbRegistry = new LoadBalancerRegistry(); + private OutlierDetectionLoadBalancer loadBalancer; + + private final List servers = Lists.newArrayList(); + private final Map, Subchannel> subchannels = Maps.newLinkedHashMap(); + private final Map subchannelStateListeners + = Maps.newLinkedHashMap(); + + @Before + public void setUp() { + for (int i = 0; i < 3; i++) { + SocketAddress addr = new FakeSocketAddress("server" + i); + EquivalentAddressGroup eag = new EquivalentAddressGroup(addr); + servers.add(eag); + Subchannel sc = mock(Subchannel.class); + subchannels.put(Arrays.asList(eag), sc); + } + + when(mockHelper.getSynchronizationContext()).thenReturn(syncContext); + when(mockHelper.getScheduledExecutorService()).thenReturn( + fakeClock.getScheduledExecutorService()); + when(mockHelper.createSubchannel(any(CreateSubchannelArgs.class))) + .then(new Answer() { + @Override + public Subchannel answer(InvocationOnMock invocation) throws Throwable { + CreateSubchannelArgs args = (CreateSubchannelArgs) invocation.getArguments()[0]; + final Subchannel subchannel = subchannels.get(args.getAddresses()); + when(subchannel.getAllAddresses()).thenReturn(args.getAddresses()); + when(subchannel.getAttributes()).thenReturn(args.getAttributes()); + doAnswer( + new Answer() { + @Override + public Void answer(InvocationOnMock invocation) throws Throwable { + subchannelStateListeners.put( + subchannel, (SubchannelStateListener) invocation.getArguments()[0]); + return null; + } + }).when(subchannel).start(any(SubchannelStateListener.class)); + return subchannel; + } + }); + + roundRobinLoadBalancer = new RoundRobinLoadBalancer(mockHelper); + //lbRegistry.register(mockChildLbProvider); + lbRegistry.register(roundRobinLbProvider); + + loadBalancer = new OutlierDetectionLoadBalancer(mockHelper, fakeClock.getTimeProvider()); + } + + @Test + public void handleNameResolutionError_noChildLb() { + loadBalancer.handleNameResolutionError(Status.DEADLINE_EXCEEDED); + + verify(mockHelper).updateBalancingState(connectivityStateCaptor.capture(), + errorPickerCaptor.capture()); + assertThat(connectivityStateCaptor.getValue()).isEqualTo(ConnectivityState.TRANSIENT_FAILURE); + } + + @Test + public void handleNameResolutionError_withChildLb() { + loadBalancer.handleResolvedAddresses(buildResolvedAddress(defaultSuccessRateConfig(mockChildLbProvider), + new EquivalentAddressGroup(mockSocketAddress))); + loadBalancer.handleNameResolutionError(Status.DEADLINE_EXCEEDED); + + verify(mockChildLb).handleNameResolutionError(Status.DEADLINE_EXCEEDED); + } + + /** {@code shutdown()} is simply delegated. */ + @Test + public void shutdown() { + loadBalancer.handleResolvedAddresses(buildResolvedAddress(defaultSuccessRateConfig(mockChildLbProvider), + new EquivalentAddressGroup(mockSocketAddress))); + loadBalancer.shutdown(); + verify(mockChildLb).shutdown(); + } + + /** Base case for accepting new resolved addresses. */ + @Test + public void handleResolvedAddresses() { + OutlierDetectionLoadBalancerConfig config = defaultSuccessRateConfig(mockChildLbProvider); + ResolvedAddresses resolvedAddresses = buildResolvedAddress(config, + new EquivalentAddressGroup(mockSocketAddress)); + + loadBalancer.handleResolvedAddresses(resolvedAddresses); + + // Handling of resolved addresses is delegated + verify(mockChildLb).handleResolvedAddresses(resolvedAddresses); + + // There is a single pending task to run the outlier detection algorithm + assertThat(fakeClock.getPendingTasks()).hasSize(1); + + // The task is scheduled to run after a delay set in the config. + ScheduledTask task = fakeClock.getPendingTasks().iterator().next(); + assertThat(task.getDelay(TimeUnit.SECONDS)).isEqualTo(config.intervalSecs); + } + + /** Outlier detection first enabled, then removed. */ + @Test + public void handleResolvedAddresses_outlierDetectionDisabled() { + OutlierDetectionLoadBalancerConfig config = defaultSuccessRateConfig(mockChildLbProvider); + ResolvedAddresses resolvedAddresses = buildResolvedAddress(config, + new EquivalentAddressGroup(mockSocketAddress)); + + loadBalancer.handleResolvedAddresses(resolvedAddresses); + + fakeClock.forwardTime(15, TimeUnit.SECONDS); + + // There is a single pending task to run the outlier detection algorithm + assertThat(fakeClock.getPendingTasks()).hasSize(1); + + loadBalancer.handleResolvedAddresses( + buildResolvedAddress(disabledConfig(), new EquivalentAddressGroup(mockSocketAddress))); + + // Pending task should be gone since OD is disabled. + assertThat(fakeClock.getPendingTasks()).isEmpty(); + + } + + /** Tests different scenarios when the timer interval in the config changes. */ + @Test + public void handleResolvedAddresses_intervalUpdate() { + OutlierDetectionLoadBalancerConfig config = customIntervalConfig(null); + ResolvedAddresses resolvedAddresses = buildResolvedAddress(config, + new EquivalentAddressGroup(mockSocketAddress)); + + loadBalancer.handleResolvedAddresses(resolvedAddresses); + + // Config update has doubled the interval + config = customIntervalConfig(config.intervalSecs * 2); + + loadBalancer.handleResolvedAddresses( + buildResolvedAddress(config, + new EquivalentAddressGroup(mockSocketAddress))); + + // If the timer has not run yet the task is just rescheduled to run after the new delay. + assertThat(fakeClock.getPendingTasks()).hasSize(1); + ScheduledTask task = fakeClock.getPendingTasks().iterator().next(); + assertThat(task.getDelay(TimeUnit.SECONDS)).isEqualTo(config.intervalSecs); + assertThat(task.dueTimeNanos).isEqualTo(TimeUnit.SECONDS.toNanos(config.intervalSecs)); + + // The new interval time has passed. The next task due time should have been pushed back another + // interval. + fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + assertThat(fakeClock.getPendingTasks()).hasSize(1); + task = fakeClock.getPendingTasks().iterator().next(); + assertThat(task.dueTimeNanos).isEqualTo( + TimeUnit.SECONDS.toNanos(config.intervalSecs + config.intervalSecs + 1)); + + // Some time passes and a second update comes down, but now the timer has had a chance to run, + // the new delay to timer start should consider when the timer last ran and if the interval is + // not changing in the config, the next task due time should remain unchanged. + fakeClock.forwardTime(4, TimeUnit.SECONDS); + task = fakeClock.getPendingTasks().iterator().next(); + loadBalancer.handleResolvedAddresses( + buildResolvedAddress(config, + new EquivalentAddressGroup(mockSocketAddress))); + assertThat(task.dueTimeNanos).isEqualTo( + TimeUnit.SECONDS.toNanos(config.intervalSecs + config.intervalSecs + 1)); + } + + // @Test + // public void pickAfterResolved() throws Exception { + // final Subchannel readySubchannel = subchannels.values().iterator().next(); + // + // loadBalancer.handleResolvedAddresses( + // buildResolvedAddress(defaultSuccessRateConfig(roundRobinLbProvider), servers)); + // + // deliverSubchannelState(readySubchannel, ConnectivityStateInfo.forNonError(READY)); + // + // verify(mockHelper, times(3)).createSubchannel(createArgsCaptor.capture()); + // + // List> capturedAddrs = new ArrayList<>(); + // for (CreateSubchannelArgs arg : createArgsCaptor.getAllValues()) { + // capturedAddrs.add(arg.getAddresses()); + // } + // + // assertThat(capturedAddrs).containsAtLeastElementsIn(subchannels.keySet()); + // for (Subchannel subchannel : subchannels.values()) { + // verify(subchannel).requestConnection(); + // verify(subchannel, never()).shutdown(); + // } + // // + // // verify(mockHelper, times(2)) + // // .updateBalancingState(stateCaptor.capture(), pickerCaptor.capture()); + // // + // // assertEquals(CONNECTING, stateCaptor.getAllValues().get(0)); + // // assertEquals(READY, stateCaptor.getAllValues().get(1)); + // // assertThat(getList(pickerCaptor.getValue())).containsExactly(readySubchannel); + // // + // // verifyNoMoreInteractions(mockHelper); + // } + + + private static class FakeSocketAddress extends SocketAddress { + final String name; + + FakeSocketAddress(String name) { + this.name = name; + } + + @Override + public String toString() { + return "FakeSocketAddress-" + name; + } + } + + private OutlierDetectionLoadBalancerConfig defaultSuccessRateConfig( + LoadBalancerProvider childLbProvider) { + return new OutlierDetectionLoadBalancerConfig(new PolicySelection(childLbProvider, null), + null, null, null, null, + new SuccessRateEjection(null, null, null, null), null); + } + + private OutlierDetectionLoadBalancerConfig customIntervalConfig(Long intervalSecs) { + return new OutlierDetectionLoadBalancerConfig(new PolicySelection(mockChildLbProvider, null), + intervalSecs != null ? intervalSecs : null, null, null, null, + new SuccessRateEjection(null, null, null, null), null); + } + + private OutlierDetectionLoadBalancerConfig disabledConfig() { + return new OutlierDetectionLoadBalancerConfig(new PolicySelection(mockChildLbProvider, null), + null, null, null, null, + null, null); + } + + private ResolvedAddresses buildResolvedAddress(OutlierDetectionLoadBalancerConfig config, + EquivalentAddressGroup... servers) { + return ResolvedAddresses.newBuilder().setAddresses(ImmutableList.copyOf(servers)) + .setLoadBalancingPolicyConfig(config).build(); + } + + private ResolvedAddresses buildResolvedAddress(OutlierDetectionLoadBalancerConfig config, + List servers) { + return ResolvedAddresses.newBuilder().setAddresses(ImmutableList.copyOf(servers)) + .setLoadBalancingPolicyConfig(config).build(); + } + + private void deliverSubchannelState(Subchannel subchannel, ConnectivityStateInfo newState) { + subchannelStateListeners.get(subchannel).onSubchannelState(newState); + } + private static List getList(SubchannelPicker picker) { + return picker instanceof ReadyPicker ? ((ReadyPicker) picker).getList() : + Collections.emptyList(); + } } From ece4f8b3d562f6b8dd33aa4f88d73b3e2bcb3415 Mon Sep 17 00:00:00 2001 From: Terry Wilson Date: Sat, 13 Aug 2022 10:30:52 -0700 Subject: [PATCH 03/18] Refactoring and more tests. --- .../util/OutlierDetectionLoadBalancer.java | 534 +++++++++++++----- .../OutlierDetectionLoadBalancerTest.java | 397 ++++++++++--- 2 files changed, 690 insertions(+), 241 deletions(-) diff --git a/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java b/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java index 6d3549a0a87..721c64cb7e9 100644 --- a/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java +++ b/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java @@ -22,9 +22,13 @@ import static java.util.concurrent.TimeUnit.NANOSECONDS; import static java.util.concurrent.TimeUnit.SECONDS; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ForwardingMap; import com.google.common.collect.Iterables; +import io.grpc.Attributes; import io.grpc.ClientStreamTracer; import io.grpc.ClientStreamTracer.StreamInfo; +import io.grpc.ConnectivityState; import io.grpc.ConnectivityStateInfo; import io.grpc.EquivalentAddressGroup; import io.grpc.LoadBalancer; @@ -35,6 +39,7 @@ import io.grpc.internal.ServiceConfigUtil.PolicySelection; import io.grpc.internal.TimeProvider; import java.util.ArrayList; +import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -43,7 +48,7 @@ import java.util.Set; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import javax.annotation.Nullable; /** @@ -55,22 +60,26 @@ */ public class OutlierDetectionLoadBalancer extends LoadBalancer { + @VisibleForTesting + final AddressTrackerMap trackerMap; + private final SynchronizationContext syncContext; private final Helper childHelper; private final GracefulSwitchLoadBalancer switchLb; - private final Map eagTrackerMap; private TimeProvider timeProvider; private final ScheduledExecutorService timeService; private ScheduledHandle detectionTimerHandle; private Long detectionTimerStartNanos; + private static final Attributes.Key EAG_INFO_ATTR_KEY = Attributes.Key.create("eagInfoKey"); + /** * Creates a new instance of {@link OutlierDetectionLoadBalancer}. */ public OutlierDetectionLoadBalancer(Helper helper, TimeProvider timeProvider) { childHelper = new ChildHelper(checkNotNull(helper, "helper")); switchLb = new GracefulSwitchLoadBalancer(childHelper); - eagTrackerMap = new HashMap<>(); + trackerMap = new AddressTrackerMap(); this.syncContext = checkNotNull(helper.getSynchronizationContext(), "syncContext"); this.timeService = checkNotNull(helper.getScheduledExecutorService(), "timeService"); this.timeProvider = timeProvider; @@ -82,14 +91,10 @@ public void handleResolvedAddresses(ResolvedAddresses resolvedAddresses) { = (OutlierDetectionLoadBalancerConfig) resolvedAddresses.getLoadBalancingPolicyConfig(); // The map should only retain entries for addresses in this latest update. - eagTrackerMap.keySet().retainAll(resolvedAddresses.getAddresses()); + trackerMap.keySet().retainAll(resolvedAddresses.getAddresses()); // Add any new ones. - for (EquivalentAddressGroup eag : resolvedAddresses.getAddresses()) { - if (!eagTrackerMap.containsKey(eag)) { - eagTrackerMap.put(eag, new EquivalentAddressGroupTracker(config)); - } - } + trackerMap.putNewTrackers(config, resolvedAddresses.getAddresses()); switchLb.switchTo(config.childPolicy.getProvider()); @@ -109,13 +114,13 @@ public void handleResolvedAddresses(ResolvedAddresses resolvedAddresses) { - detectionTimerStartNanos)); } + // If a timer has been previously created we need to cancel it and reset all the call counters + // for a fresh start. if (detectionTimerHandle != null) { detectionTimerHandle.cancel(); - // When starting the timer for the first time we reset all call counters for a clean start. - for (EquivalentAddressGroupTracker tracker : eagTrackerMap.values()) { - tracker.clearCallCounters(); - } + trackerMap.resetCallCounters(); } + detectionTimerHandle = syncContext.scheduleWithFixedDelay(new DetectionTimer(config), initialDelayNanos, SECONDS.toNanos(config.intervalSecs), NANOSECONDS, timeService); } else if (detectionTimerHandle != null) { @@ -123,12 +128,7 @@ public void handleResolvedAddresses(ResolvedAddresses resolvedAddresses) { // uneject any addresses we may have ejected. detectionTimerHandle.cancel(); detectionTimerStartNanos = null; - for (EquivalentAddressGroupTracker tracker : eagTrackerMap.values()) { - if (tracker.isEjected()) { - tracker.uneject(); - } - tracker.resetEjectionTimeMultiplier(); - } + trackerMap.cancelTracking(); } switchLb.handleResolvedAddresses(resolvedAddresses); @@ -160,22 +160,12 @@ class DetectionTimer implements Runnable { public void run() { detectionTimerStartNanos = timeProvider.currentTimeNanos(); - for (EquivalentAddressGroupTracker tracker : eagTrackerMap.values()) { - tracker.swapCounters(); - } + trackerMap.swapCounters(); OutlierEjectionAlgorithm.forConfig(config) - .ejectOutliers(eagTrackerMap, detectionTimerStartNanos); - - for (EquivalentAddressGroupTracker tracker : eagTrackerMap.values()) { - if (!tracker.isEjected()) { - tracker.decrementEjectionTimeMultiplier(); - } + .ejectOutliers(trackerMap, detectionTimerStartNanos); - if (tracker.isEjected() && tracker.maxEjectionTimeElapsed(detectionTimerStartNanos)) { - tracker.uneject(); - } - } + trackerMap.maybeUnejectOutliers(detectionTimerStartNanos); } } @@ -206,8 +196,8 @@ public Subchannel createSubchannel(CreateSubchannelArgs args) { // If the subchannel is associated with a single address that is also already in the map // the subchannel will be added to the map and be included in outlier detection. List allAddresses = subchannel.getAllAddresses(); - if (allAddresses.size() == 1 && eagTrackerMap.containsKey(allAddresses.get(0))) { - EquivalentAddressGroupTracker eagInfo = eagTrackerMap.get(allAddresses.get(0)); + if (allAddresses.size() == 1 && trackerMap.containsKey(allAddresses.get(0))) { + AddressTracker eagInfo = trackerMap.get(allAddresses.get(0)); subchannel.setEquivalentAddressGroupInfo(eagInfo); eagInfo.addSubchannel(subchannel); @@ -219,12 +209,17 @@ public Subchannel createSubchannel(CreateSubchannelArgs args) { return subchannel; } + + @Override + public void updateBalancingState(ConnectivityState newState, SubchannelPicker newPicker) { + delegate.updateBalancingState(newState, new OutlierDetectionPicker(newPicker)); + } } class OutlierDetectionSubchannel extends ForwardingSubchannel { private final Subchannel delegate; - private EquivalentAddressGroupTracker eagInfo; + private AddressTracker eagInfo; private boolean ejected; private ConnectivityStateInfo lastSubchannelState; private OutlierDetectionSubchannelStateListener subchannelStateListener; @@ -239,6 +234,11 @@ public void start(SubchannelStateListener listener) { super.start(subchannelStateListener); } + @Override + public Attributes getAttributes() { + return delegate.getAttributes().toBuilder().set(EAG_INFO_ATTR_KEY, eagInfo).build(); + } + @Override public void updateAddresses(List addresses) { // Outlier detection only supports subchannels with a single address, but the list of @@ -248,21 +248,21 @@ public void updateAddresses(List addresses) { // No change in address plurality, we replace the single one with a new one. if (getAllAddresses().size() == 1 && addresses.size() == 1) { // Remove the current subchannel from the old address it is associated with in the map. - if (eagTrackerMap.containsKey(getAddresses())) { - eagTrackerMap.get(getAddresses()).removeSubchannel(this); + if (trackerMap.containsKey(getAddresses())) { + trackerMap.get(getAddresses()).removeSubchannel(this); } // If the map has an entry for the new address, we associate this subchannel with it. EquivalentAddressGroup newAddress = Iterables.getOnlyElement(addresses); - if (eagTrackerMap.containsKey(newAddress)) { - EquivalentAddressGroupTracker tracker = eagTrackerMap.get(newAddress); + if (trackerMap.containsKey(newAddress)) { + AddressTracker tracker = trackerMap.get(newAddress); tracker.addSubchannel(this); // Make sure that the subchannel is in the same ejection state as the new tracker it is // associated with. - if (tracker.isEjected() && !ejected) { + if (tracker.subchannelsEjected() && !ejected) { eject(); - } else if (!tracker.isEjected() && ejected) { + } else if (!tracker.subchannelsEjected() && ejected) { uneject(); } } @@ -271,21 +271,21 @@ public void updateAddresses(List addresses) { // outlier detection. Remove it from all trackers and reset the call counters of all the // associated trackers. // Remove the current subchannel from the old address it is associated with in the map. - if (eagTrackerMap.containsKey(getAddresses())) { - EquivalentAddressGroupTracker tracker = eagTrackerMap.get(getAddresses()); + if (trackerMap.containsKey(getAddresses())) { + AddressTracker tracker = trackerMap.get(getAddresses()); tracker.removeSubchannel(this); tracker.clearCallCounters(); } } else if (getAllAddresses().size() > 1 && addresses.size() == 1) { // If the map has an entry for the new address, we associate this subchannel with it. EquivalentAddressGroup eag = Iterables.getOnlyElement(addresses); - if (eagTrackerMap.containsKey(eag)) { - EquivalentAddressGroupTracker tracker = eagTrackerMap.get(eag); + if (trackerMap.containsKey(eag)) { + AddressTracker tracker = trackerMap.get(eag); tracker.addSubchannel(this); // If the new address is already in the ejected state, we should also eject this // subchannel. - if (tracker.isEjected()) { + if (tracker.subchannelsEjected()) { eject(); } } @@ -299,9 +299,9 @@ public void updateAddresses(List addresses) { /** * If the {@link Subchannel} is considered for outlier detection the associated {@link - * EquivalentAddressGroupTracker} should be set. + * AddressTracker} should be set. */ - void setEquivalentAddressGroupInfo(EquivalentAddressGroupTracker eagInfo) { + void setEquivalentAddressGroupInfo(AddressTracker eagInfo) { this.eagInfo = eagInfo; } @@ -361,15 +361,15 @@ class OutlierDetectionPicker extends SubchannelPicker { public PickResult pickSubchannel(PickSubchannelArgs args) { PickResult pickResult = delegate.pickSubchannel(args); - // Because we wrap the helper used by the delegate LB we are assured that the subchannel - // picked here will be an instance of our OutlierDetectionSubchannel. - OutlierDetectionSubchannel subchannel - = (OutlierDetectionSubchannel) pickResult.getSubchannel(); + Subchannel subchannel = pickResult.getSubchannel(); + if (subchannel != null) { + return PickResult.withSubchannel(subchannel, + new ResultCountingClientStreamTracerFactory( + (AddressTracker) subchannel.getAttributes() + .get(EAG_INFO_ATTR_KEY))); + } - // The subchannel wrapper has served its purpose, we can pass on the wrapped delegate on - // in case another layer of wrapping assumes a particular subchannel sub-type. - return PickResult.withSubchannel(subchannel.delegate(), - new ResultCountingClientStreamTracerFactory(subchannel)); + return pickResult; } /** @@ -377,34 +377,34 @@ public PickResult pickSubchannel(PickSubchannelArgs args) { */ class ResultCountingClientStreamTracerFactory extends ClientStreamTracer.Factory { - private final OutlierDetectionSubchannel subchannel; + private final AddressTracker tracker; - ResultCountingClientStreamTracerFactory(OutlierDetectionSubchannel subchannel) { - this.subchannel = subchannel; + ResultCountingClientStreamTracerFactory(AddressTracker tracker) { + this.tracker = tracker; } @Override public ClientStreamTracer newClientStreamTracer(StreamInfo info, Metadata headers) { - return new ResultCountingClientStreamTracer(subchannel); + return new ResultCountingClientStreamTracer(tracker); } } /** * Counts the results (successful/unsuccessful) of a particular {@link * OutlierDetectionSubchannel}s streams and increments the counter in the associated {@link - * EquivalentAddressGroupTracker}. + * AddressTracker}. */ class ResultCountingClientStreamTracer extends ClientStreamTracer { - private final OutlierDetectionSubchannel subchannel; + AddressTracker tracker; - public ResultCountingClientStreamTracer(OutlierDetectionSubchannel subchannel) { - this.subchannel = subchannel; + public ResultCountingClientStreamTracer(AddressTracker tracker) { + this.tracker = tracker; } @Override public void streamClosed(Status status) { - subchannel.eagInfo.incrementCallCount(status.isOk()); + tracker.incrementCallCount(status.isOk()); } } } @@ -413,7 +413,7 @@ public void streamClosed(Status status) { * Tracks additional information about a set of equivalent addresses needed for outlier * detection. */ - static class EquivalentAddressGroupTracker { + static class AddressTracker { private final OutlierDetectionLoadBalancerConfig config; private CallCounter activeCallCounter = new CallCounter(); @@ -422,7 +422,7 @@ static class EquivalentAddressGroupTracker { private int ejectionTimeMultiplier; private final Set subchannels = new HashSet<>(); - EquivalentAddressGroupTracker(OutlierDetectionLoadBalancerConfig config) { + AddressTracker(OutlierDetectionLoadBalancerConfig config) { this.config = config; } @@ -452,10 +452,14 @@ void incrementCallCount(boolean success) { } /** - * The total number of calls in the active call counter. + * The total number of calls in the inactive call counter. */ long volume() { - return (long) activeCallCounter.successCount.get() + activeCallCounter.failureCount.get(); + return inactiveCallCounter.successCount.get() + inactiveCallCounter.failureCount.get(); + } + + double successRate() { + return ((double) inactiveCallCounter.successCount.get()) / volume(); } void clearCallCounters() { @@ -483,7 +487,7 @@ void swapCounters() { /** * Ejects the address from use. */ - void eject(long ejectionTimeNanos) { + void ejectSubchannels(long ejectionTimeNanos) { this.ejectionTimeNanos = ejectionTimeNanos; ejectionTimeMultiplier++; for (OutlierDetectionSubchannel subchannel : subchannels) { @@ -494,7 +498,7 @@ void eject(long ejectionTimeNanos) { /** * Uneject a currently ejected address. */ - void uneject() { + void unejectSubchannels() { checkState(ejectionTimeNanos == null, "not currently ejected"); ejectionTimeNanos = null; for (OutlierDetectionSubchannel subchannel : subchannels) { @@ -502,7 +506,7 @@ void uneject() { } } - boolean isEjected() { + boolean subchannelsEjected() { return ejectionTimeNanos != null; } @@ -512,17 +516,117 @@ public boolean maxEjectionTimeElapsed(long currentTimeNanos) { long maxEjectionTimeNanos = ejectionTimeNanos + Math.min( SECONDS.toNanos(config.baseEjectionTimeSecs) * ejectionTimeMultiplier, Math.max(SECONDS.toNanos(config.baseEjectionTimeSecs), - SECONDS.toNanos(config.maxEjectionTimeSecs))); + SECONDS.toNanos(config.maxEjectionTimeSecs))); return currentTimeNanos > maxEjectionTimeNanos; } private static class CallCounter { - AtomicInteger successCount = new AtomicInteger(); - AtomicInteger failureCount = new AtomicInteger(); + + AtomicLong successCount = new AtomicLong(); + AtomicLong failureCount = new AtomicLong(); + } + } + + /** + * Maintains a mapping from addresses to their trackers. + */ + static class AddressTrackerMap extends ForwardingMap { + private final Map trackerMap; + + AddressTrackerMap() { + trackerMap = new HashMap<>(); + } + + @Override + protected Map delegate() { + return trackerMap; + } + + /** Adds a new tracker for the addresses that don't already have one. */ + void putNewTrackers(OutlierDetectionLoadBalancerConfig config, + Collection addresses) { + for (EquivalentAddressGroup address : addresses) { + if (!trackerMap.containsKey(address)) { + trackerMap.put(address, new AddressTracker(config)); + } + } + } + + /** Resets the call counters for all the trackers in the map. */ + void resetCallCounters() { + for (AddressTracker tracker : trackerMap.values()) { + tracker.clearCallCounters(); + } + } + + /** + * When OD gets disabled we need to uneject any subchannels that may have been ejected and + * to reset the ejection time multiplier. + */ + void cancelTracking() { + for (AddressTracker tracker : trackerMap.values()) { + if (tracker.subchannelsEjected()) { + tracker.unejectSubchannels(); + } + tracker.resetEjectionTimeMultiplier(); + } + } + + /** Swaps the active and inactive counters for each tracker. */ + void swapCounters() { + for (AddressTracker tracker : trackerMap.values()) { + tracker.swapCounters(); + } + } + + /** + * At the end of a timer run we need to decrement the ejection time multiplier for trackers + * that don't have ejected subchannels and uneject ones that have spent the maximum ejection + * time allowed. + */ + void maybeUnejectOutliers(Long detectionTimerStartNanos) { + for (AddressTracker tracker : trackerMap.values()) { + if (!tracker.subchannelsEjected()) { + tracker.decrementEjectionTimeMultiplier(); + } + + if (tracker.subchannelsEjected() && tracker.maxEjectionTimeElapsed( + detectionTimerStartNanos)) { + tracker.unejectSubchannels(); + } + } + } + + /** Returns only the trackers that have the minimum configured volume to be considered. */ + List trackersWithVolume(OutlierDetectionLoadBalancerConfig config) { + List trackersWithVolume = new ArrayList<>(); + for (AddressTracker tracker : trackerMap.values()) { + if (tracker.volume() >= config.successRateEjection.requestVolume) { + trackersWithVolume.add(tracker); + } + } + return trackersWithVolume; + } + + /** + * How many percent of the addresses would have their subchannels ejected if we proceeded + * with the next ejection. + */ + double nextEjectionPercentage() { + int totalAddresses = 0; + int ejectedAddresses = 0; + for (AddressTracker tracker : trackerMap.values()) { + totalAddresses++; + if (tracker.subchannelsEjected()) { + ejectedAddresses++; + } + } + return ((double)ejectedAddresses + 1 / totalAddresses) * 100; } } + /** * Implementations provide different ways of ejecting outlier addresses.. */ @@ -530,11 +634,9 @@ interface OutlierEjectionAlgorithm { /** * Is the given {@link EquivalentAddressGroup} an outlier based on the past call results stored - * in {@link EquivalentAddressGroupTracker}. + * in {@link AddressTracker}. */ - void ejectOutliers( - Map eagTrackerMap, - long ejectionTimeMillis); + void ejectOutliers(AddressTrackerMap trackerMap, long ejectionTimeMillis); @Nullable static OutlierEjectionAlgorithm forConfig(OutlierDetectionLoadBalancerConfig config) { @@ -558,69 +660,63 @@ static class SuccessRateOutlierEjectionAlgorithm implements OutlierEjectionAlgor } @Override - public void ejectOutliers( - Map eagTrackerMap, - long ejectionTimeNanos) { + public void ejectOutliers(AddressTrackerMap trackerMap, long ejectionTimeNanos) { // Only consider addresses that have the minimum request volume specified in the config. - List trackersWithVolume = new ArrayList<>(); - for (EquivalentAddressGroupTracker tracker : eagTrackerMap.values()) { - if (tracker.volume() >= config.successRateEjection.requestVolume) { - trackersWithVolume.add(tracker); - } - } + List trackersWithVolume = trackerMap.trackersWithVolume(config); // If we don't have enough addresses with significant volume then there's nothing to do. if (trackersWithVolume.size() < config.successRateEjection.minimumHosts || trackersWithVolume.size() == 0) { return; } - // Calculate mean and standard deviation of the successful calls. - int totalSuccessCount = 0; - for (EquivalentAddressGroupTracker tracker : trackersWithVolume) { - totalSuccessCount += tracker.activeCallCounter.successCount.get(); - } - double mean = totalSuccessCount / trackersWithVolume.size(); - - double squaredDifferenceSum = 0; - for (EquivalentAddressGroupTracker tracker : trackersWithVolume) { - double difference = tracker.activeCallCounter.successCount.get() - mean; - squaredDifferenceSum += difference * difference; + // Calculate mean and standard deviation of the fractions of successful calls. + List successRates = new ArrayList<>(); + for (AddressTracker tracker : trackersWithVolume) { + successRates.add(tracker.successRate()); } - double variance = squaredDifferenceSum / trackersWithVolume.size(); + double mean = mean(successRates); + double stdev = standardDeviation(successRates, mean); - double stdev = Math.sqrt(variance); - - for (EquivalentAddressGroupTracker tracker : eagTrackerMap.values()) { - // If we have already ejected addresses past the max percentage, stop here - int totalAddresses = 0; - int ejectedAddresses = 0; - for (EquivalentAddressGroupTracker t : eagTrackerMap.values()) { - totalAddresses++; - if (t.isEjected()) { - ejectedAddresses++; - } - } - double ejectedPercentage = (ejectedAddresses / totalAddresses) * 100; - if (ejectedPercentage > config.maxEjectionPercent) { + for (AddressTracker tracker : trackerMap.values()) { + // If an ejection now would take us past the max configured ejection percentagem stop here. + if (trackerMap.nextEjectionPercentage() > config.maxEjectionPercent) { return; } - // If this address does not have enough volume to be considered, skip to the next one. - if (tracker.volume() < config.successRateEjection.requestVolume) { - continue; - } - // If success rate is below the threshold, eject the address. - double successRate = tracker.activeCallCounter.successCount.get() / tracker.volume(); - if (successRate < mean - stdev * (config.successRateEjection.stdevFactor / 1000)) { + double requiredSuccessRate = + mean - stdev * (config.successRateEjection.stdevFactor / 1000f); + if (tracker.successRate() < requiredSuccessRate) { // Only eject some addresses based on the enforcement percentage. if (new Random().nextInt(100) < config.successRateEjection.enforcementPercentage) { - tracker.eject(ejectionTimeNanos); + tracker.ejectSubchannels(ejectionTimeNanos); } } } } + + /** Calculates the mean of the given values. */ + static double mean(Collection values) { + double totalValue = 0; + for (double value : values) { + totalValue += value; + } + + return totalValue / values.size(); + } + + /** Calculates the standard deviation for the given values and their mean. */ + static double standardDeviation(Collection values, double mean) { + double squaredDifferenceSum = 0; + for (double value : values) { + double difference = value - mean; + squaredDifferenceSum += difference * difference; + } + double variance = squaredDifferenceSum / values.size(); + + return Math.sqrt(variance); + } } static class FailurePercentageOutlierEjectionAlgorithm implements OutlierEjectionAlgorithm { @@ -632,23 +728,21 @@ static class FailurePercentageOutlierEjectionAlgorithm implements OutlierEjectio } @Override - public void ejectOutliers( - Map eagTrackerMap, - long ejectionTimeMillis) { + public void ejectOutliers(AddressTrackerMap trackerMap, long ejectionTimeMillis) { // If we don't have the minimum amount of addresses the config calls for, then return. - if (eagTrackerMap.size() < config.failurePercentageEjection.minimumHosts) { + if (trackerMap.size() < config.failurePercentageEjection.minimumHosts) { return; } // If this address does not have enough volume to be considered, skip to the next one. - for (EquivalentAddressGroupTracker tracker : eagTrackerMap.values()) { + for (AddressTracker tracker : trackerMap.values()) { // If we have already ejected addresses past the max percentage, stop here. int totalAddresses = 0; int ejectedAddresses = 0; - for (EquivalentAddressGroupTracker t : eagTrackerMap.values()) { + for (AddressTracker t : trackerMap.values()) { totalAddresses++; - if (t.isEjected()) { + if (t.subchannelsEjected()) { ejectedAddresses++; } } @@ -668,7 +762,7 @@ public void ejectOutliers( if (failurePercentage > config.failurePercentageEjection.threshold) { // Only eject some addresses based on the enforcement percentage. if (new Random().nextInt(100) < config.failurePercentageEjection.enforcementPercentage) { - tracker.eject(ejectionTimeMillis); + tracker.ejectSubchannels(ejectionTimeMillis); } } } @@ -679,31 +773,91 @@ public void ejectOutliers( /** * The configuration for {@link OutlierDetectionLoadBalancer}. */ - static final class OutlierDetectionLoadBalancerConfig { + public static final class OutlierDetectionLoadBalancerConfig { - final long intervalSecs; - final long baseEjectionTimeSecs; - final long maxEjectionTimeSecs; + final Long intervalSecs; + final Long baseEjectionTimeSecs; + final Long maxEjectionTimeSecs; final Integer maxEjectionPercent; final SuccessRateEjection successRateEjection; final FailurePercentageEjection failurePercentageEjection; final PolicySelection childPolicy; - OutlierDetectionLoadBalancerConfig(PolicySelection childPolicy, Long intervalSecs, + private OutlierDetectionLoadBalancerConfig(Long intervalSecs, Long baseEjectionTimeSecs, Long maxEjectionTimeSecs, Integer maxEjectionPercent, SuccessRateEjection successRateEjection, - FailurePercentageEjection failurePercentageEjection) { - this.intervalSecs = intervalSecs != null ? intervalSecs : 10; - this.baseEjectionTimeSecs = baseEjectionTimeSecs != null ? baseEjectionTimeSecs : 30; - this.maxEjectionTimeSecs = maxEjectionTimeSecs != null ? maxEjectionTimeSecs : 30; - this.maxEjectionPercent = maxEjectionPercent != null ? maxEjectionPercent : 10; + FailurePercentageEjection failurePercentageEjection, + PolicySelection childPolicy) { + this.intervalSecs = intervalSecs; + this.baseEjectionTimeSecs = baseEjectionTimeSecs; + this.maxEjectionTimeSecs = maxEjectionTimeSecs; + this.maxEjectionPercent = maxEjectionPercent; this.successRateEjection = successRateEjection; this.failurePercentageEjection = failurePercentageEjection; this.childPolicy = childPolicy; } - static class SuccessRateEjection { + public static class Builder { + Long intervalSecs = 10L; + Long baseEjectionTimeSecs = 30L; + Long maxEjectionTimeSecs = 30L; + Integer maxEjectionPercent = 10; + SuccessRateEjection successRateEjection; + FailurePercentageEjection failurePercentageEjection; + PolicySelection childPolicy; + + public Builder setIntervalSecs(Long intervalSecs) { + checkArgument(intervalSecs != null); + this.intervalSecs = intervalSecs; + return this; + } + + public Builder setBaseEjectionTimeSecs(Long baseEjectionTimeSecs) { + checkArgument(baseEjectionTimeSecs != null); + this.baseEjectionTimeSecs = baseEjectionTimeSecs; + return this; + } + + public Builder setMaxEjectionTimeSecs(Long maxEjectionTimeSecs) { + checkArgument(maxEjectionTimeSecs != null); + this.maxEjectionTimeSecs = maxEjectionTimeSecs; + return this; + } + + public Builder setMaxEjectionPercent(Integer maxEjectionPercent) { + checkArgument(maxEjectionPercent != null); + this.maxEjectionPercent = maxEjectionPercent; + return this; + } + + public Builder setSuccessRateEjection( + SuccessRateEjection successRateEjection) { + this.successRateEjection = successRateEjection; + return this; + } + + public Builder setFailurePercentageEjection( + FailurePercentageEjection failurePercentageEjection) { + this.failurePercentageEjection = failurePercentageEjection; + return this; + } + + public Builder setChildPolicy(PolicySelection childPolicy) { + checkState(childPolicy != null); + this.childPolicy = childPolicy; + return this; + } + + public OutlierDetectionLoadBalancerConfig build() { + checkState(childPolicy != null); + return new OutlierDetectionLoadBalancerConfig(intervalSecs, baseEjectionTimeSecs, + maxEjectionTimeSecs, maxEjectionPercent, successRateEjection, failurePercentageEjection, + childPolicy); + } + } + + public static class SuccessRateEjection { final Integer stdevFactor; final Integer enforcementPercentage; @@ -712,15 +866,51 @@ static class SuccessRateEjection { SuccessRateEjection(Integer stdevFactor, Integer enforcementPercentage, Integer minimumHosts, Integer requestVolume) { - this.stdevFactor = stdevFactor != null ? stdevFactor : 1900; - this.enforcementPercentage = enforcementPercentage != null ? enforcementPercentage : 100; - this.minimumHosts = minimumHosts != null ? minimumHosts : 5; - this.requestVolume = requestVolume != null ? requestVolume : 100; + this.stdevFactor = stdevFactor; + this.enforcementPercentage = enforcementPercentage; + this.minimumHosts = minimumHosts; + this.requestVolume = requestVolume; } - } - static class FailurePercentageEjection { + public static final class Builder { + Integer stdevFactor = 1900; + Integer enforcementPercentage = 100; + Integer minimumHosts = 5; + Integer requestVolume = 100; + + public Builder setStdevFactor(Integer stdevFactor) { + checkArgument(stdevFactor != null); + this.stdevFactor = stdevFactor; + return this; + } + + public Builder setEnforcementPercentage(Integer enforcementPercentage) { + checkArgument(enforcementPercentage != null); + this.enforcementPercentage = enforcementPercentage; + return this; + } + + public Builder setMinimumHosts(Integer minimumHosts) { + checkArgument(minimumHosts != null); + this.minimumHosts = minimumHosts; + return this; + } + + public Builder setRequestVolume(Integer requestVolume) { + checkArgument(requestVolume != null); + this.requestVolume = requestVolume; + return this; + } + + public SuccessRateEjection build() { + return new SuccessRateEjection(stdevFactor, enforcementPercentage, minimumHosts, + requestVolume); + } + } + } + + public static class FailurePercentageEjection { final Integer threshold; final Integer enforcementPercentage; final Integer minimumHosts; @@ -728,10 +918,46 @@ static class FailurePercentageEjection { FailurePercentageEjection(Integer threshold, Integer enforcementPercentage, Integer minimumHosts, Integer requestVolume) { - this.threshold = threshold != null ? threshold : 85; - this.enforcementPercentage = enforcementPercentage != null ? enforcementPercentage : 100; - this.minimumHosts = minimumHosts != null ? minimumHosts : 5; - this.requestVolume = requestVolume != null ? requestVolume : 50; + this.threshold = threshold; + this.enforcementPercentage = enforcementPercentage; + this.minimumHosts = minimumHosts; + this.requestVolume = requestVolume; + } + + public static class Builder { + Integer threshold = 85; + Integer enforcementPercentage = 100; + Integer minimumHosts = 5; + Integer requestVolume = 50; + + public Builder setThreshold(Integer threshold) { + checkArgument(threshold != null); + this.threshold = threshold; + return this; + } + + public Builder setEnforcementPercentage(Integer enforcementPercentage) { + checkArgument(enforcementPercentage != null); + this.enforcementPercentage = enforcementPercentage; + return this; + } + + public Builder setMinimumHosts(Integer minimumHosts) { + checkArgument(minimumHosts != null); + this.minimumHosts = minimumHosts; + return this; + } + + public Builder setRequestVolume(Integer requestVolume) { + checkArgument(requestVolume != null); + this.requestVolume = requestVolume; + return this; + } + + public FailurePercentageEjection build() { + return new FailurePercentageEjection(threshold, enforcementPercentage, minimumHosts, + requestVolume); + } } } @@ -743,4 +969,10 @@ boolean outlierDetectionEnabled() { return successRateEjection != null || failurePercentageEjection != null; } } + + /** Math needed in outlier detection. */ + static class OutlierDetectionMath { + + + } } diff --git a/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java b/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java index c26dd0ce9d1..2b82c99597f 100644 --- a/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java +++ b/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java @@ -17,27 +17,27 @@ package io.grpc.util; import static com.google.common.truth.Truth.assertThat; -import static io.grpc.ConnectivityState.CONNECTING; import static io.grpc.ConnectivityState.READY; -import static org.junit.Assert.assertEquals; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.verifyNoMoreInteractions; import static org.mockito.Mockito.when; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import io.grpc.ClientStreamTracer; import io.grpc.ConnectivityState; import io.grpc.ConnectivityStateInfo; import io.grpc.EquivalentAddressGroup; import io.grpc.LoadBalancer; import io.grpc.LoadBalancer.CreateSubchannelArgs; import io.grpc.LoadBalancer.Helper; +import io.grpc.LoadBalancer.PickResult; +import io.grpc.LoadBalancer.PickSubchannelArgs; import io.grpc.LoadBalancer.ResolvedAddresses; import io.grpc.LoadBalancer.Subchannel; import io.grpc.LoadBalancer.SubchannelPicker; @@ -52,11 +52,14 @@ import io.grpc.internal.TestUtils.StandardLoadBalancerProvider; import io.grpc.util.OutlierDetectionLoadBalancer.OutlierDetectionLoadBalancerConfig; import io.grpc.util.OutlierDetectionLoadBalancer.OutlierDetectionLoadBalancerConfig.SuccessRateEjection; +import io.grpc.util.OutlierDetectionLoadBalancer.OutlierDetectionMath; +import io.grpc.util.OutlierDetectionLoadBalancer.OutlierDetectionSubchannel; +import io.grpc.util.OutlierDetectionLoadBalancer.SuccessRateOutlierEjectionAlgorithm; import io.grpc.util.RoundRobinLoadBalancer.ReadyPicker; import java.net.SocketAddress; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; @@ -113,10 +116,9 @@ public LoadBalancer newLoadBalancer(Helper helper) { "round_robin") { @Override public LoadBalancer newLoadBalancer(Helper helper) { - return roundRobinLoadBalancer; + return new RoundRobinLoadBalancer(helper); } }; - private RoundRobinLoadBalancer roundRobinLoadBalancer; private final FakeClock fakeClock = new FakeClock(); private final SynchronizationContext syncContext = new SynchronizationContext( @@ -131,12 +133,17 @@ public void uncaughtException(Thread t, Throwable e) { private final List servers = Lists.newArrayList(); private final Map, Subchannel> subchannels = Maps.newLinkedHashMap(); - private final Map subchannelStateListeners - = Maps.newLinkedHashMap(); + private final Map subchannelStateListeners = Maps.newLinkedHashMap(); + + private Subchannel subchannel1; + private Subchannel subchannel2; + private Subchannel subchannel3; + private Subchannel subchannel4; + private Subchannel subchannel5; @Before public void setUp() { - for (int i = 0; i < 3; i++) { + for (int i = 0; i < 5; i++) { SocketAddress addr = new FakeSocketAddress("server" + i); EquivalentAddressGroup eag = new EquivalentAddressGroup(addr); servers.add(eag); @@ -144,32 +151,36 @@ public void setUp() { subchannels.put(Arrays.asList(eag), sc); } + Iterator subchannelIterator = subchannels.values().iterator(); + subchannel1 = subchannelIterator.next(); + subchannel2 = subchannelIterator.next(); + subchannel3 = subchannelIterator.next(); + subchannel4 = subchannelIterator.next(); + subchannel5 = subchannelIterator.next(); + when(mockHelper.getSynchronizationContext()).thenReturn(syncContext); when(mockHelper.getScheduledExecutorService()).thenReturn( fakeClock.getScheduledExecutorService()); - when(mockHelper.createSubchannel(any(CreateSubchannelArgs.class))) - .then(new Answer() { + when(mockHelper.createSubchannel(any(CreateSubchannelArgs.class))).then( + new Answer() { @Override public Subchannel answer(InvocationOnMock invocation) throws Throwable { CreateSubchannelArgs args = (CreateSubchannelArgs) invocation.getArguments()[0]; final Subchannel subchannel = subchannels.get(args.getAddresses()); when(subchannel.getAllAddresses()).thenReturn(args.getAddresses()); when(subchannel.getAttributes()).thenReturn(args.getAttributes()); - doAnswer( - new Answer() { - @Override - public Void answer(InvocationOnMock invocation) throws Throwable { - subchannelStateListeners.put( - subchannel, (SubchannelStateListener) invocation.getArguments()[0]); - return null; - } - }).when(subchannel).start(any(SubchannelStateListener.class)); + doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocation) throws Throwable { + subchannelStateListeners.put(subchannel, + (SubchannelStateListener) invocation.getArguments()[0]); + return null; + } + }).when(subchannel).start(any(SubchannelStateListener.class)); return subchannel; } }); - roundRobinLoadBalancer = new RoundRobinLoadBalancer(mockHelper); - //lbRegistry.register(mockChildLbProvider); lbRegistry.register(roundRobinLbProvider); loadBalancer = new OutlierDetectionLoadBalancer(mockHelper, fakeClock.getTimeProvider()); @@ -186,26 +197,38 @@ public void handleNameResolutionError_noChildLb() { @Test public void handleNameResolutionError_withChildLb() { - loadBalancer.handleResolvedAddresses(buildResolvedAddress(defaultSuccessRateConfig(mockChildLbProvider), + loadBalancer.handleResolvedAddresses(buildResolvedAddress( + new OutlierDetectionLoadBalancerConfig.Builder() + .setSuccessRateEjection(new SuccessRateEjection.Builder().build()) + .setChildPolicy(new PolicySelection(mockChildLbProvider, null)).build(), new EquivalentAddressGroup(mockSocketAddress))); loadBalancer.handleNameResolutionError(Status.DEADLINE_EXCEEDED); verify(mockChildLb).handleNameResolutionError(Status.DEADLINE_EXCEEDED); } - /** {@code shutdown()} is simply delegated. */ + /** + * {@code shutdown()} is simply delegated. + */ @Test public void shutdown() { - loadBalancer.handleResolvedAddresses(buildResolvedAddress(defaultSuccessRateConfig(mockChildLbProvider), + loadBalancer.handleResolvedAddresses(buildResolvedAddress( + new OutlierDetectionLoadBalancerConfig.Builder() + .setSuccessRateEjection(new SuccessRateEjection.Builder().build()) + .setChildPolicy(new PolicySelection(mockChildLbProvider, null)).build(), new EquivalentAddressGroup(mockSocketAddress))); loadBalancer.shutdown(); verify(mockChildLb).shutdown(); } - /** Base case for accepting new resolved addresses. */ + /** + * Base case for accepting new resolved addresses. + */ @Test public void handleResolvedAddresses() { - OutlierDetectionLoadBalancerConfig config = defaultSuccessRateConfig(mockChildLbProvider); + OutlierDetectionLoadBalancerConfig config = new OutlierDetectionLoadBalancerConfig.Builder() + .setSuccessRateEjection(new SuccessRateEjection.Builder().build()) + .setChildPolicy(new PolicySelection(mockChildLbProvider, null)).build(); ResolvedAddresses resolvedAddresses = buildResolvedAddress(config, new EquivalentAddressGroup(mockSocketAddress)); @@ -222,10 +245,14 @@ public void handleResolvedAddresses() { assertThat(task.getDelay(TimeUnit.SECONDS)).isEqualTo(config.intervalSecs); } - /** Outlier detection first enabled, then removed. */ + /** + * Outlier detection first enabled, then removed. + */ @Test public void handleResolvedAddresses_outlierDetectionDisabled() { - OutlierDetectionLoadBalancerConfig config = defaultSuccessRateConfig(mockChildLbProvider); + OutlierDetectionLoadBalancerConfig config = new OutlierDetectionLoadBalancerConfig.Builder() + .setSuccessRateEjection(new SuccessRateEjection.Builder().build()) + .setChildPolicy(new PolicySelection(mockChildLbProvider, null)).build(); ResolvedAddresses resolvedAddresses = buildResolvedAddress(config, new EquivalentAddressGroup(mockSocketAddress)); @@ -236,29 +263,37 @@ public void handleResolvedAddresses_outlierDetectionDisabled() { // There is a single pending task to run the outlier detection algorithm assertThat(fakeClock.getPendingTasks()).hasSize(1); + config = new OutlierDetectionLoadBalancerConfig.Builder().setChildPolicy( + new PolicySelection(mockChildLbProvider, null)).build(); loadBalancer.handleResolvedAddresses( - buildResolvedAddress(disabledConfig(), new EquivalentAddressGroup(mockSocketAddress))); + buildResolvedAddress(config, new EquivalentAddressGroup(mockSocketAddress))); // Pending task should be gone since OD is disabled. assertThat(fakeClock.getPendingTasks()).isEmpty(); } - /** Tests different scenarios when the timer interval in the config changes. */ + /** + * Tests different scenarios when the timer interval in the config changes. + */ @Test public void handleResolvedAddresses_intervalUpdate() { - OutlierDetectionLoadBalancerConfig config = customIntervalConfig(null); + OutlierDetectionLoadBalancerConfig config = new OutlierDetectionLoadBalancerConfig.Builder() + .setSuccessRateEjection(new SuccessRateEjection.Builder().build()) + .setChildPolicy(new PolicySelection(mockChildLbProvider, null)).build(); ResolvedAddresses resolvedAddresses = buildResolvedAddress(config, new EquivalentAddressGroup(mockSocketAddress)); loadBalancer.handleResolvedAddresses(resolvedAddresses); // Config update has doubled the interval - config = customIntervalConfig(config.intervalSecs * 2); + config = new OutlierDetectionLoadBalancerConfig.Builder() + .setIntervalSecs(config.intervalSecs * 2) + .setSuccessRateEjection(new SuccessRateEjection.Builder().build()) + .setChildPolicy(new PolicySelection(mockChildLbProvider, null)).build(); loadBalancer.handleResolvedAddresses( - buildResolvedAddress(config, - new EquivalentAddressGroup(mockSocketAddress))); + buildResolvedAddress(config, new EquivalentAddressGroup(mockSocketAddress))); // If the timer has not run yet the task is just rescheduled to run after the new delay. assertThat(fakeClock.getPendingTasks()).hasSize(1); @@ -280,46 +315,219 @@ public void handleResolvedAddresses_intervalUpdate() { fakeClock.forwardTime(4, TimeUnit.SECONDS); task = fakeClock.getPendingTasks().iterator().next(); loadBalancer.handleResolvedAddresses( - buildResolvedAddress(config, - new EquivalentAddressGroup(mockSocketAddress))); + buildResolvedAddress(config, new EquivalentAddressGroup(mockSocketAddress))); assertThat(task.dueTimeNanos).isEqualTo( TimeUnit.SECONDS.toNanos(config.intervalSecs + config.intervalSecs + 1)); } - // @Test - // public void pickAfterResolved() throws Exception { - // final Subchannel readySubchannel = subchannels.values().iterator().next(); - // - // loadBalancer.handleResolvedAddresses( - // buildResolvedAddress(defaultSuccessRateConfig(roundRobinLbProvider), servers)); - // - // deliverSubchannelState(readySubchannel, ConnectivityStateInfo.forNonError(READY)); - // - // verify(mockHelper, times(3)).createSubchannel(createArgsCaptor.capture()); - // - // List> capturedAddrs = new ArrayList<>(); - // for (CreateSubchannelArgs arg : createArgsCaptor.getAllValues()) { - // capturedAddrs.add(arg.getAddresses()); - // } - // - // assertThat(capturedAddrs).containsAtLeastElementsIn(subchannels.keySet()); - // for (Subchannel subchannel : subchannels.values()) { - // verify(subchannel).requestConnection(); - // verify(subchannel, never()).shutdown(); - // } - // // - // // verify(mockHelper, times(2)) - // // .updateBalancingState(stateCaptor.capture(), pickerCaptor.capture()); - // // - // // assertEquals(CONNECTING, stateCaptor.getAllValues().get(0)); - // // assertEquals(READY, stateCaptor.getAllValues().get(1)); - // // assertThat(getList(pickerCaptor.getValue())).containsExactly(readySubchannel); - // // - // // verifyNoMoreInteractions(mockHelper); - // } + /** + * Confirm basic picking works by delegating to round_robin. + */ + @Test + public void delegatePick() throws Exception { + OutlierDetectionLoadBalancerConfig config = new OutlierDetectionLoadBalancerConfig.Builder() + .setSuccessRateEjection(new SuccessRateEjection.Builder().build()) + .setChildPolicy(new PolicySelection(roundRobinLbProvider, null)).build(); + + loadBalancer.handleResolvedAddresses(buildResolvedAddress(config, servers.get(0))); + + // Make one of the subchannels READY. + final Subchannel readySubchannel = subchannels.values().iterator().next(); + deliverSubchannelState(readySubchannel, ConnectivityStateInfo.forNonError(READY)); + + verify(mockHelper, times(3)).updateBalancingState(stateCaptor.capture(), + pickerCaptor.capture()); + + // Make sure that we can pick the single READY subchannel. + SubchannelPicker picker = pickerCaptor.getAllValues().get(2); + PickResult pickResult = picker.pickSubchannel(mock(PickSubchannelArgs.class)); + assertThat(((OutlierDetectionSubchannel) pickResult.getSubchannel()).delegate()).isEqualTo( + readySubchannel); + } + + /** + * The success rate algorithm leaves a healthy set of addresses alone. + */ + @Test + public void successRateNoOutliers() { + OutlierDetectionLoadBalancerConfig config = new OutlierDetectionLoadBalancerConfig.Builder().setMaxEjectionPercent( + 50).setSuccessRateEjection( + new SuccessRateEjection.Builder().setMinimumHosts(3).setRequestVolume(10).build()) + .setChildPolicy(new PolicySelection(roundRobinLbProvider, null)).build(); + + loadBalancer.handleResolvedAddresses(buildResolvedAddress(config, servers)); + + generateLoad(config, ImmutableMap.of()); + + // Move forward in time to a point where the detection timer has fired. + fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + + assertThat(loadBalancer.trackerMap.get(servers.get(0)).subchannelsEjected()).isFalse(); + assertThat(loadBalancer.trackerMap.get(servers.get(1)).subchannelsEjected()).isFalse(); + assertThat(loadBalancer.trackerMap.get(servers.get(2)).subchannelsEjected()).isFalse(); + assertThat(loadBalancer.trackerMap.get(servers.get(3)).subchannelsEjected()).isFalse(); + assertThat(loadBalancer.trackerMap.get(servers.get(4)).subchannelsEjected()).isFalse(); + } + + /** + * The success rate algorithm ejects the outlier. + */ + @Test + public void successRateOneOutlier() { + OutlierDetectionLoadBalancerConfig config = new OutlierDetectionLoadBalancerConfig.Builder() + .setMaxEjectionPercent(50) + .setSuccessRateEjection( + new SuccessRateEjection.Builder() + .setMinimumHosts(3) + .setRequestVolume(10).build()) + .setChildPolicy(new PolicySelection(roundRobinLbProvider, null)).build(); + + generateLoad(config, ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); + + // Move forward in time to a point where the detection timer has fired. + fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + + // The one subchannel that was returning errors should be ejected. + assertThat(loadBalancer.trackerMap.get(servers.get(0)).subchannelsEjected()).isTrue(); + assertThat(loadBalancer.trackerMap.get(servers.get(1)).subchannelsEjected()).isFalse(); + assertThat(loadBalancer.trackerMap.get(servers.get(2)).subchannelsEjected()).isFalse(); + assertThat(loadBalancer.trackerMap.get(servers.get(3)).subchannelsEjected()).isFalse(); + assertThat(loadBalancer.trackerMap.get(servers.get(4)).subchannelsEjected()).isFalse(); + } + + /** + * The success rate algorithm ignores addresses without enough volume.. + */ + @Test + public void successRateOneOutlier_notEnoughVolume() { + OutlierDetectionLoadBalancerConfig config = new OutlierDetectionLoadBalancerConfig.Builder() + .setMaxEjectionPercent(50) + .setSuccessRateEjection( + new SuccessRateEjection.Builder() + .setMinimumHosts(3) + .setRequestVolume(100).build()) // We won't produce this much volume... + .setChildPolicy(new PolicySelection(roundRobinLbProvider, null)).build(); + + generateLoad(config, ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); + + // Move forward in time to a point where the detection timer has fired. + fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + + // The one subchannel that was returning errors should be ejected. + assertThat(loadBalancer.trackerMap.get(servers.get(0)).subchannelsEjected()).isFalse(); + assertThat(loadBalancer.trackerMap.get(servers.get(1)).subchannelsEjected()).isFalse(); + assertThat(loadBalancer.trackerMap.get(servers.get(2)).subchannelsEjected()).isFalse(); + assertThat(loadBalancer.trackerMap.get(servers.get(3)).subchannelsEjected()).isFalse(); + assertThat(loadBalancer.trackerMap.get(servers.get(4)).subchannelsEjected()).isFalse(); + } + + /** + * The success rate algorithm does not apply if enough addresses have the required volume. + */ + @Test + public void successRateOneOutlier_notEnoughAddressesWithVolume() { + OutlierDetectionLoadBalancerConfig config = new OutlierDetectionLoadBalancerConfig.Builder() + .setMaxEjectionPercent(50) + .setSuccessRateEjection( + new SuccessRateEjection.Builder() + .setMinimumHosts(6) // We don't have this many hosts... + .setRequestVolume(10).build()) + .setChildPolicy(new PolicySelection(roundRobinLbProvider, null)).build(); + + generateLoad(config, ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); + + // Move forward in time to a point where the detection timer has fired. + fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + + // No subchannels should have been ejected. + assertThat(loadBalancer.trackerMap.get(servers.get(0)).subchannelsEjected()).isFalse(); + assertThat(loadBalancer.trackerMap.get(servers.get(1)).subchannelsEjected()).isFalse(); + assertThat(loadBalancer.trackerMap.get(servers.get(2)).subchannelsEjected()).isFalse(); + assertThat(loadBalancer.trackerMap.get(servers.get(3)).subchannelsEjected()).isFalse(); + assertThat(loadBalancer.trackerMap.get(servers.get(4)).subchannelsEjected()).isFalse(); + } + + /** + * Two outliers get ejected. + */ + @Test + public void successRateTwoOutliers() { + OutlierDetectionLoadBalancerConfig config = new OutlierDetectionLoadBalancerConfig.Builder() + .setMaxEjectionPercent(50) + .setSuccessRateEjection( + new SuccessRateEjection.Builder() + .setMinimumHosts(3) + .setRequestVolume(10) + .setStdevFactor(1).build()) + .setChildPolicy(new PolicySelection(roundRobinLbProvider, null)).build(); + + generateLoad(config, ImmutableMap.of( + subchannel1, Status.DEADLINE_EXCEEDED, + subchannel2, Status.DEADLINE_EXCEEDED)); + + // Move forward in time to a point where the detection timer has fired. + fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + + // The one subchannel that was returning errors should be ejected. + assertThat(loadBalancer.trackerMap.get(servers.get(0)).subchannelsEjected()).isTrue(); + assertThat(loadBalancer.trackerMap.get(servers.get(1)).subchannelsEjected()).isTrue(); + assertThat(loadBalancer.trackerMap.get(servers.get(2)).subchannelsEjected()).isFalse(); + assertThat(loadBalancer.trackerMap.get(servers.get(3)).subchannelsEjected()).isFalse(); + assertThat(loadBalancer.trackerMap.get(servers.get(4)).subchannelsEjected()).isFalse(); + } + + /** + * Two outliers. but only one gets ejected because we have reached the max ejection percentage. + */ + @Test + public void successRateTwoOutliers_maxEjectionPercentage() { + OutlierDetectionLoadBalancerConfig config = new OutlierDetectionLoadBalancerConfig.Builder() + .setMaxEjectionPercent(20) + .setSuccessRateEjection( + new SuccessRateEjection.Builder() + .setMinimumHosts(3) + .setRequestVolume(10) + .setStdevFactor(1).build()) + .setChildPolicy(new PolicySelection(roundRobinLbProvider, null)).build(); + + generateLoad(config, ImmutableMap.of( + subchannel1, Status.DEADLINE_EXCEEDED, + subchannel2, Status.DEADLINE_EXCEEDED)); + + // Move forward in time to a point where the detection timer has fired. + fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + + int totalEjected = 0; + for (EquivalentAddressGroup address: servers) { + totalEjected += loadBalancer.trackerMap.get(address).subchannelsEjected() ? 1 : 0; + } + + // Even if all subchannels were failing, we should have not ejected more than the configured + // maximum percentage. + assertThat((double) totalEjected / servers.size()).isAtMost( + (double) config.maxEjectionPercent / 100); + } + + /** + * The failure percentage algorithm leaves a healthy set of addresses alone. + */ + @Test + public void failurePercentageHealthy() { + } + + + @Test + public void mathChecksOut() { + ImmutableList values = ImmutableList.of(600d, 470d, 170d, 430d, 300d); + double mean = SuccessRateOutlierEjectionAlgorithm.mean(values); + double stdev = SuccessRateOutlierEjectionAlgorithm.standardDeviation(values, mean); + assertThat(mean).isEqualTo(394); + assertThat(stdev).isEqualTo(147.32277488562318); + } private static class FakeSocketAddress extends SocketAddress { + final String name; FakeSocketAddress(String name) { @@ -332,25 +540,6 @@ public String toString() { } } - private OutlierDetectionLoadBalancerConfig defaultSuccessRateConfig( - LoadBalancerProvider childLbProvider) { - return new OutlierDetectionLoadBalancerConfig(new PolicySelection(childLbProvider, null), - null, null, null, null, - new SuccessRateEjection(null, null, null, null), null); - } - - private OutlierDetectionLoadBalancerConfig customIntervalConfig(Long intervalSecs) { - return new OutlierDetectionLoadBalancerConfig(new PolicySelection(mockChildLbProvider, null), - intervalSecs != null ? intervalSecs : null, null, null, null, - new SuccessRateEjection(null, null, null, null), null); - } - - private OutlierDetectionLoadBalancerConfig disabledConfig() { - return new OutlierDetectionLoadBalancerConfig(new PolicySelection(mockChildLbProvider, null), - null, null, null, null, - null, null); - } - private ResolvedAddresses buildResolvedAddress(OutlierDetectionLoadBalancerConfig config, EquivalentAddressGroup... servers) { return ResolvedAddresses.newBuilder().setAddresses(ImmutableList.copyOf(servers)) @@ -368,7 +557,35 @@ private void deliverSubchannelState(Subchannel subchannel, ConnectivityStateInfo } private static List getList(SubchannelPicker picker) { - return picker instanceof ReadyPicker ? ((ReadyPicker) picker).getList() : - Collections.emptyList(); + return picker instanceof ReadyPicker ? ((ReadyPicker) picker).getList() + : Collections.emptyList(); + } + + // Generates 30 calls, 10 each across the subchannels. + private void generateLoad(OutlierDetectionLoadBalancerConfig config, + Map statusMap) { + loadBalancer.handleResolvedAddresses(buildResolvedAddress(config, servers)); + + deliverSubchannelState(subchannel1, ConnectivityStateInfo.forNonError(READY)); + deliverSubchannelState(subchannel2, ConnectivityStateInfo.forNonError(READY)); + deliverSubchannelState(subchannel3, ConnectivityStateInfo.forNonError(READY)); + deliverSubchannelState(subchannel4, ConnectivityStateInfo.forNonError(READY)); + deliverSubchannelState(subchannel5, ConnectivityStateInfo.forNonError(READY)); + + verify(mockHelper, times(7)).updateBalancingState(stateCaptor.capture(), + pickerCaptor.capture()); + SubchannelPicker picker = pickerCaptor.getAllValues().get(6); + + for (int i = 0; i < 100; i++) { + PickResult pickResult = picker + .pickSubchannel(mock(PickSubchannelArgs.class)); + ClientStreamTracer clientStreamTracer = pickResult.getStreamTracerFactory() + .newClientStreamTracer(null, null); + + // Fail all the calls for the first subchannel. + Subchannel subchannel = ((OutlierDetectionSubchannel) pickResult.getSubchannel()).delegate(); + clientStreamTracer.streamClosed( + statusMap.containsKey(subchannel) ? statusMap.get(subchannel) : Status.OK); + } } } From ee4e748d631df11b6239ac7bb3aa2446bbc3ecd3 Mon Sep 17 00:00:00 2001 From: Terry Wilson Date: Sat, 13 Aug 2022 15:04:48 -0700 Subject: [PATCH 04/18] failure percentage algo tested --- .../util/OutlierDetectionLoadBalancer.java | 44 ++--- .../OutlierDetectionLoadBalancerTest.java | 162 +++++++++++++++--- 2 files changed, 154 insertions(+), 52 deletions(-) diff --git a/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java b/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java index 721c64cb7e9..bc692ed6518 100644 --- a/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java +++ b/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java @@ -52,8 +52,8 @@ import javax.annotation.Nullable; /** - * Wraps a child {@code LoadBalancer} while monitoring for outliers backends and removing them from - * use by the child LB. + * Wraps a child {@code LoadBalancer} while monitoring for outlier backends and removing them from + * the use of the child LB. * *

This implements the outlier detection gRFC: * https://github.com/grpc/proposal/blob/master/A50-xds-outlier-detection.md @@ -462,6 +462,10 @@ long volume() { return ((double) inactiveCallCounter.successCount.get()) / volume(); } + double failureRate() { + return ((double)inactiveCallCounter.failureCount.get()) / volume(); + } + void clearCallCounters() { activeCallCounter.successCount.set(0); activeCallCounter.failureCount.set(0); @@ -622,7 +626,7 @@ List trackersWithVolume(OutlierDetectionLoadBalancerConfig confi ejectedAddresses++; } } - return ((double)ejectedAddresses + 1 / totalAddresses) * 100; + return ((double)(ejectedAddresses + 1) / totalAddresses) * 100; } } @@ -650,6 +654,11 @@ static OutlierEjectionAlgorithm forConfig(OutlierDetectionLoadBalancerConfig con } } + /** + * This algorithm ejects addresses that don't maintain a required rate of successful calls. The + * required rate is not fixed, but is based on the mean and standard deviation of the success + * rates of all of the addresses. + */ static class SuccessRateOutlierEjectionAlgorithm implements OutlierEjectionAlgorithm { private final OutlierDetectionLoadBalancerConfig config; @@ -737,18 +746,8 @@ public void ejectOutliers(AddressTrackerMap trackerMap, long ejectionTimeMillis) // If this address does not have enough volume to be considered, skip to the next one. for (AddressTracker tracker : trackerMap.values()) { - // If we have already ejected addresses past the max percentage, stop here. - int totalAddresses = 0; - int ejectedAddresses = 0; - for (AddressTracker t : trackerMap.values()) { - totalAddresses++; - if (t.subchannelsEjected()) { - ejectedAddresses++; - } - } - double ejectedPercentage = (ejectedAddresses / totalAddresses) * 100; - - if (ejectedPercentage > config.maxEjectionPercent) { + // If an ejection now would take us past the max configured ejection percentagem stop here. + if (trackerMap.nextEjectionPercentage() > config.maxEjectionPercent) { return; } @@ -756,11 +755,10 @@ public void ejectOutliers(AddressTrackerMap trackerMap, long ejectionTimeMillis) continue; } - // If the failure percentage is above the threshold. - long failurePercentage = - (tracker.activeCallCounter.failureCount.get() / tracker.volume()) * 100; - if (failurePercentage > config.failurePercentageEjection.threshold) { - // Only eject some addresses based on the enforcement percentage. + // If the failure rate is above the threshold, we should eject... + double maxFailureRate = ((double)config.failurePercentageEjection.threshold) / 100; + if (tracker.failureRate() > maxFailureRate) { + // ...but only enforce this based on the enforcement percentage. if (new Random().nextInt(100) < config.failurePercentageEjection.enforcementPercentage) { tracker.ejectSubchannels(ejectionTimeMillis); } @@ -969,10 +967,4 @@ boolean outlierDetectionEnabled() { return successRateEjection != null || failurePercentageEjection != null; } } - - /** Math needed in outlier detection. */ - static class OutlierDetectionMath { - - - } } diff --git a/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java b/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java index 2b82c99597f..add4ca27522 100644 --- a/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java +++ b/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java @@ -27,6 +27,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import io.grpc.ClientStreamTracer; @@ -50,9 +51,10 @@ import io.grpc.internal.FakeClock.ScheduledTask; import io.grpc.internal.ServiceConfigUtil.PolicySelection; import io.grpc.internal.TestUtils.StandardLoadBalancerProvider; +import io.grpc.util.OutlierDetectionLoadBalancer.AddressTracker; import io.grpc.util.OutlierDetectionLoadBalancer.OutlierDetectionLoadBalancerConfig; +import io.grpc.util.OutlierDetectionLoadBalancer.OutlierDetectionLoadBalancerConfig.FailurePercentageEjection; import io.grpc.util.OutlierDetectionLoadBalancer.OutlierDetectionLoadBalancerConfig.SuccessRateEjection; -import io.grpc.util.OutlierDetectionLoadBalancer.OutlierDetectionMath; import io.grpc.util.OutlierDetectionLoadBalancer.OutlierDetectionSubchannel; import io.grpc.util.OutlierDetectionLoadBalancer.SuccessRateOutlierEjectionAlgorithm; import io.grpc.util.RoundRobinLoadBalancer.ReadyPicker; @@ -62,6 +64,8 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; import java.util.concurrent.TimeUnit; import org.junit.Before; import org.junit.Rule; @@ -350,8 +354,9 @@ public void delegatePick() throws Exception { */ @Test public void successRateNoOutliers() { - OutlierDetectionLoadBalancerConfig config = new OutlierDetectionLoadBalancerConfig.Builder().setMaxEjectionPercent( - 50).setSuccessRateEjection( + OutlierDetectionLoadBalancerConfig config = new OutlierDetectionLoadBalancerConfig.Builder() + .setMaxEjectionPercent(50) + .setSuccessRateEjection( new SuccessRateEjection.Builder().setMinimumHosts(3).setRequestVolume(10).build()) .setChildPolicy(new PolicySelection(roundRobinLbProvider, null)).build(); @@ -362,11 +367,8 @@ public void successRateNoOutliers() { // Move forward in time to a point where the detection timer has fired. fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); - assertThat(loadBalancer.trackerMap.get(servers.get(0)).subchannelsEjected()).isFalse(); - assertThat(loadBalancer.trackerMap.get(servers.get(1)).subchannelsEjected()).isFalse(); - assertThat(loadBalancer.trackerMap.get(servers.get(2)).subchannelsEjected()).isFalse(); - assertThat(loadBalancer.trackerMap.get(servers.get(3)).subchannelsEjected()).isFalse(); - assertThat(loadBalancer.trackerMap.get(servers.get(4)).subchannelsEjected()).isFalse(); + // No outliers, no ejections. + assertEjectedChannels(ImmutableSet.of()); } /** @@ -388,11 +390,7 @@ public void successRateOneOutlier() { fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); // The one subchannel that was returning errors should be ejected. - assertThat(loadBalancer.trackerMap.get(servers.get(0)).subchannelsEjected()).isTrue(); - assertThat(loadBalancer.trackerMap.get(servers.get(1)).subchannelsEjected()).isFalse(); - assertThat(loadBalancer.trackerMap.get(servers.get(2)).subchannelsEjected()).isFalse(); - assertThat(loadBalancer.trackerMap.get(servers.get(3)).subchannelsEjected()).isFalse(); - assertThat(loadBalancer.trackerMap.get(servers.get(4)).subchannelsEjected()).isFalse(); + assertEjectedChannels(ImmutableSet.of(servers.get(0))); } /** @@ -413,12 +411,8 @@ public void successRateOneOutlier_notEnoughVolume() { // Move forward in time to a point where the detection timer has fired. fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); - // The one subchannel that was returning errors should be ejected. - assertThat(loadBalancer.trackerMap.get(servers.get(0)).subchannelsEjected()).isFalse(); - assertThat(loadBalancer.trackerMap.get(servers.get(1)).subchannelsEjected()).isFalse(); - assertThat(loadBalancer.trackerMap.get(servers.get(2)).subchannelsEjected()).isFalse(); - assertThat(loadBalancer.trackerMap.get(servers.get(3)).subchannelsEjected()).isFalse(); - assertThat(loadBalancer.trackerMap.get(servers.get(4)).subchannelsEjected()).isFalse(); + // The address should not have been ejected.. + assertEjectedChannels(ImmutableSet.of()); } /** @@ -440,11 +434,31 @@ public void successRateOneOutlier_notEnoughAddressesWithVolume() { fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); // No subchannels should have been ejected. - assertThat(loadBalancer.trackerMap.get(servers.get(0)).subchannelsEjected()).isFalse(); - assertThat(loadBalancer.trackerMap.get(servers.get(1)).subchannelsEjected()).isFalse(); - assertThat(loadBalancer.trackerMap.get(servers.get(2)).subchannelsEjected()).isFalse(); - assertThat(loadBalancer.trackerMap.get(servers.get(3)).subchannelsEjected()).isFalse(); - assertThat(loadBalancer.trackerMap.get(servers.get(4)).subchannelsEjected()).isFalse(); + assertEjectedChannels(ImmutableSet.of()); + } + + /** + * The enforcementPercentage configuration should be honored. + */ + @Test + public void successRateOneOutlier_enforcementPercentage() { + OutlierDetectionLoadBalancerConfig config = new OutlierDetectionLoadBalancerConfig.Builder() + .setMaxEjectionPercent(50) + .setSuccessRateEjection( + new SuccessRateEjection.Builder() + .setMinimumHosts(3) + .setRequestVolume(10) + .setEnforcementPercentage(0) + .build()) + .setChildPolicy(new PolicySelection(roundRobinLbProvider, null)).build(); + + generateLoad(config, ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); + + // Move forward in time to a point where the detection timer has fired. + fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + + // There is one outlier, but because enforcementPercentage is 0, nothing should be ejected. + assertEjectedChannels(ImmutableSet.of()); } /** @@ -474,6 +488,8 @@ public void successRateTwoOutliers() { assertThat(loadBalancer.trackerMap.get(servers.get(2)).subchannelsEjected()).isFalse(); assertThat(loadBalancer.trackerMap.get(servers.get(3)).subchannelsEjected()).isFalse(); assertThat(loadBalancer.trackerMap.get(servers.get(4)).subchannelsEjected()).isFalse(); + + //assertEjectedChannels(ImmutableSet.of(servers.get(0), servers.get(1))); } /** @@ -508,13 +524,99 @@ public void successRateTwoOutliers_maxEjectionPercentage() { (double) config.maxEjectionPercent / 100); } + /** - * The failure percentage algorithm leaves a healthy set of addresses alone. + * The success rate algorithm leaves a healthy set of addresses alone. */ @Test - public void failurePercentageHealthy() { + public void failurePercentageNoOutliers() { + OutlierDetectionLoadBalancerConfig config = new OutlierDetectionLoadBalancerConfig.Builder() + .setMaxEjectionPercent(50) + .setFailurePercentageEjection( + new FailurePercentageEjection.Builder() + .setMinimumHosts(3) + .setRequestVolume(10).build()) + .setChildPolicy(new PolicySelection(roundRobinLbProvider, null)).build(); + + loadBalancer.handleResolvedAddresses(buildResolvedAddress(config, servers)); + + // By default all calls will return OK. + generateLoad(config, ImmutableMap.of()); + + // Move forward in time to a point where the detection timer has fired. + fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + + // No outliers, no ejections. + assertEjectedChannels(ImmutableSet.of()); } + /** + * The success rate algorithm ejects the outlier. + */ + @Test + public void failurePercentageOneOutlier() { + OutlierDetectionLoadBalancerConfig config = new OutlierDetectionLoadBalancerConfig.Builder() + .setMaxEjectionPercent(50) + .setFailurePercentageEjection( + new FailurePercentageEjection.Builder() + .setMinimumHosts(3) + .setRequestVolume(10).build()) + .setChildPolicy(new PolicySelection(roundRobinLbProvider, null)).build(); + + generateLoad(config, ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); + + // Move forward in time to a point where the detection timer has fired. + fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + + // The one subchannel that was returning errors should be ejected. + assertEjectedChannels(ImmutableSet.of(servers.get(0))); + } + + /** + * The failure percentage algorithm ignores addresses without enough volume.. + */ + @Test + public void failurePercentageOneOutlier_notEnoughVolume() { + OutlierDetectionLoadBalancerConfig config = new OutlierDetectionLoadBalancerConfig.Builder() + .setMaxEjectionPercent(50) + .setFailurePercentageEjection( + new FailurePercentageEjection.Builder() + .setMinimumHosts(3) + .setRequestVolume(100).build()) // We won't produce this much volume... + .setChildPolicy(new PolicySelection(roundRobinLbProvider, null)).build(); + + generateLoad(config, ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); + + // Move forward in time to a point where the detection timer has fired. + fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + + // We should see no ejections. + assertEjectedChannels(ImmutableSet.of()); + } + + /** + * The enforcementPercentage configuration should be honored. + */ + @Test + public void failurePercentageOneOutlier_enforcementPercentage() { + OutlierDetectionLoadBalancerConfig config = new OutlierDetectionLoadBalancerConfig.Builder() + .setMaxEjectionPercent(50) + .setFailurePercentageEjection( + new FailurePercentageEjection.Builder() + .setMinimumHosts(3) + .setRequestVolume(10) + .setEnforcementPercentage(0) + .build()) + .setChildPolicy(new PolicySelection(roundRobinLbProvider, null)).build(); + + generateLoad(config, ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); + + // Move forward in time to a point where the detection timer has fired. + fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + + // There is one outlier, but because enforcementPercentage is 0, nothing should be ejected. + assertEjectedChannels(ImmutableSet.of()); + } @Test public void mathChecksOut() { @@ -588,4 +690,12 @@ private void generateLoad(OutlierDetectionLoadBalancerConfig config, statusMap.containsKey(subchannel) ? statusMap.get(subchannel) : Status.OK); } } + + // Asserts that the given addresses are ejected and the rest are not. + void assertEjectedChannels(Set addresses) { + for (Entry entry : loadBalancer.trackerMap.entrySet()) { + assertThat(entry.getValue().subchannelsEjected()).isEqualTo( + addresses.contains(entry.getKey())); + } + } } From ffe1dd48f93a901c275b1660e4d6cd0b534c1586 Mon Sep 17 00:00:00 2001 From: Terry Wilson Date: Sun, 14 Aug 2022 15:00:30 -0700 Subject: [PATCH 05/18] Added tests for subchannel address updates. --- .../util/OutlierDetectionLoadBalancer.java | 93 ++++-- .../test/java/io/grpc/internal/FakeClock.java | 1 - .../OutlierDetectionLoadBalancerTest.java | 279 ++++++++++++++---- 3 files changed, 291 insertions(+), 82 deletions(-) diff --git a/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java b/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java index bc692ed6518..a0676dea610 100644 --- a/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java +++ b/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java @@ -24,6 +24,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ForwardingMap; +import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import io.grpc.Attributes; import io.grpc.ClientStreamTracer; @@ -71,7 +72,8 @@ public class OutlierDetectionLoadBalancer extends LoadBalancer { private ScheduledHandle detectionTimerHandle; private Long detectionTimerStartNanos; - private static final Attributes.Key EAG_INFO_ATTR_KEY = Attributes.Key.create("eagInfoKey"); + private static final Attributes.Key EAG_INFO_ATTR_KEY = Attributes.Key.create( + "eagInfoKey"); /** * Creates a new instance of {@link OutlierDetectionLoadBalancer}. @@ -257,14 +259,6 @@ public void updateAddresses(List addresses) { if (trackerMap.containsKey(newAddress)) { AddressTracker tracker = trackerMap.get(newAddress); tracker.addSubchannel(this); - - // Make sure that the subchannel is in the same ejection state as the new tracker it is - // associated with. - if (tracker.subchannelsEjected() && !ejected) { - eject(); - } else if (!tracker.subchannelsEjected() && ejected) { - uneject(); - } } } else if (getAllAddresses().size() == 1 && addresses.size() > 1) { // We go from a single address to having multiple, making this subchannel uneligible for @@ -277,24 +271,19 @@ public void updateAddresses(List addresses) { tracker.clearCallCounters(); } } else if (getAllAddresses().size() > 1 && addresses.size() == 1) { - // If the map has an entry for the new address, we associate this subchannel with it. + // We go from, previously uneligble, multiple address mode to a single address. If the map + // has an entry for the new address, we associate this subchannel with it. EquivalentAddressGroup eag = Iterables.getOnlyElement(addresses); if (trackerMap.containsKey(eag)) { AddressTracker tracker = trackerMap.get(eag); tracker.addSubchannel(this); - - // If the new address is already in the ejected state, we should also eject this - // subchannel. - if (tracker.subchannelsEjected()) { - eject(); - } } } // We could also have multiple addresses and get an update for multiple new ones. This is // a no-op as we will just continue to ignore multiple address subchannels. - super.updateAddresses(addresses); + delegate.updateAddresses(addresses); } /** @@ -318,6 +307,10 @@ void uneject() { } } + boolean isEjected() { + return ejected; + } + @Override protected Subchannel delegate() { return delegate; @@ -365,8 +358,7 @@ public PickResult pickSubchannel(PickSubchannelArgs args) { if (subchannel != null) { return PickResult.withSubchannel(subchannel, new ResultCountingClientStreamTracerFactory( - (AddressTracker) subchannel.getAttributes() - .get(EAG_INFO_ATTR_KEY))); + subchannel.getAttributes().get(EAG_INFO_ATTR_KEY))); } return pickResult; @@ -427,6 +419,13 @@ static class AddressTracker { } boolean addSubchannel(OutlierDetectionSubchannel subchannel) { + // Make sure that the subchannel is in the same ejection state as the new tracker it is + // associated with. + if (subchannelsEjected() && !subchannel.isEjected()) { + subchannel.eject(); + } else if (!subchannelsEjected() && subchannel.isEjected()) { + subchannel.uneject(); + } return subchannels.add(subchannel); } @@ -438,6 +437,11 @@ boolean containsSubchannel(OutlierDetectionSubchannel subchannel) { return subchannels.contains(subchannel); } + @VisibleForTesting + Set getSubchannels() { + return ImmutableSet.copyOf(subchannels); + } + void incrementCallCount(boolean success) { // If neither algorithm is configured, no point in incrementing counters. if (config.successRateEjection == null && config.failurePercentageEjection == null) { @@ -451,19 +455,21 @@ void incrementCallCount(boolean success) { } } - /** - * The total number of calls in the inactive call counter. - */ - long volume() { + @VisibleForTesting + long activeVolume() { + return activeCallCounter.successCount.get() + activeCallCounter.failureCount.get(); + } + + long inactiveVolume() { return inactiveCallCounter.successCount.get() + inactiveCallCounter.failureCount.get(); } double successRate() { - return ((double) inactiveCallCounter.successCount.get()) / volume(); + return ((double) inactiveCallCounter.successCount.get()) / inactiveVolume(); } double failureRate() { - return ((double)inactiveCallCounter.failureCount.get()) / volume(); + return ((double)inactiveCallCounter.failureCount.get()) / inactiveVolume(); } void clearCallCounters() { @@ -488,9 +494,6 @@ void swapCounters() { inactiveCallCounter = tempCounter; } - /** - * Ejects the address from use. - */ void ejectSubchannels(long ejectionTimeNanos) { this.ejectionTimeNanos = ejectionTimeNanos; ejectionTimeMultiplier++; @@ -503,7 +506,7 @@ void ejectSubchannels(long ejectionTimeNanos) { * Uneject a currently ejected address. */ void unejectSubchannels() { - checkState(ejectionTimeNanos == null, "not currently ejected"); + checkState(ejectionTimeNanos != null, "not currently ejected"); ejectionTimeNanos = null; for (OutlierDetectionSubchannel subchannel : subchannels) { subchannel.uneject(); @@ -606,7 +609,7 @@ void maybeUnejectOutliers(Long detectionTimerStartNanos) { List trackersWithVolume(OutlierDetectionLoadBalancerConfig config) { List trackersWithVolume = new ArrayList<>(); for (AddressTracker tracker : trackerMap.values()) { - if (tracker.volume() >= config.successRateEjection.requestVolume) { + if (tracker.inactiveVolume() >= config.successRateEjection.requestVolume) { trackersWithVolume.add(tracker); } } @@ -751,7 +754,7 @@ public void ejectOutliers(AddressTrackerMap trackerMap, long ejectionTimeMillis) return; } - if (tracker.volume() < config.failurePercentageEjection.requestVolume) { + if (tracker.inactiveVolume() < config.failurePercentageEjection.requestVolume) { continue; } @@ -796,6 +799,7 @@ private OutlierDetectionLoadBalancerConfig(Long intervalSecs, this.childPolicy = childPolicy; } + /** Builds a new {@link OutlierDetectionLoadBalancerConfig}. */ public static class Builder { Long intervalSecs = 10L; Long baseEjectionTimeSecs = 30L; @@ -805,48 +809,56 @@ public static class Builder { FailurePercentageEjection failurePercentageEjection; PolicySelection childPolicy; + /** The interval between outlier detection sweeps. */ public Builder setIntervalSecs(Long intervalSecs) { checkArgument(intervalSecs != null); this.intervalSecs = intervalSecs; return this; } + /** The base time an address is ejected for. */ public Builder setBaseEjectionTimeSecs(Long baseEjectionTimeSecs) { checkArgument(baseEjectionTimeSecs != null); this.baseEjectionTimeSecs = baseEjectionTimeSecs; return this; } + /** The longest time an address can be ejected. */ public Builder setMaxEjectionTimeSecs(Long maxEjectionTimeSecs) { checkArgument(maxEjectionTimeSecs != null); this.maxEjectionTimeSecs = maxEjectionTimeSecs; return this; } + /** The algorithm agnostic maximum percentage of addresses that can be ejected. */ public Builder setMaxEjectionPercent(Integer maxEjectionPercent) { checkArgument(maxEjectionPercent != null); this.maxEjectionPercent = maxEjectionPercent; return this; } + /** Set to enable success rate eejction. */ public Builder setSuccessRateEjection( SuccessRateEjection successRateEjection) { this.successRateEjection = successRateEjection; return this; } + /** Set to enable failure percentage ejection. */ public Builder setFailurePercentageEjection( FailurePercentageEjection failurePercentageEjection) { this.failurePercentageEjection = failurePercentageEjection; return this; } + /** Sets the child policy the {@link OutlierDetectionLoadBalancer} delegates to. */ public Builder setChildPolicy(PolicySelection childPolicy) { checkState(childPolicy != null); this.childPolicy = childPolicy; return this; } + /** Builds a new instance of {@link OutlierDetectionLoadBalancerConfig}. */ public OutlierDetectionLoadBalancerConfig build() { checkState(childPolicy != null); return new OutlierDetectionLoadBalancerConfig(intervalSecs, baseEjectionTimeSecs, @@ -855,6 +867,7 @@ public OutlierDetectionLoadBalancerConfig build() { } } + /** The configuration for success rate ejection. */ public static class SuccessRateEjection { final Integer stdevFactor; @@ -870,6 +883,7 @@ public static class SuccessRateEjection { this.requestVolume = requestVolume; } + /** Builds new instances of {@link SuccessRateEjection}. */ public static final class Builder { Integer stdevFactor = 1900; @@ -877,30 +891,37 @@ public static final class Builder { Integer minimumHosts = 5; Integer requestVolume = 100; + /** The product of this and the standard deviation of success rates determine the ejection + * threshold. + */ public Builder setStdevFactor(Integer stdevFactor) { checkArgument(stdevFactor != null); this.stdevFactor = stdevFactor; return this; } + /** Only eject this percentage of outliers. */ public Builder setEnforcementPercentage(Integer enforcementPercentage) { checkArgument(enforcementPercentage != null); this.enforcementPercentage = enforcementPercentage; return this; } + /** The minimum amount of hosts needed for success rate ejection. */ public Builder setMinimumHosts(Integer minimumHosts) { checkArgument(minimumHosts != null); this.minimumHosts = minimumHosts; return this; } + /** The minimum address request volume to be considered for success rate ejection. */ public Builder setRequestVolume(Integer requestVolume) { checkArgument(requestVolume != null); this.requestVolume = requestVolume; return this; } + /** Builds a new instance of {@link SuccessRateEjection}. */ public SuccessRateEjection build() { return new SuccessRateEjection(stdevFactor, enforcementPercentage, minimumHosts, requestVolume); @@ -908,6 +929,7 @@ public SuccessRateEjection build() { } } + /** The configuration for failure percentage ejection. */ public static class FailurePercentageEjection { final Integer threshold; final Integer enforcementPercentage; @@ -922,36 +944,45 @@ public static class FailurePercentageEjection { this.requestVolume = requestVolume; } + /** For building new {@link FailurePercentageEjection} instances. */ public static class Builder { Integer threshold = 85; Integer enforcementPercentage = 100; Integer minimumHosts = 5; Integer requestVolume = 50; + /** The failure percentage that will result in an address being considered an outlier. */ public Builder setThreshold(Integer threshold) { checkArgument(threshold != null); this.threshold = threshold; return this; } + /** Only eject this percentage of outliers. */ public Builder setEnforcementPercentage(Integer enforcementPercentage) { checkArgument(enforcementPercentage != null); this.enforcementPercentage = enforcementPercentage; return this; } + /** The minimum amount of host for failure percentage ejection to be enabled. */ public Builder setMinimumHosts(Integer minimumHosts) { checkArgument(minimumHosts != null); this.minimumHosts = minimumHosts; return this; } + /** + * The request volume required for an address to be considered for failure percentage + * ejection. + */ public Builder setRequestVolume(Integer requestVolume) { checkArgument(requestVolume != null); this.requestVolume = requestVolume; return this; } + /** Builds a new instance of {@link FailurePercentageEjection}. */ public FailurePercentageEjection build() { return new FailurePercentageEjection(threshold, enforcementPercentage, minimumHosts, requestVolume); diff --git a/core/src/test/java/io/grpc/internal/FakeClock.java b/core/src/test/java/io/grpc/internal/FakeClock.java index b6a88de00ba..9cc9178f1ff 100644 --- a/core/src/test/java/io/grpc/internal/FakeClock.java +++ b/core/src/test/java/io/grpc/internal/FakeClock.java @@ -248,7 +248,6 @@ class ScheduleWithFixedDelayTask extends ScheduledTask { @Override void run() { - long startTimeNanos = currentTimeNanos; command.run(); if (!isCancelled()) { schedule(this, delayNanos, TimeUnit.NANOSECONDS); diff --git a/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java b/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java index add4ca27522..dee2ce62930 100644 --- a/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java +++ b/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java @@ -44,7 +44,6 @@ import io.grpc.LoadBalancer.SubchannelPicker; import io.grpc.LoadBalancer.SubchannelStateListener; import io.grpc.LoadBalancerProvider; -import io.grpc.LoadBalancerRegistry; import io.grpc.Status; import io.grpc.SynchronizationContext; import io.grpc.internal.FakeClock; @@ -57,10 +56,8 @@ import io.grpc.util.OutlierDetectionLoadBalancer.OutlierDetectionLoadBalancerConfig.SuccessRateEjection; import io.grpc.util.OutlierDetectionLoadBalancer.OutlierDetectionSubchannel; import io.grpc.util.OutlierDetectionLoadBalancer.SuccessRateOutlierEjectionAlgorithm; -import io.grpc.util.RoundRobinLoadBalancer.ReadyPicker; import java.net.SocketAddress; import java.util.Arrays; -import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -68,6 +65,7 @@ import java.util.Set; import java.util.concurrent.TimeUnit; import org.junit.Before; +import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; @@ -89,8 +87,6 @@ public class OutlierDetectionLoadBalancerTest { @Rule public final MockitoRule mockitoRule = MockitoJUnit.rule(); - @Mock - private Helper helper; @Mock private LoadBalancer mockChildLb; @Mock @@ -106,8 +102,6 @@ public class OutlierDetectionLoadBalancerTest { private ArgumentCaptor pickerCaptor; @Captor private ArgumentCaptor stateCaptor; - @Captor - private ArgumentCaptor createArgsCaptor; private final LoadBalancerProvider mockChildLbProvider = new StandardLoadBalancerProvider( "foo_policy") { @@ -132,12 +126,12 @@ public void uncaughtException(Thread t, Throwable e) { throw new AssertionError(e); } }); - private LoadBalancerRegistry lbRegistry = new LoadBalancerRegistry(); private OutlierDetectionLoadBalancer loadBalancer; private final List servers = Lists.newArrayList(); private final Map, Subchannel> subchannels = Maps.newLinkedHashMap(); - private final Map subchannelStateListeners = Maps.newLinkedHashMap(); + private final Map subchannelStateListeners + = Maps.newLinkedHashMap(); private Subchannel subchannel1; private Subchannel subchannel2; @@ -185,8 +179,6 @@ public Void answer(InvocationOnMock invocation) throws Throwable { } }); - lbRegistry.register(roundRobinLbProvider); - loadBalancer = new OutlierDetectionLoadBalancer(mockHelper, fakeClock.getTimeProvider()); } @@ -362,13 +354,13 @@ public void successRateNoOutliers() { loadBalancer.handleResolvedAddresses(buildResolvedAddress(config, servers)); - generateLoad(config, ImmutableMap.of()); + generateLoad(ImmutableMap.of()); // Move forward in time to a point where the detection timer has fired. fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); // No outliers, no ejections. - assertEjectedChannels(ImmutableSet.of()); + assertEjectedSubchannels(ImmutableSet.of()); } /** @@ -384,13 +376,49 @@ public void successRateOneOutlier() { .setRequestVolume(10).build()) .setChildPolicy(new PolicySelection(roundRobinLbProvider, null)).build(); - generateLoad(config, ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); + loadBalancer.handleResolvedAddresses(buildResolvedAddress(config, servers)); + + generateLoad(ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); // Move forward in time to a point where the detection timer has fired. fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); // The one subchannel that was returning errors should be ejected. - assertEjectedChannels(ImmutableSet.of(servers.get(0))); + assertEjectedSubchannels(ImmutableSet.of(servers.get(0))); + } + + /** + * The success rate algorithm ejects the outlier but after some time it should get unejected + * if it stops being an outlier.. + */ + @Test + public void successRateOneOutlier_unejected() { + OutlierDetectionLoadBalancerConfig config = new OutlierDetectionLoadBalancerConfig.Builder() + .setMaxEjectionPercent(50) + .setSuccessRateEjection( + new SuccessRateEjection.Builder() + .setMinimumHosts(3) + .setRequestVolume(10).build()) + .setChildPolicy(new PolicySelection(roundRobinLbProvider, null)).build(); + + loadBalancer.handleResolvedAddresses(buildResolvedAddress(config, servers)); + + generateLoad(ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); + + // Move forward in time to a point where the detection timer has fired. + fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + + // The one subchannel that was returning errors should be ejected. + assertEjectedSubchannels(ImmutableSet.of(servers.get(0))); + + // Now we produce more load, but the subchannel start working and is no longer an outlier. + generateLoad(ImmutableMap.of()); + + // Move forward in time to a point where the detection timer has fired. + fakeClock.forwardTime(config.maxEjectionTimeSecs + 1, TimeUnit.SECONDS); + + // No subchannels should remain ejected. + assertEjectedSubchannels(ImmutableSet.of()); } /** @@ -406,13 +434,15 @@ public void successRateOneOutlier_notEnoughVolume() { .setRequestVolume(100).build()) // We won't produce this much volume... .setChildPolicy(new PolicySelection(roundRobinLbProvider, null)).build(); - generateLoad(config, ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); + loadBalancer.handleResolvedAddresses(buildResolvedAddress(config, servers)); + + generateLoad(ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); // Move forward in time to a point where the detection timer has fired. fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); // The address should not have been ejected.. - assertEjectedChannels(ImmutableSet.of()); + assertEjectedSubchannels(ImmutableSet.of()); } /** @@ -428,13 +458,15 @@ public void successRateOneOutlier_notEnoughAddressesWithVolume() { .setRequestVolume(10).build()) .setChildPolicy(new PolicySelection(roundRobinLbProvider, null)).build(); - generateLoad(config, ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); + loadBalancer.handleResolvedAddresses(buildResolvedAddress(config, servers)); + + generateLoad(ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); // Move forward in time to a point where the detection timer has fired. fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); // No subchannels should have been ejected. - assertEjectedChannels(ImmutableSet.of()); + assertEjectedSubchannels(ImmutableSet.of()); } /** @@ -452,13 +484,15 @@ public void successRateOneOutlier_enforcementPercentage() { .build()) .setChildPolicy(new PolicySelection(roundRobinLbProvider, null)).build(); - generateLoad(config, ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); + loadBalancer.handleResolvedAddresses(buildResolvedAddress(config, servers)); + + generateLoad(ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); // Move forward in time to a point where the detection timer has fired. fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); // There is one outlier, but because enforcementPercentage is 0, nothing should be ejected. - assertEjectedChannels(ImmutableSet.of()); + assertEjectedSubchannels(ImmutableSet.of()); } /** @@ -475,7 +509,9 @@ public void successRateTwoOutliers() { .setStdevFactor(1).build()) .setChildPolicy(new PolicySelection(roundRobinLbProvider, null)).build(); - generateLoad(config, ImmutableMap.of( + loadBalancer.handleResolvedAddresses(buildResolvedAddress(config, servers)); + + generateLoad(ImmutableMap.of( subchannel1, Status.DEADLINE_EXCEEDED, subchannel2, Status.DEADLINE_EXCEEDED)); @@ -483,13 +519,7 @@ public void successRateTwoOutliers() { fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); // The one subchannel that was returning errors should be ejected. - assertThat(loadBalancer.trackerMap.get(servers.get(0)).subchannelsEjected()).isTrue(); - assertThat(loadBalancer.trackerMap.get(servers.get(1)).subchannelsEjected()).isTrue(); - assertThat(loadBalancer.trackerMap.get(servers.get(2)).subchannelsEjected()).isFalse(); - assertThat(loadBalancer.trackerMap.get(servers.get(3)).subchannelsEjected()).isFalse(); - assertThat(loadBalancer.trackerMap.get(servers.get(4)).subchannelsEjected()).isFalse(); - - //assertEjectedChannels(ImmutableSet.of(servers.get(0), servers.get(1))); + assertEjectedSubchannels(ImmutableSet.of(servers.get(0), servers.get(1))); } /** @@ -506,7 +536,9 @@ public void successRateTwoOutliers_maxEjectionPercentage() { .setStdevFactor(1).build()) .setChildPolicy(new PolicySelection(roundRobinLbProvider, null)).build(); - generateLoad(config, ImmutableMap.of( + loadBalancer.handleResolvedAddresses(buildResolvedAddress(config, servers)); + + generateLoad(ImmutableMap.of( subchannel1, Status.DEADLINE_EXCEEDED, subchannel2, Status.DEADLINE_EXCEEDED)); @@ -541,13 +573,13 @@ public void failurePercentageNoOutliers() { loadBalancer.handleResolvedAddresses(buildResolvedAddress(config, servers)); // By default all calls will return OK. - generateLoad(config, ImmutableMap.of()); + generateLoad(ImmutableMap.of()); // Move forward in time to a point where the detection timer has fired. fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); // No outliers, no ejections. - assertEjectedChannels(ImmutableSet.of()); + assertEjectedSubchannels(ImmutableSet.of()); } /** @@ -563,13 +595,15 @@ public void failurePercentageOneOutlier() { .setRequestVolume(10).build()) .setChildPolicy(new PolicySelection(roundRobinLbProvider, null)).build(); - generateLoad(config, ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); + loadBalancer.handleResolvedAddresses(buildResolvedAddress(config, servers)); + + generateLoad(ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); // Move forward in time to a point where the detection timer has fired. fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); // The one subchannel that was returning errors should be ejected. - assertEjectedChannels(ImmutableSet.of(servers.get(0))); + assertEjectedSubchannels(ImmutableSet.of(servers.get(0))); } /** @@ -585,13 +619,15 @@ public void failurePercentageOneOutlier_notEnoughVolume() { .setRequestVolume(100).build()) // We won't produce this much volume... .setChildPolicy(new PolicySelection(roundRobinLbProvider, null)).build(); - generateLoad(config, ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); + loadBalancer.handleResolvedAddresses(buildResolvedAddress(config, servers)); + + generateLoad(ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); // Move forward in time to a point where the detection timer has fired. fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); // We should see no ejections. - assertEjectedChannels(ImmutableSet.of()); + assertEjectedSubchannels(ImmutableSet.of()); } /** @@ -609,13 +645,165 @@ public void failurePercentageOneOutlier_enforcementPercentage() { .build()) .setChildPolicy(new PolicySelection(roundRobinLbProvider, null)).build(); - generateLoad(config, ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); + loadBalancer.handleResolvedAddresses(buildResolvedAddress(config, servers)); + + generateLoad(ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); // Move forward in time to a point where the detection timer has fired. fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); // There is one outlier, but because enforcementPercentage is 0, nothing should be ejected. - assertEjectedChannels(ImmutableSet.of()); + assertEjectedSubchannels(ImmutableSet.of()); + } + + /** + * When the address a subchannel is associated with changes it should get tracked under the new + * address and its ejection state should match what the address has. + */ + @Test + public void subchannelUpdateAddress_singleReplaced() { + OutlierDetectionLoadBalancerConfig config = new OutlierDetectionLoadBalancerConfig.Builder() + .setMaxEjectionPercent(50) + .setFailurePercentageEjection( + new FailurePercentageEjection.Builder() + .setMinimumHosts(3) + .setRequestVolume(10).build()) + .setChildPolicy(new PolicySelection(roundRobinLbProvider, null)).build(); + + loadBalancer.handleResolvedAddresses(buildResolvedAddress(config, servers)); + + generateLoad(ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); + + // Move forward in time to a point where the detection timer has fired. + fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + + EquivalentAddressGroup oldAddress = servers.get(0); + AddressTracker oldAddressTracker = loadBalancer.trackerMap.get(oldAddress); + EquivalentAddressGroup newAddress = servers.get(1); + AddressTracker newAddressTracker = loadBalancer.trackerMap.get(newAddress); + + // The one subchannel that was returning errors should be ejected. + assertEjectedSubchannels(ImmutableSet.of(oldAddress)); + + // The ejected subchannel gets updated with another address in the map that is not ejected + OutlierDetectionSubchannel subchannel = oldAddressTracker.getSubchannels() + .iterator().next(); + subchannel.updateAddresses(ImmutableList.of(newAddress)); + + // The replaced address should no longer have the subchannel associated with it. + assertThat(oldAddressTracker.getSubchannels()).doesNotContain(subchannel); + + // The new address should instead have the subchannel. + assertThat(newAddressTracker.getSubchannels()).contains(subchannel); + + // Since the new address is not ejected, the ejected subchannel moving over to it should also + // become unejected. + assertThat(subchannel.isEjected()).isFalse(); + } + + /** + * If a single address gets replaced by multiple, the subchannel becomes uneligible for outlier + * detection. + */ + @Test + public void subchannelUpdateAddress_singleReplacedWithMultiple() { + OutlierDetectionLoadBalancerConfig config = new OutlierDetectionLoadBalancerConfig.Builder() + .setMaxEjectionPercent(50) + .setFailurePercentageEjection( + new FailurePercentageEjection.Builder() + .setMinimumHosts(3) + .setRequestVolume(10).build()) + .setChildPolicy(new PolicySelection(roundRobinLbProvider, null)).build(); + + loadBalancer.handleResolvedAddresses(buildResolvedAddress(config, servers)); + + generateLoad(ImmutableMap.of()); + + // Move forward in time to a point where the detection timer has fired. + fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + + EquivalentAddressGroup oldAddress = servers.get(0); + AddressTracker oldAddressTracker = loadBalancer.trackerMap.get(oldAddress); + EquivalentAddressGroup newAddress1 = servers.get(1); + EquivalentAddressGroup newAddress2 = servers.get(2); + + OutlierDetectionSubchannel subchannel = oldAddressTracker.getSubchannels() + .iterator().next(); + + // The subchannel gets updated with two new addresses + ImmutableList addressUpdate + = ImmutableList.of(newAddress1, newAddress2); + subchannel.updateAddresses(addressUpdate); + when(subchannel1.getAllAddresses()).thenReturn(addressUpdate); + + // The replaced address should no longer be tracked. + assertThat(oldAddressTracker.getSubchannels()).doesNotContain(subchannel); + + // The old tracker should also have its call counters cleared. + assertThat(oldAddressTracker.activeVolume()).isEqualTo(0); + assertThat(oldAddressTracker.inactiveVolume()).isEqualTo(0); + } + + /** + * A subchannel with multiple addresses will again become eligible for outlier detection if it + * receives an update with a single address. + * + *

TODO: Figure out how to test this scenario, round_robin does not support multiple addresses + * and fails the transition from multiple addresses to single. + */ + @Ignore + public void subchannelUpdateAddress_multipleReplacedWithSingle() { + OutlierDetectionLoadBalancerConfig config = new OutlierDetectionLoadBalancerConfig.Builder() + .setMaxEjectionPercent(50) + .setFailurePercentageEjection( + new FailurePercentageEjection.Builder() + .setMinimumHosts(3) + .setRequestVolume(10).build()) + .setChildPolicy(new PolicySelection(roundRobinLbProvider, null)).build(); + + loadBalancer.handleResolvedAddresses(buildResolvedAddress(config, servers)); + + generateLoad(ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); + + // Move forward in time to a point where the detection timer has fired. + fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + + EquivalentAddressGroup oldAddress = servers.get(0); + AddressTracker oldAddressTracker = loadBalancer.trackerMap.get(oldAddress); + EquivalentAddressGroup newAddress1 = servers.get(1); + AddressTracker newAddressTracker1 = loadBalancer.trackerMap.get(newAddress1); + EquivalentAddressGroup newAddress2 = servers.get(2); + + // The old subchannel was returning errors and should be ejected. + assertEjectedSubchannels(ImmutableSet.of(oldAddress)); + + OutlierDetectionSubchannel subchannel = oldAddressTracker.getSubchannels() + .iterator().next(); + + // The subchannel gets updated with two new addresses + ImmutableList addressUpdate + = ImmutableList.of(newAddress1, newAddress2); + subchannel.updateAddresses(addressUpdate); + when(subchannel1.getAllAddresses()).thenReturn(addressUpdate); + + // The replaced address should no longer be tracked. + assertThat(oldAddressTracker.getSubchannels()).doesNotContain(subchannel); + + // The old tracker should also have its call counters cleared. + assertThat(oldAddressTracker.activeVolume()).isEqualTo(0); + assertThat(oldAddressTracker.inactiveVolume()).isEqualTo(0); + + // Another update takes the subchannel back to a single address. + addressUpdate = ImmutableList.of(newAddress1); + subchannel.updateAddresses(addressUpdate); + when(subchannel1.getAllAddresses()).thenReturn(addressUpdate); + + // The subchannel is now associated with the single new address. + assertThat(newAddressTracker1.getSubchannels()).contains(subchannel); + + // The previously ejected subchannel should become unejected as it is now associated with an + // unejected address. + assertThat(subchannel.isEjected()).isFalse(); } @Test @@ -658,16 +846,8 @@ private void deliverSubchannelState(Subchannel subchannel, ConnectivityStateInfo subchannelStateListeners.get(subchannel).onSubchannelState(newState); } - private static List getList(SubchannelPicker picker) { - return picker instanceof ReadyPicker ? ((ReadyPicker) picker).getList() - : Collections.emptyList(); - } - - // Generates 30 calls, 10 each across the subchannels. - private void generateLoad(OutlierDetectionLoadBalancerConfig config, - Map statusMap) { - loadBalancer.handleResolvedAddresses(buildResolvedAddress(config, servers)); - + // Generates 100 calls, 20 each across the subchannels. Default status is OK. + private void generateLoad(Map statusMap) { deliverSubchannelState(subchannel1, ConnectivityStateInfo.forNonError(READY)); deliverSubchannelState(subchannel2, ConnectivityStateInfo.forNonError(READY)); deliverSubchannelState(subchannel3, ConnectivityStateInfo.forNonError(READY)); @@ -684,7 +864,6 @@ private void generateLoad(OutlierDetectionLoadBalancerConfig config, ClientStreamTracer clientStreamTracer = pickResult.getStreamTracerFactory() .newClientStreamTracer(null, null); - // Fail all the calls for the first subchannel. Subchannel subchannel = ((OutlierDetectionSubchannel) pickResult.getSubchannel()).delegate(); clientStreamTracer.streamClosed( statusMap.containsKey(subchannel) ? statusMap.get(subchannel) : Status.OK); @@ -692,7 +871,7 @@ private void generateLoad(OutlierDetectionLoadBalancerConfig config, } // Asserts that the given addresses are ejected and the rest are not. - void assertEjectedChannels(Set addresses) { + void assertEjectedSubchannels(Set addresses) { for (Entry entry : loadBalancer.trackerMap.entrySet()) { assertThat(entry.getValue().subchannelsEjected()).isEqualTo( addresses.contains(entry.getKey())); From 4b8e58eaa3b494040d1d32194a17bfa952fb0da1 Mon Sep 17 00:00:00 2001 From: Terry Wilson Date: Mon, 15 Aug 2022 13:17:51 -0700 Subject: [PATCH 06/18] Reset inactive call counter when swapping call counters. --- .../util/OutlierDetectionLoadBalancer.java | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java b/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java index a0676dea610..85ae8a02d32 100644 --- a/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java +++ b/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java @@ -268,7 +268,7 @@ public void updateAddresses(List addresses) { if (trackerMap.containsKey(getAddresses())) { AddressTracker tracker = trackerMap.get(getAddresses()); tracker.removeSubchannel(this); - tracker.clearCallCounters(); + tracker.resetCallCounters(); } } else if (getAllAddresses().size() > 1 && addresses.size() == 1) { // We go from, previously uneligble, multiple address mode to a single address. If the map @@ -472,11 +472,9 @@ long inactiveVolume() { return ((double)inactiveCallCounter.failureCount.get()) / inactiveVolume(); } - void clearCallCounters() { - activeCallCounter.successCount.set(0); - activeCallCounter.failureCount.set(0); - inactiveCallCounter.successCount.set(0); - inactiveCallCounter.failureCount.set(0); + void resetCallCounters() { + activeCallCounter.reset(); + inactiveCallCounter.reset(); } void decrementEjectionTimeMultiplier() { @@ -489,6 +487,7 @@ void resetEjectionTimeMultiplier() { } void swapCounters() { + inactiveCallCounter.reset(); CallCounter tempCounter = activeCallCounter; activeCallCounter = inactiveCallCounter; inactiveCallCounter = tempCounter; @@ -529,9 +528,13 @@ public boolean maxEjectionTimeElapsed(long currentTimeNanos) { } private static class CallCounter { - AtomicLong successCount = new AtomicLong(); AtomicLong failureCount = new AtomicLong(); + + void reset() { + successCount.set(0); + failureCount.set(0); + } } } @@ -563,7 +566,7 @@ void putNewTrackers(OutlierDetectionLoadBalancerConfig config, /** Resets the call counters for all the trackers in the map. */ void resetCallCounters() { for (AddressTracker tracker : trackerMap.values()) { - tracker.clearCallCounters(); + tracker.resetCallCounters(); } } From 4278053ed62eec0f56c5cd175f3e142b344d7654 Mon Sep 17 00:00:00 2001 From: Terry Wilson Date: Mon, 15 Aug 2022 14:09:08 -0700 Subject: [PATCH 07/18] Fix bug in min request volume logic. --- .../util/OutlierDetectionLoadBalancer.java | 2 +- .../OutlierDetectionLoadBalancerTest.java | 32 +++++++++++++++---- 2 files changed, 26 insertions(+), 8 deletions(-) diff --git a/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java b/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java index 85ae8a02d32..6319fd6010a 100644 --- a/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java +++ b/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java @@ -693,7 +693,7 @@ public void ejectOutliers(AddressTrackerMap trackerMap, long ejectionTimeNanos) double mean = mean(successRates); double stdev = standardDeviation(successRates, mean); - for (AddressTracker tracker : trackerMap.values()) { + for (AddressTracker tracker : trackersWithVolume) { // If an ejection now would take us past the max configured ejection percentagem stop here. if (trackerMap.nextEjectionPercentage() > config.maxEjectionPercent) { return; diff --git a/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java b/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java index dee2ce62930..2aeaf71c4b4 100644 --- a/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java +++ b/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java @@ -58,6 +58,7 @@ import io.grpc.util.OutlierDetectionLoadBalancer.SuccessRateOutlierEjectionAlgorithm; import java.net.SocketAddress; import java.util.Arrays; +import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -422,7 +423,7 @@ public void successRateOneOutlier_unejected() { } /** - * The success rate algorithm ignores addresses without enough volume.. + * The success rate algorithm ignores addresses without enough volume. */ @Test public void successRateOneOutlier_notEnoughVolume() { @@ -431,17 +432,20 @@ public void successRateOneOutlier_notEnoughVolume() { .setSuccessRateEjection( new SuccessRateEjection.Builder() .setMinimumHosts(3) - .setRequestVolume(100).build()) // We won't produce this much volume... + .setRequestVolume(20).build()) .setChildPolicy(new PolicySelection(roundRobinLbProvider, null)).build(); loadBalancer.handleResolvedAddresses(buildResolvedAddress(config, servers)); - generateLoad(ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); + // We produce an outlier, but don't give it enough calls to reach the minimum volume. + generateLoad( + ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED), + ImmutableMap.of(subchannel1, 19)); // Move forward in time to a point where the detection timer has fired. fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); - // The address should not have been ejected.. + // The address should not have been ejected. assertEjectedSubchannels(ImmutableSet.of()); } @@ -846,8 +850,13 @@ private void deliverSubchannelState(Subchannel subchannel, ConnectivityStateInfo subchannelStateListeners.get(subchannel).onSubchannelState(newState); } - // Generates 100 calls, 20 each across the subchannels. Default status is OK. private void generateLoad(Map statusMap) { + generateLoad(statusMap, null); + } + + // Generates 100 calls, 20 each across the subchannels. Default status is OK. + private void generateLoad(Map statusMap, + Map maxCallsMap) { deliverSubchannelState(subchannel1, ConnectivityStateInfo.forNonError(READY)); deliverSubchannelState(subchannel2, ConnectivityStateInfo.forNonError(READY)); deliverSubchannelState(subchannel3, ConnectivityStateInfo.forNonError(READY)); @@ -858,6 +867,7 @@ private void generateLoad(Map statusMap) { pickerCaptor.capture()); SubchannelPicker picker = pickerCaptor.getAllValues().get(6); + HashMap callCountMap = new HashMap<>(); for (int i = 0; i < 100; i++) { PickResult pickResult = picker .pickSubchannel(mock(PickSubchannelArgs.class)); @@ -865,8 +875,16 @@ private void generateLoad(Map statusMap) { .newClientStreamTracer(null, null); Subchannel subchannel = ((OutlierDetectionSubchannel) pickResult.getSubchannel()).delegate(); - clientStreamTracer.streamClosed( - statusMap.containsKey(subchannel) ? statusMap.get(subchannel) : Status.OK); + + int maxCalls = + maxCallsMap != null && maxCallsMap.containsKey(subchannel) + ? maxCallsMap.get(subchannel) : Integer.MAX_VALUE; + int calls = callCountMap.containsKey(subchannel) ? callCountMap.get(subchannel) : 0; + if (calls < maxCalls) { + callCountMap.put(subchannel, ++calls); + clientStreamTracer.streamClosed( + statusMap.containsKey(subchannel) ? statusMap.get(subchannel) : Status.OK); + } } } From 0191a3b012e4b0e1380e561f8c1eed0c4693cf93 Mon Sep 17 00:00:00 2001 From: Terry Wilson Date: Mon, 15 Aug 2022 14:45:05 -0700 Subject: [PATCH 08/18] Support running multiple ejection algorithms. --- .../util/OutlierDetectionLoadBalancer.java | 21 ++--- .../OutlierDetectionLoadBalancerTest.java | 80 +++++++++++++++++++ 2 files changed, 92 insertions(+), 9 deletions(-) diff --git a/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java b/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java index 6319fd6010a..c11a174efe2 100644 --- a/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java +++ b/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java @@ -24,6 +24,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ForwardingMap; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import io.grpc.Attributes; @@ -164,8 +165,9 @@ public void run() { trackerMap.swapCounters(); - OutlierEjectionAlgorithm.forConfig(config) - .ejectOutliers(trackerMap, detectionTimerStartNanos); + for (OutlierEjectionAlgorithm algo : OutlierEjectionAlgorithm.forConfig(config)) { + algo.ejectOutliers(trackerMap, detectionTimerStartNanos); + } trackerMap.maybeUnejectOutliers(detectionTimerStartNanos); } @@ -649,14 +651,15 @@ interface OutlierEjectionAlgorithm { void ejectOutliers(AddressTrackerMap trackerMap, long ejectionTimeMillis); @Nullable - static OutlierEjectionAlgorithm forConfig(OutlierDetectionLoadBalancerConfig config) { + static List forConfig(OutlierDetectionLoadBalancerConfig config) { + ImmutableList.Builder algoListBuilder = ImmutableList.builder(); if (config.successRateEjection != null) { - return new SuccessRateOutlierEjectionAlgorithm(config); - } else if (config.failurePercentageEjection != null) { - return new FailurePercentageOutlierEjectionAlgorithm(config); - } else { - return null; + algoListBuilder.add(new SuccessRateOutlierEjectionAlgorithm(config)); } + if (config.failurePercentageEjection != null) { + algoListBuilder.add(new FailurePercentageOutlierEjectionAlgorithm(config)); + } + return algoListBuilder.build(); } } @@ -752,7 +755,7 @@ public void ejectOutliers(AddressTrackerMap trackerMap, long ejectionTimeMillis) // If this address does not have enough volume to be considered, skip to the next one. for (AddressTracker tracker : trackerMap.values()) { - // If an ejection now would take us past the max configured ejection percentagem stop here. + // If an ejection now would take us past the max configured ejection percentage stop here. if (trackerMap.nextEjectionPercentage() > config.maxEjectionPercent) { return; } diff --git a/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java b/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java index 2aeaf71c4b4..407cd1071b4 100644 --- a/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java +++ b/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java @@ -810,6 +810,86 @@ public void subchannelUpdateAddress_multipleReplacedWithSingle() { assertThat(subchannel.isEjected()).isFalse(); } + /** Both algorithms configured, but no outliers. */ + @Test + public void successRateAndFailurePercentage_noOutliers() { + OutlierDetectionLoadBalancerConfig config = new OutlierDetectionLoadBalancerConfig.Builder() + .setMaxEjectionPercent(50) + .setSuccessRateEjection( + new SuccessRateEjection.Builder() + .setMinimumHosts(3) + .setRequestVolume(10).build()) + .setFailurePercentageEjection( + new FailurePercentageEjection.Builder() + .setMinimumHosts(3) + .setRequestVolume(10).build()) + .setChildPolicy(new PolicySelection(roundRobinLbProvider, null)).build(); + + loadBalancer.handleResolvedAddresses(buildResolvedAddress(config, servers)); + + generateLoad(ImmutableMap.of()); + + // Move forward in time to a point where the detection timer has fired. + fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + + // No outliers, no ejections. + assertEjectedSubchannels(ImmutableSet.of()); + } + + /** Both algorithms configured, success rate detects an outlier. */ + @Test + public void successRateAndFailurePercentage_successRateOutlier() { + OutlierDetectionLoadBalancerConfig config = new OutlierDetectionLoadBalancerConfig.Builder() + .setMaxEjectionPercent(50) + .setSuccessRateEjection( + new SuccessRateEjection.Builder() + .setMinimumHosts(3) + .setRequestVolume(10).build()) + .setFailurePercentageEjection( + new FailurePercentageEjection.Builder() + .setMinimumHosts(3) + .setRequestVolume(10) + .setEnforcementPercentage(0).build()) // Configured, but not enforcing. + .setChildPolicy(new PolicySelection(roundRobinLbProvider, null)).build(); + + loadBalancer.handleResolvedAddresses(buildResolvedAddress(config, servers)); + + generateLoad(ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); + + // Move forward in time to a point where the detection timer has fired. + fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + + // The one subchannel that was returning errors should be ejected. + assertEjectedSubchannels(ImmutableSet.of(servers.get(0))); + } + + /** Both algorithms configured, error percentage detects an outlier. */ + @Test + public void successRateAndFailurePercentage_errorPercentageOutlier() { + OutlierDetectionLoadBalancerConfig config = new OutlierDetectionLoadBalancerConfig.Builder() + .setMaxEjectionPercent(50) + .setSuccessRateEjection( + new SuccessRateEjection.Builder() + .setMinimumHosts(3) + .setRequestVolume(10) + .setEnforcementPercentage(0).build()) + .setFailurePercentageEjection( + new FailurePercentageEjection.Builder() + .setMinimumHosts(3) + .setRequestVolume(10).build()) // Configured, but not enforcing. + .setChildPolicy(new PolicySelection(roundRobinLbProvider, null)).build(); + + loadBalancer.handleResolvedAddresses(buildResolvedAddress(config, servers)); + + generateLoad(ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); + + // Move forward in time to a point where the detection timer has fired. + fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + + // The one subchannel that was returning errors should be ejected. + assertEjectedSubchannels(ImmutableSet.of(servers.get(0))); + } + @Test public void mathChecksOut() { ImmutableList values = ImmutableList.of(600d, 470d, 170d, 430d, 300d); From ae969a868cf17d8734cd293fbc0757873a55bbef Mon Sep 17 00:00:00 2001 From: Terry Wilson Date: Mon, 15 Aug 2022 15:14:42 -0700 Subject: [PATCH 09/18] Add some comments. --- .../io/grpc/util/OutlierDetectionLoadBalancer.java | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java b/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java index c11a174efe2..04db07897bc 100644 --- a/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java +++ b/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java @@ -420,6 +420,10 @@ static class AddressTracker { this.config = config; } + /** + * Adds a subchannel to the tracker, while assuring that the subchannel ejection status is + * updated to match the tracker's if needed. + */ boolean addSubchannel(OutlierDetectionSubchannel subchannel) { // Make sure that the subchannel is in the same ejection state as the new tracker it is // associated with. @@ -488,6 +492,13 @@ void resetEjectionTimeMultiplier() { ejectionTimeMultiplier = 0; } + /** + * Swaps the active and inactive counters. + * + *

Note that this method is not thread safe as the swap is not done atomically. This is + * expected to only be called from the timer that is scheduled at a fixed delay, assuring that + * only one timer is active at a time. + */ void swapCounters() { inactiveCallCounter.reset(); CallCounter tempCounter = activeCallCounter; @@ -529,6 +540,7 @@ public boolean maxEjectionTimeElapsed(long currentTimeNanos) { return currentTimeNanos > maxEjectionTimeNanos; } + /** Tracks both successful and failed call counts. */ private static class CallCounter { AtomicLong successCount = new AtomicLong(); AtomicLong failureCount = new AtomicLong(); From 4ba771a05fc6e921c2bc0884d03d8131de308249 Mon Sep 17 00:00:00 2001 From: Terry Wilson Date: Tue, 16 Aug 2022 11:25:41 -0700 Subject: [PATCH 10/18] Properly track SocketAddress instead of EquivalentAddressGroup. --- .../util/OutlierDetectionLoadBalancer.java | 106 +++++++++++------- .../OutlierDetectionLoadBalancerTest.java | 60 +++++----- 2 files changed, 101 insertions(+), 65 deletions(-) diff --git a/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java b/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java index 04db07897bc..51ca8c89e0b 100644 --- a/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java +++ b/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java @@ -40,6 +40,7 @@ import io.grpc.SynchronizationContext.ScheduledHandle; import io.grpc.internal.ServiceConfigUtil.PolicySelection; import io.grpc.internal.TimeProvider; +import java.net.SocketAddress; import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; @@ -73,8 +74,8 @@ public class OutlierDetectionLoadBalancer extends LoadBalancer { private ScheduledHandle detectionTimerHandle; private Long detectionTimerStartNanos; - private static final Attributes.Key EAG_INFO_ATTR_KEY = Attributes.Key.create( - "eagInfoKey"); + private static final Attributes.Key ADDRESS_TRACKER_ATTR_KEY + = Attributes.Key.create("addressTrackerKey"); /** * Creates a new instance of {@link OutlierDetectionLoadBalancer}. @@ -94,10 +95,14 @@ public void handleResolvedAddresses(ResolvedAddresses resolvedAddresses) { = (OutlierDetectionLoadBalancerConfig) resolvedAddresses.getLoadBalancingPolicyConfig(); // The map should only retain entries for addresses in this latest update. - trackerMap.keySet().retainAll(resolvedAddresses.getAddresses()); + ArrayList addresses = new ArrayList<>(); + for (EquivalentAddressGroup addressGroup : resolvedAddresses.getAddresses()) { + addresses.addAll(addressGroup.getAddresses()); + } + trackerMap.keySet().retainAll(addresses); // Add any new ones. - trackerMap.putNewTrackers(config, resolvedAddresses.getAddresses()); + trackerMap.putNewTrackers(config, addresses); switchLb.switchTo(config.childPolicy.getProvider()); @@ -199,14 +204,14 @@ public Subchannel createSubchannel(CreateSubchannelArgs args) { // If the subchannel is associated with a single address that is also already in the map // the subchannel will be added to the map and be included in outlier detection. - List allAddresses = subchannel.getAllAddresses(); - if (allAddresses.size() == 1 && trackerMap.containsKey(allAddresses.get(0))) { - AddressTracker eagInfo = trackerMap.get(allAddresses.get(0)); - subchannel.setEquivalentAddressGroupInfo(eagInfo); - eagInfo.addSubchannel(subchannel); + List addressGroups = subchannel.getAllAddresses(); + if (hasSingleAddress(addressGroups) + && trackerMap.containsKey(addressGroups.get(0).getAddresses().get(0))) { + AddressTracker tracker = trackerMap.get(addressGroups.get(0).getAddresses().get(0)); + tracker.addSubchannel(subchannel); // If this address has already been ejected, we need to immediately eject this Subchannel. - if (eagInfo.ejectionTimeNanos != null) { + if (tracker.ejectionTimeNanos != null) { subchannel.eject(); } } @@ -223,7 +228,7 @@ public void updateBalancingState(ConnectivityState newState, SubchannelPicker ne class OutlierDetectionSubchannel extends ForwardingSubchannel { private final Subchannel delegate; - private AddressTracker eagInfo; + private AddressTracker addressTracker; private boolean ejected; private ConnectivityStateInfo lastSubchannelState; private OutlierDetectionSubchannelStateListener subchannelStateListener; @@ -240,60 +245,70 @@ public void start(SubchannelStateListener listener) { @Override public Attributes getAttributes() { - return delegate.getAttributes().toBuilder().set(EAG_INFO_ATTR_KEY, eagInfo).build(); + if (addressTracker != null) { + return delegate.getAttributes().toBuilder().set(ADDRESS_TRACKER_ATTR_KEY, addressTracker).build(); + } else { + return delegate.getAttributes(); + } } @Override - public void updateAddresses(List addresses) { + public void updateAddresses(List addressGroups) { // Outlier detection only supports subchannels with a single address, but the list of - // addresses associated with a subchannel can change at any time, so we need to react to + // addressGroups associated with a subchannel can change at any time, so we need to react to // changes in the address list plurality. // No change in address plurality, we replace the single one with a new one. - if (getAllAddresses().size() == 1 && addresses.size() == 1) { + if (hasSingleAddress(getAllAddresses()) && hasSingleAddress(addressGroups)) { // Remove the current subchannel from the old address it is associated with in the map. - if (trackerMap.containsKey(getAddresses())) { - trackerMap.get(getAddresses()).removeSubchannel(this); + if (trackerMap.containsValue(addressTracker)) { + addressTracker.removeSubchannel(this); } // If the map has an entry for the new address, we associate this subchannel with it. - EquivalentAddressGroup newAddress = Iterables.getOnlyElement(addresses); - if (trackerMap.containsKey(newAddress)) { - AddressTracker tracker = trackerMap.get(newAddress); - tracker.addSubchannel(this); + SocketAddress address = Iterables.getOnlyElement( + Iterables.getOnlyElement(addressGroups).getAddresses()); + if (trackerMap.containsKey(address)) { + trackerMap.get(address).addSubchannel(this); } - } else if (getAllAddresses().size() == 1 && addresses.size() > 1) { + } else if (hasSingleAddress(getAllAddresses()) && !hasSingleAddress(addressGroups)) { // We go from a single address to having multiple, making this subchannel uneligible for // outlier detection. Remove it from all trackers and reset the call counters of all the // associated trackers. // Remove the current subchannel from the old address it is associated with in the map. - if (trackerMap.containsKey(getAddresses())) { - AddressTracker tracker = trackerMap.get(getAddresses()); + if (trackerMap.containsKey(Iterables.getOnlyElement(getAddresses().getAddresses()))) { + AddressTracker tracker = trackerMap.get( + Iterables.getOnlyElement(getAddresses().getAddresses())); tracker.removeSubchannel(this); tracker.resetCallCounters(); } - } else if (getAllAddresses().size() > 1 && addresses.size() == 1) { + } else if (!hasSingleAddress(getAllAddresses()) && hasSingleAddress(addressGroups)) { // We go from, previously uneligble, multiple address mode to a single address. If the map // has an entry for the new address, we associate this subchannel with it. - EquivalentAddressGroup eag = Iterables.getOnlyElement(addresses); - if (trackerMap.containsKey(eag)) { - AddressTracker tracker = trackerMap.get(eag); + SocketAddress address = Iterables.getOnlyElement( + Iterables.getOnlyElement(addressGroups).getAddresses()); + if (trackerMap.containsKey(address)) { + AddressTracker tracker = trackerMap.get(address); tracker.addSubchannel(this); } } - // We could also have multiple addresses and get an update for multiple new ones. This is + // We could also have multiple addressGroups and get an update for multiple new ones. This is // a no-op as we will just continue to ignore multiple address subchannels. - delegate.updateAddresses(addresses); + delegate.updateAddresses(addressGroups); } /** * If the {@link Subchannel} is considered for outlier detection the associated {@link * AddressTracker} should be set. */ - void setEquivalentAddressGroupInfo(AddressTracker eagInfo) { - this.eagInfo = eagInfo; + void setAddressTracker(AddressTracker addressTracker) { + this.addressTracker = addressTracker; + } + + void clearAddressTracker() { + this.addressTracker = null; } void eject() { @@ -360,7 +375,7 @@ public PickResult pickSubchannel(PickSubchannelArgs args) { if (subchannel != null) { return PickResult.withSubchannel(subchannel, new ResultCountingClientStreamTracerFactory( - subchannel.getAttributes().get(EAG_INFO_ATTR_KEY))); + subchannel.getAttributes().get(ADDRESS_TRACKER_ATTR_KEY))); } return pickResult; @@ -432,10 +447,12 @@ boolean addSubchannel(OutlierDetectionSubchannel subchannel) { } else if (!subchannelsEjected() && subchannel.isEjected()) { subchannel.uneject(); } + subchannel.setAddressTracker(this); return subchannels.add(subchannel); } boolean removeSubchannel(OutlierDetectionSubchannel subchannel) { + subchannel.clearAddressTracker(); return subchannels.remove(subchannel); } @@ -555,22 +572,22 @@ void reset() { /** * Maintains a mapping from addresses to their trackers. */ - static class AddressTrackerMap extends ForwardingMap { - private final Map trackerMap; + static class AddressTrackerMap extends ForwardingMap { + private final Map trackerMap; AddressTrackerMap() { trackerMap = new HashMap<>(); } @Override - protected Map delegate() { + protected Map delegate() { return trackerMap; } - /** Adds a new tracker for the addresses that don't already have one. */ + /** Adds a new tracker for every address in the given EAGs. */ void putNewTrackers(OutlierDetectionLoadBalancerConfig config, - Collection addresses) { - for (EquivalentAddressGroup address : addresses) { + Collection addresses) { + for (SocketAddress address : addresses) { if (!trackerMap.containsKey(address)) { trackerMap.put(address, new AddressTracker(config)); } @@ -788,6 +805,17 @@ public void ejectOutliers(AddressTrackerMap trackerMap, long ejectionTimeMillis) } } + /** Counts how many addresses are in a given address group. */ + private static boolean hasSingleAddress(List addressGroups) { + int addressCount = 0; + for (EquivalentAddressGroup addressGroup : addressGroups) { + addressCount += addressGroup.getAddresses().size(); + if (addressCount > 1) { + return false; + } + } + return true; + } /** * The configuration for {@link OutlierDetectionLoadBalancer}. diff --git a/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java b/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java index 407cd1071b4..39b3208ec23 100644 --- a/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java +++ b/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java @@ -385,7 +385,7 @@ public void successRateOneOutlier() { fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); // The one subchannel that was returning errors should be ejected. - assertEjectedSubchannels(ImmutableSet.of(servers.get(0))); + assertEjectedSubchannels(ImmutableSet.of(servers.get(0).getAddresses().get(0))); } /** @@ -410,7 +410,7 @@ public void successRateOneOutlier_unejected() { fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); // The one subchannel that was returning errors should be ejected. - assertEjectedSubchannels(ImmutableSet.of(servers.get(0))); + assertEjectedSubchannels(ImmutableSet.of(servers.get(0).getAddresses().get(0))); // Now we produce more load, but the subchannel start working and is no longer an outlier. generateLoad(ImmutableMap.of()); @@ -523,7 +523,8 @@ public void successRateTwoOutliers() { fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); // The one subchannel that was returning errors should be ejected. - assertEjectedSubchannels(ImmutableSet.of(servers.get(0), servers.get(1))); + assertEjectedSubchannels(ImmutableSet.of(servers.get(0).getAddresses().get(0), + servers.get(1).getAddresses().get(0))); } /** @@ -550,8 +551,10 @@ public void successRateTwoOutliers_maxEjectionPercentage() { fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); int totalEjected = 0; - for (EquivalentAddressGroup address: servers) { - totalEjected += loadBalancer.trackerMap.get(address).subchannelsEjected() ? 1 : 0; + for (EquivalentAddressGroup addressGroup: servers) { + totalEjected += + loadBalancer.trackerMap.get(addressGroup.getAddresses().get(0)).subchannelsEjected() ? 1 + : 0; } // Even if all subchannels were failing, we should have not ejected more than the configured @@ -607,7 +610,7 @@ public void failurePercentageOneOutlier() { fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); // The one subchannel that was returning errors should be ejected. - assertEjectedSubchannels(ImmutableSet.of(servers.get(0))); + assertEjectedSubchannels(ImmutableSet.of(servers.get(0).getAddresses().get(0))); } /** @@ -681,18 +684,20 @@ public void subchannelUpdateAddress_singleReplaced() { // Move forward in time to a point where the detection timer has fired. fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); - EquivalentAddressGroup oldAddress = servers.get(0); - AddressTracker oldAddressTracker = loadBalancer.trackerMap.get(oldAddress); - EquivalentAddressGroup newAddress = servers.get(1); - AddressTracker newAddressTracker = loadBalancer.trackerMap.get(newAddress); + EquivalentAddressGroup oldAddressGroup = servers.get(0); + AddressTracker oldAddressTracker = loadBalancer.trackerMap.get( + oldAddressGroup.getAddresses().get(0)); + EquivalentAddressGroup newAddressGroup = servers.get(1); + AddressTracker newAddressTracker = loadBalancer.trackerMap.get( + newAddressGroup.getAddresses().get(0)); // The one subchannel that was returning errors should be ejected. - assertEjectedSubchannels(ImmutableSet.of(oldAddress)); + assertEjectedSubchannels(ImmutableSet.of(oldAddressGroup.getAddresses().get(0))); // The ejected subchannel gets updated with another address in the map that is not ejected OutlierDetectionSubchannel subchannel = oldAddressTracker.getSubchannels() .iterator().next(); - subchannel.updateAddresses(ImmutableList.of(newAddress)); + subchannel.updateAddresses(ImmutableList.of(newAddressGroup)); // The replaced address should no longer have the subchannel associated with it. assertThat(oldAddressTracker.getSubchannels()).doesNotContain(subchannel); @@ -726,8 +731,9 @@ public void subchannelUpdateAddress_singleReplacedWithMultiple() { // Move forward in time to a point where the detection timer has fired. fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); - EquivalentAddressGroup oldAddress = servers.get(0); - AddressTracker oldAddressTracker = loadBalancer.trackerMap.get(oldAddress); + EquivalentAddressGroup oldAddressGroup = servers.get(0); + AddressTracker oldAddressTracker = loadBalancer.trackerMap.get( + oldAddressGroup.getAddresses().get(0)); EquivalentAddressGroup newAddress1 = servers.get(1); EquivalentAddressGroup newAddress2 = servers.get(2); @@ -772,21 +778,23 @@ public void subchannelUpdateAddress_multipleReplacedWithSingle() { // Move forward in time to a point where the detection timer has fired. fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); - EquivalentAddressGroup oldAddress = servers.get(0); - AddressTracker oldAddressTracker = loadBalancer.trackerMap.get(oldAddress); - EquivalentAddressGroup newAddress1 = servers.get(1); - AddressTracker newAddressTracker1 = loadBalancer.trackerMap.get(newAddress1); - EquivalentAddressGroup newAddress2 = servers.get(2); + EquivalentAddressGroup oldAddressGroup = servers.get(0); + AddressTracker oldAddressTracker = loadBalancer.trackerMap.get( + oldAddressGroup.getAddresses().get(0)); + EquivalentAddressGroup newAddressGroup1 = servers.get(1); + AddressTracker newAddressTracker1 = loadBalancer.trackerMap.get( + newAddressGroup1.getAddresses().get(0)); + EquivalentAddressGroup newAddressGroup2 = servers.get(2); // The old subchannel was returning errors and should be ejected. - assertEjectedSubchannels(ImmutableSet.of(oldAddress)); + assertEjectedSubchannels(ImmutableSet.of(oldAddressGroup.getAddresses().get(0))); OutlierDetectionSubchannel subchannel = oldAddressTracker.getSubchannels() .iterator().next(); // The subchannel gets updated with two new addresses ImmutableList addressUpdate - = ImmutableList.of(newAddress1, newAddress2); + = ImmutableList.of(newAddressGroup1, newAddressGroup2); subchannel.updateAddresses(addressUpdate); when(subchannel1.getAllAddresses()).thenReturn(addressUpdate); @@ -798,7 +806,7 @@ public void subchannelUpdateAddress_multipleReplacedWithSingle() { assertThat(oldAddressTracker.inactiveVolume()).isEqualTo(0); // Another update takes the subchannel back to a single address. - addressUpdate = ImmutableList.of(newAddress1); + addressUpdate = ImmutableList.of(newAddressGroup1); subchannel.updateAddresses(addressUpdate); when(subchannel1.getAllAddresses()).thenReturn(addressUpdate); @@ -860,7 +868,7 @@ public void successRateAndFailurePercentage_successRateOutlier() { fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); // The one subchannel that was returning errors should be ejected. - assertEjectedSubchannels(ImmutableSet.of(servers.get(0))); + assertEjectedSubchannels(ImmutableSet.of(servers.get(0).getAddresses().get(0))); } /** Both algorithms configured, error percentage detects an outlier. */ @@ -887,7 +895,7 @@ public void successRateAndFailurePercentage_errorPercentageOutlier() { fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); // The one subchannel that was returning errors should be ejected. - assertEjectedSubchannels(ImmutableSet.of(servers.get(0))); + assertEjectedSubchannels(ImmutableSet.of(servers.get(0).getAddresses().get(0))); } @Test @@ -969,8 +977,8 @@ private void generateLoad(Map statusMap, } // Asserts that the given addresses are ejected and the rest are not. - void assertEjectedSubchannels(Set addresses) { - for (Entry entry : loadBalancer.trackerMap.entrySet()) { + void assertEjectedSubchannels(Set addresses) { + for (Entry entry : loadBalancer.trackerMap.entrySet()) { assertThat(entry.getValue().subchannelsEjected()).isEqualTo( addresses.contains(entry.getKey())); } From d92bc96806d4dca6b7e12534612b5bdfcab85c6d Mon Sep 17 00:00:00 2001 From: Terry Wilson Date: Tue, 16 Aug 2022 13:31:29 -0700 Subject: [PATCH 11/18] Mark active counter volatile. --- .../main/java/io/grpc/util/OutlierDetectionLoadBalancer.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java b/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java index 51ca8c89e0b..f9e27017a88 100644 --- a/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java +++ b/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java @@ -425,7 +425,9 @@ public void streamClosed(Status status) { static class AddressTracker { private final OutlierDetectionLoadBalancerConfig config; - private CallCounter activeCallCounter = new CallCounter(); + // Marked as volatile to assure that when the inactive counter is swapped in as the new active + // one, all threads see the change and don't hold on to a reference to the now inactive counter. + private volatile CallCounter activeCallCounter = new CallCounter(); private CallCounter inactiveCallCounter = new CallCounter(); private Long ejectionTimeNanos; private int ejectionTimeMultiplier; From 3695f641f00146d944bf92a5092fec2b81eed2c7 Mon Sep 17 00:00:00 2001 From: Terry Wilson Date: Tue, 16 Aug 2022 14:38:50 -0700 Subject: [PATCH 12/18] Remove unnecessary use of Iterables.getOnlyElement() --- .../io/grpc/util/OutlierDetectionLoadBalancer.java | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java b/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java index f9e27017a88..5882599b4af 100644 --- a/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java +++ b/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java @@ -266,8 +266,7 @@ public void updateAddresses(List addressGroups) { } // If the map has an entry for the new address, we associate this subchannel with it. - SocketAddress address = Iterables.getOnlyElement( - Iterables.getOnlyElement(addressGroups).getAddresses()); + SocketAddress address = addressGroups.get(0).getAddresses().get(0); if (trackerMap.containsKey(address)) { trackerMap.get(address).addSubchannel(this); } @@ -276,17 +275,15 @@ public void updateAddresses(List addressGroups) { // outlier detection. Remove it from all trackers and reset the call counters of all the // associated trackers. // Remove the current subchannel from the old address it is associated with in the map. - if (trackerMap.containsKey(Iterables.getOnlyElement(getAddresses().getAddresses()))) { - AddressTracker tracker = trackerMap.get( - Iterables.getOnlyElement(getAddresses().getAddresses())); + if (trackerMap.containsKey(getAddresses().getAddresses().get(0))) { + AddressTracker tracker = trackerMap.get(getAddresses().getAddresses().get(0)); tracker.removeSubchannel(this); tracker.resetCallCounters(); } } else if (!hasSingleAddress(getAllAddresses()) && hasSingleAddress(addressGroups)) { // We go from, previously uneligble, multiple address mode to a single address. If the map // has an entry for the new address, we associate this subchannel with it. - SocketAddress address = Iterables.getOnlyElement( - Iterables.getOnlyElement(addressGroups).getAddresses()); + SocketAddress address = addressGroups.get(0).getAddresses().get(0); if (trackerMap.containsKey(address)) { AddressTracker tracker = trackerMap.get(address); tracker.addSubchannel(this); From 9fd326d803cdfd35c223372ca745d66968aa716a Mon Sep 17 00:00:00 2001 From: Terry Wilson Date: Tue, 16 Aug 2022 14:56:24 -0700 Subject: [PATCH 13/18] Store last subchannel state for every update. --- .../main/java/io/grpc/util/OutlierDetectionLoadBalancer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java b/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java index 5882599b4af..ad963255860 100644 --- a/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java +++ b/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java @@ -343,8 +343,8 @@ class OutlierDetectionSubchannelStateListener implements SubchannelStateListener @Override public void onSubchannelState(ConnectivityStateInfo newState) { + lastSubchannelState = newState; if (!ejected) { - lastSubchannelState = newState; delegate.onSubchannelState(newState); } } From b438bf7b3108b8ab3eff5318cde6b3f1f9d1c4e8 Mon Sep 17 00:00:00 2001 From: Terry Wilson Date: Tue, 16 Aug 2022 15:21:53 -0700 Subject: [PATCH 14/18] Support config updates. --- .../util/OutlierDetectionLoadBalancer.java | 17 +++++-- .../OutlierDetectionLoadBalancerTest.java | 45 +++++++++++++++++++ 2 files changed, 59 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java b/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java index ad963255860..5e6431e8328 100644 --- a/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java +++ b/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java @@ -26,7 +26,6 @@ import com.google.common.collect.ForwardingMap; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Iterables; import io.grpc.Attributes; import io.grpc.ClientStreamTracer; import io.grpc.ClientStreamTracer.StreamInfo; @@ -101,6 +100,8 @@ public void handleResolvedAddresses(ResolvedAddresses resolvedAddresses) { } trackerMap.keySet().retainAll(addresses); + trackerMap.updateTrackerConfigs(config); + // Add any new ones. trackerMap.putNewTrackers(config, addresses); @@ -421,7 +422,7 @@ public void streamClosed(Status status) { */ static class AddressTracker { - private final OutlierDetectionLoadBalancerConfig config; + private OutlierDetectionLoadBalancerConfig config; // Marked as volatile to assure that when the inactive counter is swapped in as the new active // one, all threads see the change and don't hold on to a reference to the now inactive counter. private volatile CallCounter activeCallCounter = new CallCounter(); @@ -434,6 +435,10 @@ static class AddressTracker { this.config = config; } + void setConfig(OutlierDetectionLoadBalancerConfig config) { + this.config = config; + } + /** * Adds a subchannel to the tracker, while assuring that the subchannel ejection status is * updated to match the tracker's if needed. @@ -583,7 +588,13 @@ protected Map delegate() { return trackerMap; } - /** Adds a new tracker for every address in the given EAGs. */ + void updateTrackerConfigs(OutlierDetectionLoadBalancerConfig config) { + for (AddressTracker tracker: trackerMap.values()) { + tracker.setConfig(config); + } + } + + /** Adds a new tracker for every given address. */ void putNewTrackers(OutlierDetectionLoadBalancerConfig config, Collection addresses) { for (SocketAddress address : addresses) { diff --git a/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java b/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java index 39b3208ec23..caf48b0522a 100644 --- a/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java +++ b/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java @@ -388,6 +388,51 @@ public void successRateOneOutlier() { assertEjectedSubchannels(ImmutableSet.of(servers.get(0).getAddresses().get(0))); } + /** + * The success rate algorithm ejects the outlier, but then the config changes so that similar + * behavior no longer gets ejected. + */ + @Test + public void successRateOneOutlier_configChange() { + OutlierDetectionLoadBalancerConfig config = new OutlierDetectionLoadBalancerConfig.Builder() + .setMaxEjectionPercent(50) + .setSuccessRateEjection( + new SuccessRateEjection.Builder() + .setMinimumHosts(3) + .setRequestVolume(10).build()) + .setChildPolicy(new PolicySelection(roundRobinLbProvider, null)).build(); + + loadBalancer.handleResolvedAddresses(buildResolvedAddress(config, servers)); + + generateLoad(ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); + + // Move forward in time to a point where the detection timer has fired. + fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + + // The one subchannel that was returning errors should be ejected. + assertEjectedSubchannels(ImmutableSet.of(servers.get(0).getAddresses().get(0))); + + // New config sets enforcement percentage to 0. + config = new OutlierDetectionLoadBalancerConfig.Builder() + .setMaxEjectionPercent(50) + .setSuccessRateEjection( + new SuccessRateEjection.Builder() + .setMinimumHosts(3) + .setRequestVolume(10) + .setEnforcementPercentage(0).build()) + .setChildPolicy(new PolicySelection(roundRobinLbProvider, null)).build(); + + loadBalancer.handleResolvedAddresses(buildResolvedAddress(config, servers)); + + generateLoad(ImmutableMap.of(subchannel2, Status.DEADLINE_EXCEEDED)); + + // Move forward in time to a point where the detection timer has fired. + fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + + // Since we brought enforcement percentage to 0, no additional ejection should have happened. + assertEjectedSubchannels(ImmutableSet.of(servers.get(0).getAddresses().get(0))); + } + /** * The success rate algorithm ejects the outlier but after some time it should get unejected * if it stops being an outlier.. From 605d8e5144fbd27ebf1b7f092c7853d5b762c7ab Mon Sep 17 00:00:00 2001 From: Terry Wilson Date: Tue, 16 Aug 2022 15:26:47 -0700 Subject: [PATCH 15/18] Move trackersWithVolume() to SuccessRateOutlierEjectionAlgorithm. --- .../util/OutlierDetectionLoadBalancer.java | 27 ++++++++++--------- 1 file changed, 15 insertions(+), 12 deletions(-) diff --git a/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java b/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java index 5e6431e8328..e343d0e366f 100644 --- a/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java +++ b/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java @@ -649,17 +649,6 @@ void maybeUnejectOutliers(Long detectionTimerStartNanos) { } } - /** Returns only the trackers that have the minimum configured volume to be considered. */ - List trackersWithVolume(OutlierDetectionLoadBalancerConfig config) { - List trackersWithVolume = new ArrayList<>(); - for (AddressTracker tracker : trackerMap.values()) { - if (tracker.inactiveVolume() >= config.successRateEjection.requestVolume) { - trackersWithVolume.add(tracker); - } - } - return trackersWithVolume; - } - /** * How many percent of the addresses would have their subchannels ejected if we proceeded * with the next ejection. @@ -720,7 +709,7 @@ static class SuccessRateOutlierEjectionAlgorithm implements OutlierEjectionAlgor public void ejectOutliers(AddressTrackerMap trackerMap, long ejectionTimeNanos) { // Only consider addresses that have the minimum request volume specified in the config. - List trackersWithVolume = trackerMap.trackersWithVolume(config); + List trackersWithVolume = trackersWithVolume(trackerMap, config); // If we don't have enough addresses with significant volume then there's nothing to do. if (trackersWithVolume.size() < config.successRateEjection.minimumHosts || trackersWithVolume.size() == 0) { @@ -753,7 +742,20 @@ public void ejectOutliers(AddressTrackerMap trackerMap, long ejectionTimeNanos) } } + /** Returns only the trackers that have the minimum configured volume to be considered. */ + private List trackersWithVolume(AddressTrackerMap trackerMap, + OutlierDetectionLoadBalancerConfig config) { + List trackersWithVolume = new ArrayList<>(); + for (AddressTracker tracker : trackerMap.values()) { + if (tracker.inactiveVolume() >= config.successRateEjection.requestVolume) { + trackersWithVolume.add(tracker); + } + } + return trackersWithVolume; + } + /** Calculates the mean of the given values. */ + @VisibleForTesting static double mean(Collection values) { double totalValue = 0; for (double value : values) { @@ -764,6 +766,7 @@ static double mean(Collection values) { } /** Calculates the standard deviation for the given values and their mean. */ + @VisibleForTesting static double standardDeviation(Collection values, double mean) { double squaredDifferenceSum = 0; for (double value : values) { From f8191b3f1ad2152196d88f48a579790955057932 Mon Sep 17 00:00:00 2001 From: Terry Wilson Date: Tue, 16 Aug 2022 15:52:28 -0700 Subject: [PATCH 16/18] Various review comment fixes. --- .../java/io/grpc/SynchronizationContext.java | 3 +- .../util/OutlierDetectionLoadBalancer.java | 46 +++++++++++-------- 2 files changed, 29 insertions(+), 20 deletions(-) diff --git a/api/src/main/java/io/grpc/SynchronizationContext.java b/api/src/main/java/io/grpc/SynchronizationContext.java index 5fb59f14fe9..fe4243ec227 100644 --- a/api/src/main/java/io/grpc/SynchronizationContext.java +++ b/api/src/main/java/io/grpc/SynchronizationContext.java @@ -187,7 +187,8 @@ public void run() { @Override public String toString() { - return task.toString() + "(scheduled in SynchronizationContext)"; + return task.toString() + "(scheduled in SynchronizationContext with delay of " + delay + + ")"; } }, initialDelay, delay, unit); return new ScheduledHandle(runnable, future); diff --git a/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java b/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java index e343d0e366f..857c23421cb 100644 --- a/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java +++ b/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java @@ -180,7 +180,7 @@ public void run() { } /** - * This child helper wraps the provided helper so that is can hand out wrapped {@link + * This child helper wraps the provided helper so that it can hand out wrapped {@link * OutlierDetectionSubchannel}s and manage the address info map. */ class ChildHelper extends ForwardingLoadBalancerHelper { @@ -247,7 +247,8 @@ public void start(SubchannelStateListener listener) { @Override public Attributes getAttributes() { if (addressTracker != null) { - return delegate.getAttributes().toBuilder().set(ADDRESS_TRACKER_ATTR_KEY, addressTracker).build(); + return delegate.getAttributes().toBuilder().set(ADDRESS_TRACKER_ATTR_KEY, addressTracker) + .build(); } else { return delegate.getAttributes(); } @@ -553,10 +554,11 @@ boolean subchannelsEjected() { public boolean maxEjectionTimeElapsed(long currentTimeNanos) { // The instant in time beyond which the address should no longer be ejected. Also making sure // we honor any maximum ejection time setting. + long maxEjectionDurationSecs = Math.max(SECONDS.toNanos(config.baseEjectionTimeSecs), + SECONDS.toNanos(config.maxEjectionTimeSecs)); long maxEjectionTimeNanos = ejectionTimeNanos + Math.min( SECONDS.toNanos(config.baseEjectionTimeSecs) * ejectionTimeMultiplier, - Math.max(SECONDS.toNanos(config.baseEjectionTimeSecs), - SECONDS.toNanos(config.maxEjectionTimeSecs))); + maxEjectionDurationSecs); return currentTimeNanos > maxEjectionTimeNanos; } @@ -654,6 +656,9 @@ void maybeUnejectOutliers(Long detectionTimerStartNanos) { * with the next ejection. */ double nextEjectionPercentage() { + if (trackerMap.isEmpty()) { + return 0; + } int totalAddresses = 0; int ejectedAddresses = 0; for (AddressTracker tracker : trackerMap.values()) { @@ -672,12 +677,10 @@ void maybeUnejectOutliers(Long detectionTimerStartNanos) { */ interface OutlierEjectionAlgorithm { - /** - * Is the given {@link EquivalentAddressGroup} an outlier based on the past call results stored - * in {@link AddressTracker}. - */ - void ejectOutliers(AddressTrackerMap trackerMap, long ejectionTimeMillis); + /** Eject any outlier addresses. */ + void ejectOutliers(AddressTrackerMap trackerMap, long ejectionTimeNanos); + /** Builds a list of algorithms that are enabled in the given config. */ @Nullable static List forConfig(OutlierDetectionLoadBalancerConfig config) { ImmutableList.Builder algoListBuilder = ImmutableList.builder(); @@ -724,15 +727,16 @@ public void ejectOutliers(AddressTrackerMap trackerMap, long ejectionTimeNanos) double mean = mean(successRates); double stdev = standardDeviation(successRates, mean); + double requiredSuccessRate = + mean - stdev * (config.successRateEjection.stdevFactor / 1000f); + for (AddressTracker tracker : trackersWithVolume) { - // If an ejection now would take us past the max configured ejection percentagem stop here. + // If an ejection now would take us past the max configured ejection percentage, stop here. if (trackerMap.nextEjectionPercentage() > config.maxEjectionPercent) { return; } // If success rate is below the threshold, eject the address. - double requiredSuccessRate = - mean - stdev * (config.successRateEjection.stdevFactor / 1000f); if (tracker.successRate() < requiredSuccessRate) { // Only eject some addresses based on the enforcement percentage. if (new Random().nextInt(100) < config.successRateEjection.enforcementPercentage) { @@ -788,7 +792,7 @@ static class FailurePercentageOutlierEjectionAlgorithm implements OutlierEjectio } @Override - public void ejectOutliers(AddressTrackerMap trackerMap, long ejectionTimeMillis) { + public void ejectOutliers(AddressTrackerMap trackerMap, long ejectionTimeNanos) { // If we don't have the minimum amount of addresses the config calls for, then return. if (trackerMap.size() < config.failurePercentageEjection.minimumHosts) { @@ -811,7 +815,7 @@ public void ejectOutliers(AddressTrackerMap trackerMap, long ejectionTimeMillis) if (tracker.failureRate() > maxFailureRate) { // ...but only enforce this based on the enforcement percentage. if (new Random().nextInt(100) < config.failurePercentageEjection.enforcementPercentage) { - tracker.ejectSubchannels(ejectionTimeMillis); + tracker.ejectSubchannels(ejectionTimeNanos); } } } @@ -896,7 +900,7 @@ public Builder setMaxEjectionPercent(Integer maxEjectionPercent) { return this; } - /** Set to enable success rate eejction. */ + /** Set to enable success rate ejection. */ public Builder setSuccessRateEjection( SuccessRateEjection successRateEjection) { this.successRateEjection = successRateEjection; @@ -962,6 +966,7 @@ public Builder setStdevFactor(Integer stdevFactor) { /** Only eject this percentage of outliers. */ public Builder setEnforcementPercentage(Integer enforcementPercentage) { checkArgument(enforcementPercentage != null); + checkArgument(enforcementPercentage >= 0 && enforcementPercentage <= 100); this.enforcementPercentage = enforcementPercentage; return this; } @@ -969,6 +974,7 @@ public Builder setEnforcementPercentage(Integer enforcementPercentage) { /** The minimum amount of hosts needed for success rate ejection. */ public Builder setMinimumHosts(Integer minimumHosts) { checkArgument(minimumHosts != null); + checkArgument(minimumHosts >= 0); this.minimumHosts = minimumHosts; return this; } @@ -976,6 +982,7 @@ public Builder setMinimumHosts(Integer minimumHosts) { /** The minimum address request volume to be considered for success rate ejection. */ public Builder setRequestVolume(Integer requestVolume) { checkArgument(requestVolume != null); + checkArgument(requestVolume >= 0); this.requestVolume = requestVolume; return this; } @@ -1013,6 +1020,7 @@ public static class Builder { /** The failure percentage that will result in an address being considered an outlier. */ public Builder setThreshold(Integer threshold) { checkArgument(threshold != null); + checkArgument(threshold >= 0 && threshold <= 100); this.threshold = threshold; return this; } @@ -1020,6 +1028,7 @@ public Builder setThreshold(Integer threshold) { /** Only eject this percentage of outliers. */ public Builder setEnforcementPercentage(Integer enforcementPercentage) { checkArgument(enforcementPercentage != null); + checkArgument(enforcementPercentage >= 0 && enforcementPercentage <= 100); this.enforcementPercentage = enforcementPercentage; return this; } @@ -1027,6 +1036,7 @@ public Builder setEnforcementPercentage(Integer enforcementPercentage) { /** The minimum amount of host for failure percentage ejection to be enabled. */ public Builder setMinimumHosts(Integer minimumHosts) { checkArgument(minimumHosts != null); + checkArgument(minimumHosts >= 0); this.minimumHosts = minimumHosts; return this; } @@ -1037,6 +1047,7 @@ public Builder setMinimumHosts(Integer minimumHosts) { */ public Builder setRequestVolume(Integer requestVolume) { checkArgument(requestVolume != null); + checkArgument(requestVolume >= 0); this.requestVolume = requestVolume; return this; } @@ -1049,11 +1060,8 @@ public FailurePercentageEjection build() { } } - /** - * Determine if outlier detection is at all enabled in this config. - */ + /** Determine if any outlier detection algorithms are enabled in the config. */ boolean outlierDetectionEnabled() { - // One of the two supported algorithms needs to be configured. return successRateEjection != null || failurePercentageEjection != null; } } From d152f4afe0520180660b39a36a2b400c8f832aa2 Mon Sep 17 00:00:00 2001 From: Terry Wilson Date: Tue, 16 Aug 2022 21:04:39 -0700 Subject: [PATCH 17/18] dd test for overlapping success rate and error percentage ejections. --- .../OutlierDetectionLoadBalancerTest.java | 47 ++++++++++++++++++- 1 file changed, 45 insertions(+), 2 deletions(-) diff --git a/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java b/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java index caf48b0522a..1a8d9f018b9 100644 --- a/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java +++ b/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java @@ -17,6 +17,7 @@ package io.grpc.util; import static com.google.common.truth.Truth.assertThat; +import static com.google.common.truth.Truth.assertWithMessage; import static io.grpc.ConnectivityState.READY; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doAnswer; @@ -708,6 +709,47 @@ public void failurePercentageOneOutlier_enforcementPercentage() { assertEjectedSubchannels(ImmutableSet.of()); } + /** Success rate detects two outliers and error percentage three. */ + @Test + public void successRateAndFailurePercentageThreeOutliers() { + OutlierDetectionLoadBalancerConfig config = new OutlierDetectionLoadBalancerConfig.Builder() + .setMaxEjectionPercent(100) + .setSuccessRateEjection( + new SuccessRateEjection.Builder() + .setMinimumHosts(3) + .setRequestVolume(10) + .setStdevFactor(1).build()) + .setFailurePercentageEjection( + new FailurePercentageEjection.Builder() + .setThreshold(0) + .setMinimumHosts(3) + .setRequestVolume(1) + .build()) + .setChildPolicy(new PolicySelection(roundRobinLbProvider, null)).build(); + + loadBalancer.handleResolvedAddresses(buildResolvedAddress(config, servers)); + + // Three subchannels with problems, but one only has a single call that failed. + // This is not enough for success rate to catch, but failure percentage is + // configured with a 0 tolerance threshold. + generateLoad( + ImmutableMap.of( + subchannel1, Status.DEADLINE_EXCEEDED, + subchannel2, Status.DEADLINE_EXCEEDED, + subchannel3, Status.DEADLINE_EXCEEDED), + ImmutableMap.of(subchannel3, 1)); + + // Move forward in time to a point where the detection timer has fired. + fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + + // Should see thee ejected, success rate cathes the first two, error percentage the + // same two plus the subchannel with the single failure. + assertEjectedSubchannels(ImmutableSet.of( + servers.get(0).getAddresses().get(0), + servers.get(1).getAddresses().get(0), + servers.get(2).getAddresses().get(0))); + } + /** * When the address a subchannel is associated with changes it should get tracked under the new * address and its ejection state should match what the address has. @@ -1024,8 +1066,9 @@ private void generateLoad(Map statusMap, // Asserts that the given addresses are ejected and the rest are not. void assertEjectedSubchannels(Set addresses) { for (Entry entry : loadBalancer.trackerMap.entrySet()) { - assertThat(entry.getValue().subchannelsEjected()).isEqualTo( - addresses.contains(entry.getKey())); + assertWithMessage("not ejected: " + entry.getKey()) + .that(entry.getValue().subchannelsEjected()) + .isEqualTo(addresses.contains(entry.getKey())); } } } From e47bc290070c42e5b4b15bfea40be8583eb0d267 Mon Sep 17 00:00:00 2001 From: Terry Wilson Date: Wed, 17 Aug 2022 11:30:34 -0700 Subject: [PATCH 18/18] Add OutlierDetectionLoadBalancerProvider. --- .../io/grpc/LoadBalancerRegistryTest.java | 8 +- .../util/OutlierDetectionLoadBalancer.java | 69 ++++---- .../OutlierDetectionLoadBalancerProvider.java | 158 ++++++++++++++++++ .../services/io.grpc.LoadBalancerProvider | 1 + ...lierDetectionLoadBalancerProviderTest.java | 141 ++++++++++++++++ .../OutlierDetectionLoadBalancerTest.java | 65 +++---- 6 files changed, 375 insertions(+), 67 deletions(-) create mode 100644 core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancerProvider.java create mode 100644 core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerProviderTest.java diff --git a/api/src/test/java/io/grpc/LoadBalancerRegistryTest.java b/api/src/test/java/io/grpc/LoadBalancerRegistryTest.java index e8588e5e8d8..3debc871121 100644 --- a/api/src/test/java/io/grpc/LoadBalancerRegistryTest.java +++ b/api/src/test/java/io/grpc/LoadBalancerRegistryTest.java @@ -41,7 +41,7 @@ public void getClassesViaHardcoded_classesPresent() throws Exception { @Test public void stockProviders() { LoadBalancerRegistry defaultRegistry = LoadBalancerRegistry.getDefaultRegistry(); - assertThat(defaultRegistry.providers()).hasSize(3); + assertThat(defaultRegistry.providers()).hasSize(4); LoadBalancerProvider pickFirst = defaultRegistry.getProvider("pick_first"); assertThat(pickFirst).isInstanceOf(PickFirstLoadBalancerProvider.class); @@ -52,6 +52,12 @@ public void stockProviders() { "io.grpc.util.SecretRoundRobinLoadBalancerProvider$Provider"); assertThat(roundRobin.getPriority()).isEqualTo(5); + LoadBalancerProvider outlierDetection = defaultRegistry.getProvider( + "outlier_detection_experimental"); + assertThat(outlierDetection.getClass().getName()).isEqualTo( + "io.grpc.util.OutlierDetectionLoadBalancerProvider"); + assertThat(roundRobin.getPriority()).isEqualTo(5); + LoadBalancerProvider grpclb = defaultRegistry.getProvider("grpclb"); assertThat(grpclb).isInstanceOf(GrpclbLoadBalancerProvider.class); assertThat(grpclb.getPriority()).isEqualTo(5); diff --git a/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java b/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java index 857c23421cb..3fb32fdb078 100644 --- a/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java +++ b/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancer.java @@ -20,7 +20,6 @@ import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; import static java.util.concurrent.TimeUnit.NANOSECONDS; -import static java.util.concurrent.TimeUnit.SECONDS; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ForwardingMap; @@ -49,7 +48,6 @@ import java.util.Random; import java.util.Set; import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; import javax.annotation.Nullable; @@ -114,13 +112,12 @@ public void handleResolvedAddresses(ResolvedAddresses resolvedAddresses) { if (detectionTimerStartNanos == null) { // On the first go we use the configured interval. - initialDelayNanos = TimeUnit.SECONDS.toNanos(config.intervalSecs); + initialDelayNanos = config.intervalNanos; } else { // If a timer has started earlier we cancel it and use the difference between the start // time and now as the interval. initialDelayNanos = Math.max(0L, - TimeUnit.SECONDS.toNanos(config.intervalSecs) - (timeProvider.currentTimeNanos() - - detectionTimerStartNanos)); + config.intervalNanos - (timeProvider.currentTimeNanos() - detectionTimerStartNanos)); } // If a timer has been previously created we need to cancel it and reset all the call counters @@ -131,7 +128,7 @@ public void handleResolvedAddresses(ResolvedAddresses resolvedAddresses) { } detectionTimerHandle = syncContext.scheduleWithFixedDelay(new DetectionTimer(config), - initialDelayNanos, SECONDS.toNanos(config.intervalSecs), NANOSECONDS, timeService); + initialDelayNanos, config.intervalNanos, NANOSECONDS, timeService); } else if (detectionTimerHandle != null) { // Outlier detection is not configured, but we have a lingering timer. Let's cancel it and // uneject any addresses we may have ejected. @@ -554,11 +551,12 @@ boolean subchannelsEjected() { public boolean maxEjectionTimeElapsed(long currentTimeNanos) { // The instant in time beyond which the address should no longer be ejected. Also making sure // we honor any maximum ejection time setting. - long maxEjectionDurationSecs = Math.max(SECONDS.toNanos(config.baseEjectionTimeSecs), - SECONDS.toNanos(config.maxEjectionTimeSecs)); - long maxEjectionTimeNanos = ejectionTimeNanos + Math.min( - SECONDS.toNanos(config.baseEjectionTimeSecs) * ejectionTimeMultiplier, - maxEjectionDurationSecs); + long maxEjectionDurationSecs + = Math.max(config.baseEjectionTimeNanos, config.maxEjectionTimeNanos); + long maxEjectionTimeNanos = + ejectionTimeNanos + Math.min( + config.baseEjectionTimeNanos * ejectionTimeMultiplier, + maxEjectionDurationSecs); return currentTimeNanos > maxEjectionTimeNanos; } @@ -839,23 +837,24 @@ private static boolean hasSingleAddress(List addressGrou */ public static final class OutlierDetectionLoadBalancerConfig { - final Long intervalSecs; - final Long baseEjectionTimeSecs; - final Long maxEjectionTimeSecs; + final Long intervalNanos; + final Long baseEjectionTimeNanos; + final Long maxEjectionTimeNanos; final Integer maxEjectionPercent; final SuccessRateEjection successRateEjection; final FailurePercentageEjection failurePercentageEjection; final PolicySelection childPolicy; - private OutlierDetectionLoadBalancerConfig(Long intervalSecs, - Long baseEjectionTimeSecs, - Long maxEjectionTimeSecs, Integer maxEjectionPercent, + private OutlierDetectionLoadBalancerConfig(Long intervalNanos, + Long baseEjectionTimeNanos, + Long maxEjectionTimeNanos, + Integer maxEjectionPercent, SuccessRateEjection successRateEjection, FailurePercentageEjection failurePercentageEjection, PolicySelection childPolicy) { - this.intervalSecs = intervalSecs; - this.baseEjectionTimeSecs = baseEjectionTimeSecs; - this.maxEjectionTimeSecs = maxEjectionTimeSecs; + this.intervalNanos = intervalNanos; + this.baseEjectionTimeNanos = baseEjectionTimeNanos; + this.maxEjectionTimeNanos = maxEjectionTimeNanos; this.maxEjectionPercent = maxEjectionPercent; this.successRateEjection = successRateEjection; this.failurePercentageEjection = failurePercentageEjection; @@ -864,32 +863,32 @@ private OutlierDetectionLoadBalancerConfig(Long intervalSecs, /** Builds a new {@link OutlierDetectionLoadBalancerConfig}. */ public static class Builder { - Long intervalSecs = 10L; - Long baseEjectionTimeSecs = 30L; - Long maxEjectionTimeSecs = 30L; + Long intervalNanos = 10_000_000_000L; // 10s + Long baseEjectionTimeNanos = 30_000_000_000L; // 30s + Long maxEjectionTimeNanos = 30_000_000_000L; // 30s Integer maxEjectionPercent = 10; SuccessRateEjection successRateEjection; FailurePercentageEjection failurePercentageEjection; PolicySelection childPolicy; /** The interval between outlier detection sweeps. */ - public Builder setIntervalSecs(Long intervalSecs) { - checkArgument(intervalSecs != null); - this.intervalSecs = intervalSecs; + public Builder setIntervalNanos(Long intervalNanos) { + checkArgument(intervalNanos != null); + this.intervalNanos = intervalNanos; return this; } /** The base time an address is ejected for. */ - public Builder setBaseEjectionTimeSecs(Long baseEjectionTimeSecs) { - checkArgument(baseEjectionTimeSecs != null); - this.baseEjectionTimeSecs = baseEjectionTimeSecs; + public Builder setBaseEjectionTimeNanos(Long baseEjectionTimeNanos) { + checkArgument(baseEjectionTimeNanos != null); + this.baseEjectionTimeNanos = baseEjectionTimeNanos; return this; } /** The longest time an address can be ejected. */ - public Builder setMaxEjectionTimeSecs(Long maxEjectionTimeSecs) { - checkArgument(maxEjectionTimeSecs != null); - this.maxEjectionTimeSecs = maxEjectionTimeSecs; + public Builder setMaxEjectionTimeNanos(Long maxEjectionTimeNanos) { + checkArgument(maxEjectionTimeNanos != null); + this.maxEjectionTimeNanos = maxEjectionTimeNanos; return this; } @@ -924,9 +923,9 @@ public Builder setChildPolicy(PolicySelection childPolicy) { /** Builds a new instance of {@link OutlierDetectionLoadBalancerConfig}. */ public OutlierDetectionLoadBalancerConfig build() { checkState(childPolicy != null); - return new OutlierDetectionLoadBalancerConfig(intervalSecs, baseEjectionTimeSecs, - maxEjectionTimeSecs, maxEjectionPercent, successRateEjection, failurePercentageEjection, - childPolicy); + return new OutlierDetectionLoadBalancerConfig(intervalNanos, baseEjectionTimeNanos, + maxEjectionTimeNanos, maxEjectionPercent, successRateEjection, + failurePercentageEjection, childPolicy); } } diff --git a/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancerProvider.java b/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancerProvider.java new file mode 100644 index 00000000000..a92f49bd1d2 --- /dev/null +++ b/core/src/main/java/io/grpc/util/OutlierDetectionLoadBalancerProvider.java @@ -0,0 +1,158 @@ +/* + * Copyright 2022 The gRPC Authors + * + * Licensed 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 io.grpc.util; + +import io.grpc.LoadBalancer; +import io.grpc.LoadBalancer.Helper; +import io.grpc.LoadBalancerProvider; +import io.grpc.LoadBalancerRegistry; +import io.grpc.NameResolver.ConfigOrError; +import io.grpc.Status; +import io.grpc.internal.JsonUtil; +import io.grpc.internal.ServiceConfigUtil; +import io.grpc.internal.ServiceConfigUtil.LbConfig; +import io.grpc.internal.ServiceConfigUtil.PolicySelection; +import io.grpc.internal.TimeProvider; +import io.grpc.util.OutlierDetectionLoadBalancer.OutlierDetectionLoadBalancerConfig; +import io.grpc.util.OutlierDetectionLoadBalancer.OutlierDetectionLoadBalancerConfig.FailurePercentageEjection; +import io.grpc.util.OutlierDetectionLoadBalancer.OutlierDetectionLoadBalancerConfig.SuccessRateEjection; +import java.util.List; +import java.util.Map; + +public final class OutlierDetectionLoadBalancerProvider extends LoadBalancerProvider { + + @Override + public LoadBalancer newLoadBalancer(Helper helper) { + return new OutlierDetectionLoadBalancer(helper, TimeProvider.SYSTEM_TIME_PROVIDER); + } + + @Override + public boolean isAvailable() { + return true; + } + + @Override + public int getPriority() { + return 5; + } + + @Override + public String getPolicyName() { + return "outlier_detection_experimental"; + } + + @Override + public ConfigOrError parseLoadBalancingPolicyConfig(Map rawConfig) { + // Common configuration. + Long intervalNanos = JsonUtil.getStringAsDuration(rawConfig, "interval"); + Long baseEjectionTimeNanos = JsonUtil.getStringAsDuration(rawConfig, "baseEjectionTime"); + Long maxEjectionTimeNanos = JsonUtil.getStringAsDuration(rawConfig, "maxEjectionTime"); + Integer maxEjectionPercentage = JsonUtil.getNumberAsInteger(rawConfig, + "maxEjectionPercentage"); + + OutlierDetectionLoadBalancerConfig.Builder configBuilder + = new OutlierDetectionLoadBalancerConfig.Builder(); + if (intervalNanos != null) { + configBuilder.setIntervalNanos(intervalNanos); + } + if (baseEjectionTimeNanos != null) { + configBuilder.setBaseEjectionTimeNanos(baseEjectionTimeNanos); + } + if (maxEjectionTimeNanos != null) { + configBuilder.setMaxEjectionTimeNanos(maxEjectionTimeNanos); + } + if (maxEjectionPercentage != null) { + configBuilder.setMaxEjectionPercent(maxEjectionPercentage); + } + + // Success rate ejection specific configuration. + Map rawSuccessRateEjection = JsonUtil.getObject(rawConfig, "successRateEjection"); + if (rawSuccessRateEjection != null) { + SuccessRateEjection.Builder successRateEjectionBuilder = new SuccessRateEjection.Builder(); + + Integer stdevFactor = JsonUtil.getNumberAsInteger(rawSuccessRateEjection, "stdevFactor"); + Integer enforcementPercentage = JsonUtil.getNumberAsInteger(rawSuccessRateEjection, + "enforcementPercentage"); + Integer minimumHosts = JsonUtil.getNumberAsInteger(rawSuccessRateEjection, "minimumHosts"); + Integer requestVolume = JsonUtil.getNumberAsInteger(rawSuccessRateEjection, "requestVolume"); + + if (stdevFactor != null) { + successRateEjectionBuilder.setStdevFactor(stdevFactor); + } + if (enforcementPercentage != null) { + successRateEjectionBuilder.setEnforcementPercentage(enforcementPercentage); + } + if (minimumHosts != null) { + successRateEjectionBuilder.setMinimumHosts(minimumHosts); + } + if (requestVolume != null) { + successRateEjectionBuilder.setRequestVolume(requestVolume); + } + + configBuilder.setSuccessRateEjection(successRateEjectionBuilder.build()); + } + + // Failure percentage ejection specific configuration. + Map rawFailurePercentageEjection = JsonUtil.getObject(rawConfig, + "failurePercentageEjection"); + if (rawFailurePercentageEjection != null) { + FailurePercentageEjection.Builder failurePercentageEjectionBuilder + = new FailurePercentageEjection.Builder(); + + Integer threshold = JsonUtil.getNumberAsInteger(rawFailurePercentageEjection, "threshold"); + Integer enforcementPercentage = JsonUtil.getNumberAsInteger(rawFailurePercentageEjection, + "enforcementPercentage"); + Integer minimumHosts = JsonUtil.getNumberAsInteger(rawFailurePercentageEjection, + "minimumHosts"); + Integer requestVolume = JsonUtil.getNumberAsInteger(rawFailurePercentageEjection, + "requestVolume"); + + if (threshold != null) { + failurePercentageEjectionBuilder.setThreshold(threshold); + } + if (enforcementPercentage != null) { + failurePercentageEjectionBuilder.setEnforcementPercentage(enforcementPercentage); + } + if (minimumHosts != null) { + failurePercentageEjectionBuilder.setMinimumHosts(minimumHosts); + } + if (requestVolume != null) { + failurePercentageEjectionBuilder.setRequestVolume(requestVolume); + } + + configBuilder.setFailurePercentageEjection(failurePercentageEjectionBuilder.build()); + } + + // Child load balancer configuration. + List childConfigCandidates = ServiceConfigUtil.unwrapLoadBalancingConfigList( + JsonUtil.getListOfObjects(rawConfig, "childPolicy")); + if (childConfigCandidates == null || childConfigCandidates.isEmpty()) { + return ConfigOrError.fromError(Status.INTERNAL.withDescription( + "No child policy in outlier_detection_experimental LB policy: " + + rawConfig)); + } + ConfigOrError selectedConfig = + ServiceConfigUtil.selectLbPolicyFromList(childConfigCandidates, + LoadBalancerRegistry.getDefaultRegistry()); + if (selectedConfig.getError() != null) { + return selectedConfig; + } + configBuilder.setChildPolicy((PolicySelection) selectedConfig.getConfig()); + + return ConfigOrError.fromConfig(configBuilder.build()); + } +} diff --git a/core/src/main/resources/META-INF/services/io.grpc.LoadBalancerProvider b/core/src/main/resources/META-INF/services/io.grpc.LoadBalancerProvider index cb200d5f044..d68a57c4eb3 100644 --- a/core/src/main/resources/META-INF/services/io.grpc.LoadBalancerProvider +++ b/core/src/main/resources/META-INF/services/io.grpc.LoadBalancerProvider @@ -1,2 +1,3 @@ io.grpc.internal.PickFirstLoadBalancerProvider io.grpc.util.SecretRoundRobinLoadBalancerProvider$Provider +io.grpc.util.OutlierDetectionLoadBalancerProvider diff --git a/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerProviderTest.java b/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerProviderTest.java new file mode 100644 index 00000000000..5a27e6f176f --- /dev/null +++ b/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerProviderTest.java @@ -0,0 +1,141 @@ +/* + * Copyright 2022 The gRPC Authors + * + * Licensed 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 io.grpc.util; + +import static com.google.common.truth.Truth.assertThat; +import static org.junit.Assert.fail; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import io.grpc.InternalServiceProviders; +import io.grpc.LoadBalancer.Helper; +import io.grpc.LoadBalancerProvider; +import io.grpc.NameResolver.ConfigOrError; +import io.grpc.SynchronizationContext; +import io.grpc.internal.JsonParser; +import io.grpc.util.OutlierDetectionLoadBalancer.OutlierDetectionLoadBalancerConfig; +import java.io.IOException; +import java.lang.Thread.UncaughtExceptionHandler; +import java.util.Map; +import java.util.concurrent.ScheduledExecutorService; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Unit tests for {@link OutlierDetectionLoadBalancerProvider}. + */ +@RunWith(JUnit4.class) +public class OutlierDetectionLoadBalancerProviderTest { + + private final SynchronizationContext syncContext = new SynchronizationContext( + new UncaughtExceptionHandler() { + @Override + public void uncaughtException(Thread t, Throwable e) { + throw new AssertionError(e); + } + }); + private final OutlierDetectionLoadBalancerProvider provider + = new OutlierDetectionLoadBalancerProvider(); + + @Test + public void provided() { + for (LoadBalancerProvider current : InternalServiceProviders.getCandidatesViaServiceLoader( + LoadBalancerProvider.class, getClass().getClassLoader())) { + if (current instanceof OutlierDetectionLoadBalancerProvider) { + return; + } + } + fail("OutlierDetectionLoadBalancerProvider not registered"); + } + + @Test + public void providesLoadBalancer() { + Helper helper = mock(Helper.class); + when(helper.getSynchronizationContext()).thenReturn(syncContext); + when(helper.getScheduledExecutorService()).thenReturn(mock(ScheduledExecutorService.class)); + assertThat(provider.newLoadBalancer(helper)) + .isInstanceOf(OutlierDetectionLoadBalancer.class); + } + + @Test + public void parseLoadBalancingConfig_defaults() throws IOException { + String lbConfig = + "{ \"successRateEjection\" : {}, " + + "\"failurePercentageEjection\" : {}, " + + "\"childPolicy\" : [{\"round_robin\" : {}}]}"; + ConfigOrError configOrError = + provider.parseLoadBalancingPolicyConfig(parseJsonObject(lbConfig)); + assertThat(configOrError.getConfig()).isNotNull(); + OutlierDetectionLoadBalancerConfig config + = (OutlierDetectionLoadBalancerConfig) configOrError.getConfig(); + assertThat(config.successRateEjection).isNotNull(); + assertThat(config.failurePercentageEjection).isNotNull(); + assertThat(config.childPolicy.getProvider().getPolicyName()).isEqualTo("round_robin"); + } + + @Test + public void parseLoadBalancingConfig_valuesSet() throws IOException { + String lbConfig = + "{\"interval\" : \"100s\"," + + " \"baseEjectionTime\" : \"100s\"," + + " \"maxEjectionTime\" : \"100s\"," + + " \"maxEjectionPercentage\" : 100," + + " \"successRateEjection\" : {" + + " \"stdevFactor\" : 100," + + " \"enforcementPercentage\" : 100," + + " \"minimumHosts\" : 100," + + " \"requestVolume\" : 100" + + " }," + + " \"failurePercentageEjection\" : {" + + " \"threshold\" : 100," + + " \"enforcementPercentage\" : 100," + + " \"minimumHosts\" : 100," + + " \"requestVolume\" : 100" + + " }," + + "\"childPolicy\" : [{\"round_robin\" : {}}]}"; + ConfigOrError configOrError = + provider.parseLoadBalancingPolicyConfig(parseJsonObject(lbConfig)); + assertThat(configOrError.getConfig()).isNotNull(); + OutlierDetectionLoadBalancerConfig config + = (OutlierDetectionLoadBalancerConfig) configOrError.getConfig(); + + assertThat(config.intervalNanos).isEqualTo(100_000_000_000L); + assertThat(config.baseEjectionTimeNanos).isEqualTo(100_000_000_000L); + assertThat(config.maxEjectionTimeNanos).isEqualTo(100_000_000_000L); + assertThat(config.maxEjectionPercent).isEqualTo(100); + + assertThat(config.successRateEjection).isNotNull(); + assertThat(config.successRateEjection.stdevFactor).isEqualTo(100); + assertThat(config.successRateEjection.enforcementPercentage).isEqualTo(100); + assertThat(config.successRateEjection.minimumHosts).isEqualTo(100); + assertThat(config.successRateEjection.requestVolume).isEqualTo(100); + + assertThat(config.failurePercentageEjection).isNotNull(); + assertThat(config.failurePercentageEjection.threshold).isEqualTo(100); + assertThat(config.failurePercentageEjection.enforcementPercentage).isEqualTo(100); + assertThat(config.failurePercentageEjection.minimumHosts).isEqualTo(100); + assertThat(config.failurePercentageEjection.requestVolume).isEqualTo(100); + + assertThat(config.childPolicy.getProvider().getPolicyName()).isEqualTo("round_robin"); + } + + @SuppressWarnings("unchecked") + private static Map parseJsonObject(String json) throws IOException { + return (Map) JsonParser.parse(json); + } +} diff --git a/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java b/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java index 1a8d9f018b9..5b73edb4c2c 100644 --- a/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java +++ b/core/src/test/java/io/grpc/util/OutlierDetectionLoadBalancerTest.java @@ -240,7 +240,7 @@ public void handleResolvedAddresses() { // The task is scheduled to run after a delay set in the config. ScheduledTask task = fakeClock.getPendingTasks().iterator().next(); - assertThat(task.getDelay(TimeUnit.SECONDS)).isEqualTo(config.intervalSecs); + assertThat(task.getDelay(TimeUnit.NANOSECONDS)).isEqualTo(config.intervalNanos); } /** @@ -286,7 +286,7 @@ public void handleResolvedAddresses_intervalUpdate() { // Config update has doubled the interval config = new OutlierDetectionLoadBalancerConfig.Builder() - .setIntervalSecs(config.intervalSecs * 2) + .setIntervalNanos(config.intervalNanos * 2) .setSuccessRateEjection(new SuccessRateEjection.Builder().build()) .setChildPolicy(new PolicySelection(mockChildLbProvider, null)).build(); @@ -296,16 +296,15 @@ public void handleResolvedAddresses_intervalUpdate() { // If the timer has not run yet the task is just rescheduled to run after the new delay. assertThat(fakeClock.getPendingTasks()).hasSize(1); ScheduledTask task = fakeClock.getPendingTasks().iterator().next(); - assertThat(task.getDelay(TimeUnit.SECONDS)).isEqualTo(config.intervalSecs); - assertThat(task.dueTimeNanos).isEqualTo(TimeUnit.SECONDS.toNanos(config.intervalSecs)); + assertThat(task.getDelay(TimeUnit.NANOSECONDS)).isEqualTo(config.intervalNanos); + assertThat(task.dueTimeNanos).isEqualTo(config.intervalNanos); // The new interval time has passed. The next task due time should have been pushed back another // interval. - fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + forwardTime(config); assertThat(fakeClock.getPendingTasks()).hasSize(1); task = fakeClock.getPendingTasks().iterator().next(); - assertThat(task.dueTimeNanos).isEqualTo( - TimeUnit.SECONDS.toNanos(config.intervalSecs + config.intervalSecs + 1)); + assertThat(task.dueTimeNanos).isEqualTo(config.intervalNanos + config.intervalNanos + 1); // Some time passes and a second update comes down, but now the timer has had a chance to run, // the new delay to timer start should consider when the timer last ran and if the interval is @@ -314,8 +313,7 @@ public void handleResolvedAddresses_intervalUpdate() { task = fakeClock.getPendingTasks().iterator().next(); loadBalancer.handleResolvedAddresses( buildResolvedAddress(config, new EquivalentAddressGroup(mockSocketAddress))); - assertThat(task.dueTimeNanos).isEqualTo( - TimeUnit.SECONDS.toNanos(config.intervalSecs + config.intervalSecs + 1)); + assertThat(task.dueTimeNanos).isEqualTo(config.intervalNanos + config.intervalNanos + 1); } /** @@ -359,7 +357,7 @@ public void successRateNoOutliers() { generateLoad(ImmutableMap.of()); // Move forward in time to a point where the detection timer has fired. - fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + forwardTime(config); // No outliers, no ejections. assertEjectedSubchannels(ImmutableSet.of()); @@ -383,7 +381,7 @@ public void successRateOneOutlier() { generateLoad(ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); // Move forward in time to a point where the detection timer has fired. - fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + forwardTime(config); // The one subchannel that was returning errors should be ejected. assertEjectedSubchannels(ImmutableSet.of(servers.get(0).getAddresses().get(0))); @@ -408,7 +406,7 @@ public void successRateOneOutlier_configChange() { generateLoad(ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); // Move forward in time to a point where the detection timer has fired. - fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + forwardTime(config); // The one subchannel that was returning errors should be ejected. assertEjectedSubchannels(ImmutableSet.of(servers.get(0).getAddresses().get(0))); @@ -428,7 +426,7 @@ public void successRateOneOutlier_configChange() { generateLoad(ImmutableMap.of(subchannel2, Status.DEADLINE_EXCEEDED)); // Move forward in time to a point where the detection timer has fired. - fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + forwardTime(config); // Since we brought enforcement percentage to 0, no additional ejection should have happened. assertEjectedSubchannels(ImmutableSet.of(servers.get(0).getAddresses().get(0))); @@ -453,7 +451,7 @@ public void successRateOneOutlier_unejected() { generateLoad(ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); // Move forward in time to a point where the detection timer has fired. - fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + fakeClock.forwardTime(config.intervalNanos + 1, TimeUnit.NANOSECONDS); // The one subchannel that was returning errors should be ejected. assertEjectedSubchannels(ImmutableSet.of(servers.get(0).getAddresses().get(0))); @@ -462,7 +460,7 @@ public void successRateOneOutlier_unejected() { generateLoad(ImmutableMap.of()); // Move forward in time to a point where the detection timer has fired. - fakeClock.forwardTime(config.maxEjectionTimeSecs + 1, TimeUnit.SECONDS); + fakeClock.forwardTime(config.maxEjectionTimeNanos + 1, TimeUnit.NANOSECONDS); // No subchannels should remain ejected. assertEjectedSubchannels(ImmutableSet.of()); @@ -489,7 +487,7 @@ public void successRateOneOutlier_notEnoughVolume() { ImmutableMap.of(subchannel1, 19)); // Move forward in time to a point where the detection timer has fired. - fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + forwardTime(config); // The address should not have been ejected. assertEjectedSubchannels(ImmutableSet.of()); @@ -513,7 +511,7 @@ public void successRateOneOutlier_notEnoughAddressesWithVolume() { generateLoad(ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); // Move forward in time to a point where the detection timer has fired. - fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + forwardTime(config); // No subchannels should have been ejected. assertEjectedSubchannels(ImmutableSet.of()); @@ -539,7 +537,7 @@ public void successRateOneOutlier_enforcementPercentage() { generateLoad(ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); // Move forward in time to a point where the detection timer has fired. - fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + forwardTime(config); // There is one outlier, but because enforcementPercentage is 0, nothing should be ejected. assertEjectedSubchannels(ImmutableSet.of()); @@ -566,7 +564,7 @@ public void successRateTwoOutliers() { subchannel2, Status.DEADLINE_EXCEEDED)); // Move forward in time to a point where the detection timer has fired. - fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + forwardTime(config); // The one subchannel that was returning errors should be ejected. assertEjectedSubchannels(ImmutableSet.of(servers.get(0).getAddresses().get(0), @@ -594,7 +592,7 @@ public void successRateTwoOutliers_maxEjectionPercentage() { subchannel2, Status.DEADLINE_EXCEEDED)); // Move forward in time to a point where the detection timer has fired. - fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + forwardTime(config); int totalEjected = 0; for (EquivalentAddressGroup addressGroup: servers) { @@ -629,7 +627,7 @@ public void failurePercentageNoOutliers() { generateLoad(ImmutableMap.of()); // Move forward in time to a point where the detection timer has fired. - fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + forwardTime(config); // No outliers, no ejections. assertEjectedSubchannels(ImmutableSet.of()); @@ -653,7 +651,7 @@ public void failurePercentageOneOutlier() { generateLoad(ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); // Move forward in time to a point where the detection timer has fired. - fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + forwardTime(config); // The one subchannel that was returning errors should be ejected. assertEjectedSubchannels(ImmutableSet.of(servers.get(0).getAddresses().get(0))); @@ -677,7 +675,7 @@ public void failurePercentageOneOutlier_notEnoughVolume() { generateLoad(ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); // Move forward in time to a point where the detection timer has fired. - fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + forwardTime(config); // We should see no ejections. assertEjectedSubchannels(ImmutableSet.of()); @@ -703,7 +701,7 @@ public void failurePercentageOneOutlier_enforcementPercentage() { generateLoad(ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); // Move forward in time to a point where the detection timer has fired. - fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + forwardTime(config); // There is one outlier, but because enforcementPercentage is 0, nothing should be ejected. assertEjectedSubchannels(ImmutableSet.of()); @@ -740,7 +738,7 @@ public void successRateAndFailurePercentageThreeOutliers() { ImmutableMap.of(subchannel3, 1)); // Move forward in time to a point where the detection timer has fired. - fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + forwardTime(config); // Should see thee ejected, success rate cathes the first two, error percentage the // same two plus the subchannel with the single failure. @@ -769,7 +767,7 @@ public void subchannelUpdateAddress_singleReplaced() { generateLoad(ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); // Move forward in time to a point where the detection timer has fired. - fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + forwardTime(config); EquivalentAddressGroup oldAddressGroup = servers.get(0); AddressTracker oldAddressTracker = loadBalancer.trackerMap.get( @@ -816,7 +814,7 @@ public void subchannelUpdateAddress_singleReplacedWithMultiple() { generateLoad(ImmutableMap.of()); // Move forward in time to a point where the detection timer has fired. - fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + forwardTime(config); EquivalentAddressGroup oldAddressGroup = servers.get(0); AddressTracker oldAddressTracker = loadBalancer.trackerMap.get( @@ -863,7 +861,7 @@ public void subchannelUpdateAddress_multipleReplacedWithSingle() { generateLoad(ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); // Move forward in time to a point where the detection timer has fired. - fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + forwardTime(config); EquivalentAddressGroup oldAddressGroup = servers.get(0); AddressTracker oldAddressTracker = loadBalancer.trackerMap.get( @@ -925,7 +923,7 @@ public void successRateAndFailurePercentage_noOutliers() { generateLoad(ImmutableMap.of()); // Move forward in time to a point where the detection timer has fired. - fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + forwardTime(config); // No outliers, no ejections. assertEjectedSubchannels(ImmutableSet.of()); @@ -952,7 +950,7 @@ public void successRateAndFailurePercentage_successRateOutlier() { generateLoad(ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); // Move forward in time to a point where the detection timer has fired. - fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + forwardTime(config); // The one subchannel that was returning errors should be ejected. assertEjectedSubchannels(ImmutableSet.of(servers.get(0).getAddresses().get(0))); @@ -979,7 +977,7 @@ public void successRateAndFailurePercentage_errorPercentageOutlier() { generateLoad(ImmutableMap.of(subchannel1, Status.DEADLINE_EXCEEDED)); // Move forward in time to a point where the detection timer has fired. - fakeClock.forwardTime(config.intervalSecs + 1, TimeUnit.SECONDS); + forwardTime(config); // The one subchannel that was returning errors should be ejected. assertEjectedSubchannels(ImmutableSet.of(servers.get(0).getAddresses().get(0))); @@ -1063,6 +1061,11 @@ private void generateLoad(Map statusMap, } } + // Forwards time past the moment when the timer will fire. + private void forwardTime(OutlierDetectionLoadBalancerConfig config) { + fakeClock.forwardTime(config.intervalNanos + 1, TimeUnit.NANOSECONDS); + } + // Asserts that the given addresses are ejected and the rest are not. void assertEjectedSubchannels(Set addresses) { for (Entry entry : loadBalancer.trackerMap.entrySet()) {