You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by wf...@apache.org on 2017/10/31 17:20:42 UTC

aurora git commit: Refactor staticallyBannedOffers into a LRU cache

Repository: aurora
Updated Branches:
  refs/heads/master 8af269f52 -> d106b4ecc


Refactor staticallyBannedOffers into a LRU cache

Using the new `hold_offers_forever` option, it is possible for the
`staticallyBannedOffers` to grow very large in size as we never release
offers.
1. The current behavior of `staticallyBannedOffers` is (kinda) preserved.
   Entries will no longer be removed when the offer is used, but they will be
   removed within `maxOfferHoldTime`. This means cluster operators will not
   have to think about the new `offer_static_ban_cache_max_size` if they aren't
   affected by the memory leak now.
2. Cluster operators that use Aurora as a single framework and hold offers
   indefinitely can cap the size of the cache to avoid the memory leak.
3. Using an LRU cache greatly benefits quickly recurring crons and job updates.

Reviewed at https://reviews.apache.org/r/63199/


Project: http://git-wip-us.apache.org/repos/asf/aurora/repo
Commit: http://git-wip-us.apache.org/repos/asf/aurora/commit/d106b4ec
Tree: http://git-wip-us.apache.org/repos/asf/aurora/tree/d106b4ec
Diff: http://git-wip-us.apache.org/repos/asf/aurora/diff/d106b4ec

Branch: refs/heads/master
Commit: d106b4ecc9537b8e844c4edc2210b9fe1853ccc4
Parents: 8af269f
Author: Jordan Ly <jo...@gmail.com>
Authored: Tue Oct 31 10:20:27 2017 -0700
Committer: Bill Farner <wf...@apache.org>
Committed: Tue Oct 31 10:20:27 2017 -0700

----------------------------------------------------------------------
 .../aurora/benchmark/SchedulingBenchmarks.java  |  7 ++-
 .../config/validators/NotNegativeNumber.java    | 26 ++++++++++
 .../aurora/scheduler/offers/OfferManager.java   | 47 ++++++++++-------
 .../aurora/scheduler/offers/OfferSettings.java  | 54 ++++++++++++++++----
 .../aurora/scheduler/offers/OffersModule.java   | 33 +++++++++++-
 .../scheduler/config/CommandLineTest.java       |  2 +
 .../scheduler/offers/OfferManagerImplTest.java  | 31 +++++++----
 7 files changed, 159 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/d106b4ec/src/jmh/java/org/apache/aurora/benchmark/SchedulingBenchmarks.java
----------------------------------------------------------------------
diff --git a/src/jmh/java/org/apache/aurora/benchmark/SchedulingBenchmarks.java b/src/jmh/java/org/apache/aurora/benchmark/SchedulingBenchmarks.java
index e0ec793..456e780 100644
--- a/src/jmh/java/org/apache/aurora/benchmark/SchedulingBenchmarks.java
+++ b/src/jmh/java/org/apache/aurora/benchmark/SchedulingBenchmarks.java
@@ -40,6 +40,7 @@ import org.apache.aurora.common.quantity.Time;
 import org.apache.aurora.common.stats.StatsProvider;
 import org.apache.aurora.common.util.Clock;
 import org.apache.aurora.common.util.testing.FakeClock;
+import org.apache.aurora.common.util.testing.FakeTicker;
 import org.apache.aurora.gen.ServerInfo;
 import org.apache.aurora.scheduler.HostOffer;
 import org.apache.aurora.scheduler.TaskIdGenerator;
@@ -153,7 +154,11 @@ public class SchedulingBenchmarks {
               bind(OfferManager.class).to(OfferManager.OfferManagerImpl.class);
               bind(OfferManager.OfferManagerImpl.class).in(Singleton.class);
               bind(OfferSettings.class).toInstance(
-                  new OfferSettings(NO_DELAY, ImmutableList.of(OfferOrder.RANDOM)));
+                  new OfferSettings(NO_DELAY,
+                      ImmutableList.of(OfferOrder.RANDOM),
+                      Amount.of(Long.MAX_VALUE, Time.SECONDS),
+                      Long.MAX_VALUE,
+                      new FakeTicker()));
               bind(BiCache.BiCacheSettings.class).toInstance(
                   new BiCache.BiCacheSettings(DELAY_FOREVER, ""));
               bind(TaskScheduler.class).to(TaskScheduler.TaskSchedulerImpl.class);

http://git-wip-us.apache.org/repos/asf/aurora/blob/d106b4ec/src/main/java/org/apache/aurora/scheduler/config/validators/NotNegativeNumber.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/config/validators/NotNegativeNumber.java b/src/main/java/org/apache/aurora/scheduler/config/validators/NotNegativeNumber.java
new file mode 100644
index 0000000..fd0320e
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/config/validators/NotNegativeNumber.java
@@ -0,0 +1,26 @@
+/**
+ * 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 org.apache.aurora.scheduler.config.validators;
+
+import com.beust.jcommander.IValueValidator;
+import com.beust.jcommander.ParameterException;
+
+public class NotNegativeNumber implements IValueValidator<Number> {
+  @Override
+  public void validate(String name, Number value) throws ParameterException {
+    if (value.longValue() < 0) {
+      throw new ParameterException(String.format("%s must not be negative", name));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/d106b4ec/src/main/java/org/apache/aurora/scheduler/offers/OfferManager.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/offers/OfferManager.java b/src/main/java/org/apache/aurora/scheduler/offers/OfferManager.java
index 7011a4c..96b0f46 100644
--- a/src/main/java/org/apache/aurora/scheduler/offers/OfferManager.java
+++ b/src/main/java/org/apache/aurora/scheduler/offers/OfferManager.java
@@ -22,17 +22,15 @@ import javax.inject.Inject;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Optional;
+import com.google.common.cache.Cache;
 import com.google.common.collect.FluentIterable;
-import com.google.common.collect.HashMultimap;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Maps;
-import com.google.common.collect.Multimap;
-import com.google.common.collect.Multimaps;
-import com.google.common.collect.Ordering;
 import com.google.common.collect.Sets;
 import com.google.common.eventbus.Subscribe;
 
+import org.apache.aurora.common.collections.Pair;
 import org.apache.aurora.common.inject.TimedInterceptor.Timed;
 import org.apache.aurora.common.quantity.Time;
 import org.apache.aurora.common.stats.StatsProvider;
@@ -153,6 +151,8 @@ public interface OfferManager extends EventSubscriber {
     @VisibleForTesting
     static final String STATICALLY_BANNED_OFFERS = "statically_banned_offers_size";
     @VisibleForTesting
+    static final String STATICALLY_BANNED_OFFERS_HIT_RATE = "statically_banned_offers_hit_rate";
+    @VisibleForTesting
     static final String OFFER_CANCEL_FAILURES = "offer_cancel_failures";
     @VisibleForTesting
     static final String GLOBALLY_BANNED_OFFERS = "globally_banned_offers_size";
@@ -175,7 +175,7 @@ public interface OfferManager extends EventSubscriber {
 
       this.driver = requireNonNull(driver);
       this.offerSettings = requireNonNull(offerSettings);
-      this.hostOffers = new HostOffers(statsProvider, offerSettings.getOfferOrder());
+      this.hostOffers = new HostOffers(statsProvider, offerSettings);
       this.offerRaces = statsProvider.makeCounter(OFFER_ACCEPT_RACES);
       this.offerCancelFailures = statsProvider.makeCounter(OFFER_CANCEL_FAILURES);
       this.offerDecline = requireNonNull(offerDecline);
@@ -209,7 +209,7 @@ public interface OfferManager extends EventSubscriber {
 
     private Protos.Filters getOfferFilter() {
       return Protos.Filters.newBuilder()
-          .setRefuseSeconds(offerSettings.getOfferFilterDuration().as(Time.SECONDS))
+          .setRefuseSeconds(offerSettings.getFilterDuration().as(Time.SECONDS))
           .build();
     }
 
@@ -279,6 +279,15 @@ public interface OfferManager extends EventSubscriber {
     }
 
     /**
+     * Used for testing to ensure that the underlying cache's `size` method returns an accurate
+     * value by not including evicted entries.
+     */
+    @VisibleForTesting
+    public void cleanupStaticBans() {
+      hostOffers.staticallyBannedOffers.cleanUp();
+    }
+
+    /**
      * A container for the data structures used by this class, to make it easier to reason about
      * the different indices used and their consistency.
      */
@@ -289,22 +298,25 @@ public interface OfferManager extends EventSubscriber {
       private final Map<AgentID, HostOffer> offersBySlave = Maps.newHashMap();
       private final Map<String, HostOffer> offersByHost = Maps.newHashMap();
 
-      // TODO(maxim): Expose via a debug endpoint. AURORA-1136.
       // Keep track of offer->groupKey mappings that will never be matched to avoid redundant
       // scheduling attempts. See VetoGroup for more details on static ban.
-      private final Multimap<OfferID, TaskGroupKey> staticallyBannedOffers =
-          Multimaps.synchronizedMultimap(HashMultimap.create());
+      private final Cache<Pair<OfferID, TaskGroupKey>, Boolean> staticallyBannedOffers;
 
       // Keep track of globally banned offers that will never be matched to anything.
       private final Set<OfferID> globallyBannedOffers = Sets.newConcurrentHashSet();
 
-      HostOffers(StatsProvider statsProvider, Ordering<HostOffer> offerOrder) {
-        offers = new ConcurrentSkipListSet<>(offerOrder);
+      HostOffers(StatsProvider statsProvider, OfferSettings offerSettings) {
+        offers = new ConcurrentSkipListSet<>(offerSettings.getOrdering());
+        staticallyBannedOffers = offerSettings
+            .getStaticBanCacheBuilder()
+            .build();
         // Potential gotcha - since this is a ConcurrentSkipListSet, size() is more expensive.
         // Could track this separately if it turns out to pose problems.
         statsProvider.exportSize(OUTSTANDING_OFFERS, offers);
-        statsProvider.makeGauge(STATICALLY_BANNED_OFFERS, () -> staticallyBannedOffers.size());
-        statsProvider.makeGauge(GLOBALLY_BANNED_OFFERS, () -> globallyBannedOffers.size());
+        statsProvider.makeGauge(STATICALLY_BANNED_OFFERS, staticallyBannedOffers::size);
+        statsProvider.makeGauge(STATICALLY_BANNED_OFFERS_HIT_RATE,
+            () -> staticallyBannedOffers.stats().hitRate());
+        statsProvider.makeGauge(GLOBALLY_BANNED_OFFERS, globallyBannedOffers::size);
       }
 
       synchronized Optional<HostOffer> get(AgentID slaveId) {
@@ -350,7 +362,6 @@ public interface OfferManager extends EventSubscriber {
           offersBySlave.remove(removed.getOffer().getAgentId());
           offersByHost.remove(removed.getOffer().getHostname());
         }
-        staticallyBannedOffers.removeAll(id);
         globallyBannedOffers.remove(id);
         return removed != null;
       }
@@ -387,8 +398,8 @@ public interface OfferManager extends EventSubscriber {
        * @return The offers a given task group can use.
        */
       synchronized Iterable<HostOffer> getWeaklyConsistentOffers(TaskGroupKey groupKey) {
-        return Iterables.unmodifiableIterable(FluentIterable.from(offers).filter(
-            e -> !staticallyBannedOffers.containsEntry(e.getOffer().getId(), groupKey)
+        return Iterables.unmodifiableIterable(FluentIterable.from(offers).filter(e ->
+            staticallyBannedOffers.getIfPresent(Pair.of(e.getOffer().getId(), groupKey)) == null
                 && !globallyBannedOffers.contains(e.getOffer().getId())));
       }
 
@@ -398,7 +409,7 @@ public interface OfferManager extends EventSubscriber {
 
       synchronized void addStaticGroupBan(OfferID offerId, TaskGroupKey groupKey) {
         if (offersById.containsKey(offerId)) {
-          staticallyBannedOffers.put(offerId, groupKey);
+          staticallyBannedOffers.put(Pair.of(offerId, groupKey), true);
         }
       }
 
@@ -407,7 +418,7 @@ public interface OfferManager extends EventSubscriber {
         offersById.clear();
         offersBySlave.clear();
         offersByHost.clear();
-        staticallyBannedOffers.clear();
+        staticallyBannedOffers.invalidateAll();
         globallyBannedOffers.clear();
       }
     }

http://git-wip-us.apache.org/repos/asf/aurora/blob/d106b4ec/src/main/java/org/apache/aurora/scheduler/offers/OfferSettings.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/offers/OfferSettings.java b/src/main/java/org/apache/aurora/scheduler/offers/OfferSettings.java
index e060f20..57fc1a1 100644
--- a/src/main/java/org/apache/aurora/scheduler/offers/OfferSettings.java
+++ b/src/main/java/org/apache/aurora/scheduler/offers/OfferSettings.java
@@ -15,6 +15,9 @@ package org.apache.aurora.scheduler.offers;
 
 import java.util.List;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Ticker;
+import com.google.common.cache.CacheBuilder;
 import com.google.common.collect.Ordering;
 
 import org.apache.aurora.common.quantity.Amount;
@@ -28,29 +31,58 @@ import static java.util.Objects.requireNonNull;
  */
 public class OfferSettings {
 
-  private final Amount<Long, Time> offerFilterDuration;
-  private final Ordering<HostOffer> offerOrder;
+  private final Amount<Long, Time> filterDuration;
+  private final Ordering<HostOffer> ordering;
+  private final CacheBuilder<Object, Object> staticBanCacheBuilder;
 
-  public OfferSettings(Amount<Long, Time> offerFilterDuration, List<OfferOrder> offerOrder) {
-    this(offerFilterDuration, OfferOrderBuilder.create(offerOrder));
+  @VisibleForTesting
+  public OfferSettings(Amount<Long, Time> filterDuration,
+                       List<OfferOrder> ordering,
+                       Amount<Long, Time> maxHoldTime,
+                       long staticBanCacheMaxSize,
+                       Ticker staticBanCacheTicker) {
+
+    this(filterDuration,
+        OfferOrderBuilder.create(ordering),
+        maxHoldTime,
+        staticBanCacheMaxSize,
+        staticBanCacheTicker);
   }
 
-  OfferSettings(Amount<Long, Time> offerFilterDuration, Ordering<HostOffer> offerOrder) {
-    this.offerFilterDuration = requireNonNull(offerFilterDuration);
-    this.offerOrder = requireNonNull(offerOrder);
+  OfferSettings(Amount<Long, Time> filterDuration,
+                Ordering<HostOffer> ordering,
+                Amount<Long, Time> maxHoldTime,
+                long staticBanCacheMaxSize,
+                Ticker staticBanTicker) {
+
+    this.filterDuration = requireNonNull(filterDuration);
+    this.ordering = requireNonNull(ordering);
+    this.staticBanCacheBuilder = CacheBuilder.newBuilder()
+        .expireAfterWrite(maxHoldTime.as(Time.SECONDS), Time.SECONDS.getTimeUnit())
+        .maximumSize(staticBanCacheMaxSize)
+        .ticker(staticBanTicker)
+        .recordStats();
   }
 
   /**
    * Duration after which we want Mesos to re-offer unused or declined resources.
    */
-  public Amount<Long, Time> getOfferFilterDuration() {
-    return offerFilterDuration;
+  public Amount<Long, Time> getFilterDuration() {
+    return filterDuration;
   }
 
   /**
    * The ordering to use when fetching offers from OfferManager.
    */
-  public Ordering<HostOffer> getOfferOrder() {
-    return offerOrder;
+  public Ordering<HostOffer> getOrdering() {
+    return ordering;
+  }
+
+  /**
+   * The builder for the static ban cache. Cache settings (e.g. max size, entry expiration) should
+   * already be added to the builder by this point.
+   */
+  public CacheBuilder<Object, Object> getStaticBanCacheBuilder() {
+    return staticBanCacheBuilder;
   }
 }

http://git-wip-us.apache.org/repos/asf/aurora/blob/d106b4ec/src/main/java/org/apache/aurora/scheduler/offers/OffersModule.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/offers/OffersModule.java b/src/main/java/org/apache/aurora/scheduler/offers/OffersModule.java
index e6b2c55..4a6ea8d 100644
--- a/src/main/java/org/apache/aurora/scheduler/offers/OffersModule.java
+++ b/src/main/java/org/apache/aurora/scheduler/offers/OffersModule.java
@@ -23,6 +23,7 @@ import javax.inject.Singleton;
 import com.beust.jcommander.Parameter;
 import com.beust.jcommander.Parameters;
 import com.google.common.base.Supplier;
+import com.google.common.base.Ticker;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Ordering;
 import com.google.inject.AbstractModule;
@@ -39,6 +40,7 @@ import org.apache.aurora.scheduler.app.MoreModules;
 import org.apache.aurora.scheduler.config.CliOptions;
 import org.apache.aurora.scheduler.config.types.TimeAmount;
 import org.apache.aurora.scheduler.config.validators.NotNegativeAmount;
+import org.apache.aurora.scheduler.config.validators.NotNegativeNumber;
 import org.apache.aurora.scheduler.events.PubsubEventModule;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -97,6 +99,14 @@ public class OffersModule extends AbstractModule {
         description = "Custom Guice module to provide an offer ordering.")
     @SuppressWarnings("rawtypes")
     public List<Class> offerOrderModules = ImmutableList.of(OfferOrderModule.class);
+
+    @Parameter(names = "-offer_static_ban_cache_max_size",
+        validateValueWith = NotNegativeNumber.class,
+        description =
+            "The number of offers to hold in the static ban cache. If no value is specified, "
+                + "the cache will grow indefinitely. However, entries will expire within "
+                + "'min_offer_hold_time' + 'offer_hold_jitter_window' of being written.")
+    public long offerStaticBanCacheMaxSize = Long.MAX_VALUE;
   }
 
   public static class OfferOrderModule extends AbstractModule {
@@ -175,6 +185,27 @@ public class OffersModule extends AbstractModule {
   @Provides
   @Singleton
   OfferSettings provideOfferSettings(Ordering<HostOffer> offerOrdering) {
-    return new OfferSettings(cliOptions.offer.offerFilterDuration, offerOrdering);
+    // We have a dual eviction strategy for the static ban cache in OfferManager that is based on
+    // both maximum size of the cache and the length an offer is valid. We do this in order to
+    // satisfy requirements in both single- and multi-framework environments. If offers are held for
+    // a finite duration, then we can expire cache entries after offerMaxHoldTime since that is the
+    // longest it will be valid for. Additionally, cluster operators will most likely not have to
+    // worry about cache size in this case as this behavior mimics current behavior. If offers are
+    // held indefinitely, then we never expire cache entries but the cluster operator can specify a
+    // maximum size to avoid a memory leak.
+    long maxOfferHoldTime;
+    if (cliOptions.offer.holdOffersForever) {
+      maxOfferHoldTime = Long.MAX_VALUE;
+    } else {
+      maxOfferHoldTime = cliOptions.offer.minOfferHoldTime.as(Time.SECONDS)
+          + cliOptions.offer.offerHoldJitterWindow.as(Time.SECONDS);
+    }
+
+    return new OfferSettings(
+        cliOptions.offer.offerFilterDuration,
+        offerOrdering,
+        Amount.of(maxOfferHoldTime, Time.SECONDS),
+        cliOptions.offer.offerStaticBanCacheMaxSize,
+        Ticker.systemTicker());
   }
 }

http://git-wip-us.apache.org/repos/asf/aurora/blob/d106b4ec/src/test/java/org/apache/aurora/scheduler/config/CommandLineTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/config/CommandLineTest.java b/src/test/java/org/apache/aurora/scheduler/config/CommandLineTest.java
index 244422c..c2d875b 100644
--- a/src/test/java/org/apache/aurora/scheduler/config/CommandLineTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/config/CommandLineTest.java
@@ -119,6 +119,7 @@ public class CommandLineTest {
     expected.offer.holdOffersForever = true;
     expected.offer.minOfferHoldTime = TEST_TIME;
     expected.offer.offerHoldJitterWindow = TEST_TIME;
+    expected.offer.offerStaticBanCacheMaxSize = 42L;
     expected.offer.offerFilterDuration = TEST_TIME;
     expected.offer.unavailabilityThreshold = TEST_TIME;
     expected.offer.offerOrder = ImmutableList.of(OfferOrder.CPU, OfferOrder.DISK);
@@ -264,6 +265,7 @@ public class CommandLineTest {
         "-hold_offers_forever=true",
         "-min_offer_hold_time=42days",
         "-offer_hold_jitter_window=42days",
+        "-offer_static_ban_cache_max_size=42",
         "-offer_filter_duration=42days",
         "-unavailability_threshold=42days",
         "-offer_order=CPU,DISK",

http://git-wip-us.apache.org/repos/asf/aurora/blob/d106b4ec/src/test/java/org/apache/aurora/scheduler/offers/OfferManagerImplTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/offers/OfferManagerImplTest.java b/src/test/java/org/apache/aurora/scheduler/offers/OfferManagerImplTest.java
index 3d38a59..3b6b539 100644
--- a/src/test/java/org/apache/aurora/scheduler/offers/OfferManagerImplTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/offers/OfferManagerImplTest.java
@@ -23,6 +23,7 @@ import com.google.common.collect.Iterables;
 import org.apache.aurora.common.quantity.Amount;
 import org.apache.aurora.common.quantity.Time;
 import org.apache.aurora.common.testing.easymock.EasyMockTest;
+import org.apache.aurora.common.util.testing.FakeTicker;
 import org.apache.aurora.gen.HostAttributes;
 import org.apache.aurora.gen.MaintenanceMode;
 import org.apache.aurora.scheduler.HostOffer;
@@ -107,6 +108,7 @@ public class OfferManagerImplTest extends EasyMockTest {
   private static final Filters OFFER_FILTER = Filters.newBuilder()
       .setRefuseSeconds(OFFER_FILTER_SECONDS)
       .build();
+  private static final FakeTicker FAKE_TICKER = new FakeTicker();
 
   private Driver driver;
   private OfferManagerImpl offerManager;
@@ -117,7 +119,11 @@ public class OfferManagerImplTest extends EasyMockTest {
     driver = createMock(Driver.class);
     OfferSettings offerSettings = new OfferSettings(
         Amount.of(OFFER_FILTER_SECONDS, Time.SECONDS),
-        ImmutableList.of(OfferOrder.RANDOM));
+        ImmutableList.of(OfferOrder.RANDOM),
+        RETURN_DELAY,
+        Long.MAX_VALUE,
+        FAKE_TICKER
+    );
     statsProvider = new FakeStatsProvider();
     offerManager = new OfferManagerImpl(driver, offerSettings, statsProvider, new Noop());
   }
@@ -234,7 +240,7 @@ public class OfferManagerImplTest extends EasyMockTest {
   }
 
   @Test
-  public void testStaticBanIsClearedOnOfferReturn() {
+  public void testStaticBanExpiresAfterMaxHoldTime() throws InterruptedException {
     control.replay();
 
     offerManager.addOffer(OFFER_A);
@@ -243,9 +249,9 @@ public class OfferManagerImplTest extends EasyMockTest {
     assertTrue(Iterables.isEmpty(offerManager.getOffers(GROUP_KEY)));
     assertEquals(1, statsProvider.getLongValue(STATICALLY_BANNED_OFFERS));
 
-    // Make sure the static ban is cleared when the offers are returned.
-    offerManager.cancelOffer(OFFER_A_ID);
-    offerManager.addOffer(OFFER_A);
+    // Make sure the static ban expires after maximum amount of time an offer is held.
+    FAKE_TICKER.advance(RETURN_DELAY);
+    offerManager.cleanupStaticBans();
     assertEquals(OFFER_A, Iterables.getOnlyElement(offerManager.getOffers(GROUP_KEY)));
     assertEquals(0, statsProvider.getLongValue(STATICALLY_BANNED_OFFERS));
   }
@@ -351,7 +357,12 @@ public class OfferManagerImplTest extends EasyMockTest {
 
   private OfferManager createOrderedManager(List<OfferOrder> order) {
     OfferSettings settings =
-        new OfferSettings(Amount.of(OFFER_FILTER_SECONDS, Time.SECONDS), order);
+        new OfferSettings(
+            Amount.of(OFFER_FILTER_SECONDS, Time.SECONDS),
+            order,
+            RETURN_DELAY,
+            Long.MAX_VALUE,
+            FAKE_TICKER);
     return new OfferManagerImpl(driver, settings, statsProvider, new Noop());
   }
 
@@ -523,7 +534,10 @@ public class OfferManagerImplTest extends EasyMockTest {
   public void testDelayedOfferReturn() {
     OfferSettings settings = new OfferSettings(
         Amount.of(OFFER_FILTER_SECONDS, Time.SECONDS),
-        ImmutableList.of(OfferOrder.RANDOM));
+        ImmutableList.of(OfferOrder.RANDOM),
+        RETURN_DELAY,
+        Long.MAX_VALUE,
+        FAKE_TICKER);
     DelayExecutor executorMock = createMock(DelayExecutor.class);
     FakeScheduledExecutor clock = FakeScheduledExecutor.fromDelayExecutor(executorMock);
     addTearDown(clock::assertEmpty);
@@ -538,13 +552,10 @@ public class OfferManagerImplTest extends EasyMockTest {
     control.replay();
 
     offerManager.addOffer(OFFER_A);
-    offerManager.banOfferForTaskGroup(OFFER_A_ID, GROUP_KEY);
     assertEquals(1, statsProvider.getLongValue(OUTSTANDING_OFFERS));
-    assertEquals(1, statsProvider.getLongValue(STATICALLY_BANNED_OFFERS));
 
     clock.advance(RETURN_DELAY);
     assertEquals(0, statsProvider.getLongValue(OUTSTANDING_OFFERS));
-    assertEquals(0, statsProvider.getLongValue(STATICALLY_BANNED_OFFERS));
   }
 
   @Test