Merge branch 'stable-3.4' into stable-3.5

* stable-3.4:
  Fix Flogger formatting of exception in AutoAdjustCachesCommand
  Extract ChronicleMapStoreMetrics to its own class file
  Ignore the BytesStore raw type
  Emit metric to track number of put failures
  Move tryPut to ChronicleMapStore
  Move store related metrics to ChronicleMapStore
  Introduce ChronicleMapStore
  Remove redundant store parameter from ChronicleMapCacheImpl
  Do not throw when failing to store into chronicle-map
  Expose max auto resizes metric
  Allow to specify number of entries in auto-adjust-caches
  Fix link to auto-adjust caches in tuning docs
  Add acceptance test for REST auto-tuning for single cache
  Allow to restrict the caches to auto-tune
  Increase the memory used in acceptance tests
  build.md: Use correct path for external_plugin_deps.bzl

Replace diff cache IT testing with gerrit_file_diff and
git_file_diff caches.

Change-Id: I955a75c72dd61c4b36a2a96b32aaa73eacec3762
diff --git a/BUILD b/BUILD
index 45afb1a..dc34ede 100644
--- a/BUILD
+++ b/BUILD
@@ -52,7 +52,7 @@
     srcs = glob(["src/test/java/**/*IT.java"]),
     group = "server_cache",
     labels = ["server"],
-    vm_args = ["-Xmx2G"],
+    vm_args = ["-Xmx4g"],
     deps = [
         ":cache-chroniclemap__plugin",
         ":chroniclemap-test-lib",
diff --git a/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/AutoAdjustCaches.java b/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/AutoAdjustCaches.java
index 4298a81..948aaee 100644
--- a/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/AutoAdjustCaches.java
+++ b/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/AutoAdjustCaches.java
@@ -27,7 +27,11 @@
 import java.io.File;
 import java.io.IOException;
 import java.nio.file.Path;
+import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
 import java.util.concurrent.ConcurrentMap;
 import java.util.stream.Collectors;
 import org.apache.commons.io.FilenameUtils;
@@ -41,12 +45,17 @@
   protected static final String CONFIG_HEADER = "__CONFIG__";
   protected static final String TUNED_INFIX = "_tuned_";
 
+  protected static final Integer MAX_ENTRIES_MULTIPLIER = 2;
+  protected static final Integer PERCENTAGE_SIZE_INCREASE_THRESHOLD = 50;
+
   private final DynamicMap<Cache<?, ?>> cacheMap;
   private final ChronicleMapCacheConfig.Factory configFactory;
   private final Path cacheDir;
   private final AdministerCachePermission adminCachePermission;
 
   private boolean dryRun;
+  private Optional<Long> optionalMaxEntries = Optional.empty();
+  private Set<String> cacheNames = new HashSet<>();
 
   @Inject
   AutoAdjustCaches(
@@ -69,6 +78,18 @@
     this.dryRun = dryRun;
   }
 
+  public Optional<Long> getOptionalMaxEntries() {
+    return optionalMaxEntries;
+  }
+
+  public void setOptionalMaxEntries(Optional<Long> maxEntries) {
+    this.optionalMaxEntries = maxEntries;
+  }
+
+  public void addCacheNames(List<String> cacheNames) {
+    this.cacheNames.addAll(cacheNames);
+  }
+
   protected Config run(@Nullable ProgressMonitor optionalProgressMonitor)
       throws AuthException, PermissionBackendException, IOException {
     ProgressMonitor progressMonitor =
@@ -98,21 +119,24 @@
         long averageValueSize = avgSizes.getValue();
 
         ChronicleMapCacheConfig currCacheConfig = currCache.getConfig();
+        long newMaxEntries = newMaxEntries(currCache);
 
         if (currCacheConfig.getAverageKeySize() == averageKeySize
-            && currCacheConfig.getAverageValueSize() == averageValueSize) {
+            && currCacheConfig.getAverageValueSize() == averageValueSize
+            && currCacheConfig.getMaxEntries() == newMaxEntries) {
           continue;
         }
 
         ChronicleMapCacheConfig newChronicleMapCacheConfig =
-            makeChronicleMapConfig(currCache.getConfig(), averageKeySize, averageValueSize);
+            makeChronicleMapConfig(
+                currCache.getConfig(), newMaxEntries, averageKeySize, averageValueSize);
 
         updateOutputConfig(
             outputChronicleMapConfig,
             cacheName,
             averageKeySize,
             averageValueSize,
-            currCache.getConfig().getMaxEntries(),
+            newMaxEntries,
             currCache.getConfig().getMaxBloatFactor());
 
         if (!dryRun) {
@@ -176,6 +200,7 @@
 
   private ChronicleMapCacheConfig makeChronicleMapConfig(
       ChronicleMapCacheConfig currentChronicleMapConfig,
+      long newMaxEntries,
       long averageKeySize,
       long averageValueSize) {
 
@@ -184,12 +209,30 @@
         resolveNewFile(currentChronicleMapConfig.getPersistedFile().getName()),
         currentChronicleMapConfig.getExpireAfterWrite(),
         currentChronicleMapConfig.getRefreshAfterWrite(),
-        currentChronicleMapConfig.getMaxEntries(),
+        newMaxEntries,
         averageKeySize,
         averageValueSize,
         currentChronicleMapConfig.getMaxBloatFactor());
   }
 
+  private long newMaxEntries(ChronicleMapCacheImpl<Object, Object> currentCache) {
+    return getOptionalMaxEntries()
+        .orElseGet(
+            () -> {
+              double percentageUsedAutoResizes = currentCache.percentageUsedAutoResizes();
+              long currMaxEntries = currentCache.getConfig().getMaxEntries();
+
+              long newMaxEntries = currMaxEntries;
+              if (percentageUsedAutoResizes > PERCENTAGE_SIZE_INCREASE_THRESHOLD) {
+                newMaxEntries = currMaxEntries * MAX_ENTRIES_MULTIPLIER;
+              }
+              logger.atInfo().log(
+                  "Cache '%s' (maxEntries: %s) used %s%% of available space. new maxEntries will be: %s",
+                  currentCache.name(), currMaxEntries, percentageUsedAutoResizes, newMaxEntries);
+              return newMaxEntries;
+            });
+  }
+
   private File resolveNewFile(String currentFileName) {
     String newFileName =
         String.format(
@@ -228,6 +271,7 @@
             pair ->
                 pair.getValue() instanceof ChronicleMapCacheImpl
                     && ((ChronicleMapCacheImpl) pair.getValue()).diskStats().size() > 0)
+        .filter(pair -> cacheNames.isEmpty() ? true : cacheNames.contains(pair.getKey()))
         .collect(
             Collectors.toMap(
                 ImmutablePair::getKey, p -> (ChronicleMapCacheImpl<Object, Object>) p.getValue()));
diff --git a/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/AutoAdjustCachesCommand.java b/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/AutoAdjustCachesCommand.java
index 7aa240e..580bd98 100644
--- a/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/AutoAdjustCachesCommand.java
+++ b/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/AutoAdjustCachesCommand.java
@@ -19,8 +19,11 @@
 import com.google.gerrit.sshd.SshCommand;
 import com.google.inject.Inject;
 import java.io.IOException;
+import java.util.Arrays;
+import java.util.Optional;
 import org.eclipse.jgit.lib.Config;
 import org.eclipse.jgit.lib.TextProgressMonitor;
+import org.kohsuke.args4j.Argument;
 import org.kohsuke.args4j.Option;
 
 public class AutoAdjustCachesCommand extends SshCommand {
@@ -38,6 +41,24 @@
     autoAdjustCachesEngine.setDryRun(dryRun);
   }
 
+  @Option(
+      name = "--max-entries",
+      aliases = {"-m"},
+      usage = "The number of entries that the new tuned cache is going to hold.")
+  public void setMaxEntries(long maxEntries) {
+    autoAdjustCachesEngine.setOptionalMaxEntries(Optional.of(maxEntries));
+  }
+
+  @Argument(
+      index = 0,
+      required = false,
+      multiValued = true,
+      metaVar = "CACHE_NAME",
+      usage = "name of cache to be adjusted")
+  public void setCacheName(String cacheName) {
+    autoAdjustCachesEngine.addCacheNames(Arrays.asList(cacheName));
+  }
+
   @Inject
   AutoAdjustCachesCommand(AutoAdjustCaches autoAdjustCachesEngine) {
     this.autoAdjustCachesEngine = autoAdjustCachesEngine;
@@ -65,7 +86,7 @@
       stderr.println(e.getLocalizedMessage());
       throw e;
     } catch (IOException e) {
-      logger.atSevere().log("Could not create new cache", e);
+      logger.atSevere().withCause(e).log("Could not create new cache");
       stderr.println(String.format("Could not create new cache : %s", e.getLocalizedMessage()));
     }
   }
diff --git a/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/AutoAdjustCachesServlet.java b/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/AutoAdjustCachesServlet.java
index b217b0e..49f7a97 100644
--- a/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/AutoAdjustCachesServlet.java
+++ b/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/AutoAdjustCachesServlet.java
@@ -24,6 +24,7 @@
 import com.google.inject.Provider;
 import com.google.inject.Singleton;
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.Optional;
 import javax.servlet.http.HttpServlet;
 import javax.servlet.http.HttpServletRequest;
@@ -56,6 +57,16 @@
             .or(() -> Optional.ofNullable(req.getParameter("d")))
             .isPresent());
 
+    autoAdjustCachesEngine.setOptionalMaxEntries(
+        Optional.ofNullable(req.getParameter("max-entries"))
+            .or(() -> Optional.ofNullable(req.getParameter("m")))
+            .map(Long::parseLong));
+
+    String[] cacheNames = req.getParameterValues("CACHE_NAME");
+    if (cacheNames != null) {
+      autoAdjustCachesEngine.addCacheNames(Arrays.asList(cacheNames));
+    }
+
     try {
       Config outputChronicleMapConfig = autoAdjustCachesEngine.run(null);
 
diff --git a/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/ChronicleMapCacheFactory.java b/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/ChronicleMapCacheFactory.java
index e1eb5ab..1161699 100644
--- a/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/ChronicleMapCacheFactory.java
+++ b/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/ChronicleMapCacheFactory.java
@@ -42,7 +42,6 @@
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
-import net.openhft.chronicle.map.ChronicleMap;
 import org.eclipse.jgit.lib.Config;
 
 @Singleton
@@ -105,8 +104,8 @@
 
     ChronicleMapCacheImpl<K, V> cache;
     try {
-      ChronicleMap<KeyWrapper<K>, TimedValue<V>> store =
-          ChronicleMapCacheImpl.createOrRecoverStore(in, config);
+      ChronicleMapStore<K, V> store =
+          ChronicleMapCacheImpl.createOrRecoverStore(in, config, metricMaker);
 
       ChronicleMapCacheLoader<K, V> memLoader =
           new ChronicleMapCacheLoader<>(
@@ -122,8 +121,7 @@
               config,
               metricMaker,
               memLoader,
-              new InMemoryCacheLoadingFromStoreImpl<>(mem, false),
-              store);
+              new InMemoryCacheLoadingFromStoreImpl<>(mem, false));
 
     } catch (IOException e) {
       throw new UncheckedIOException(e);
@@ -158,8 +156,8 @@
     ChronicleMapCacheDefProxy<K, V> def = new ChronicleMapCacheDefProxy<>(in);
 
     try {
-      ChronicleMap<KeyWrapper<K>, TimedValue<V>> store =
-          ChronicleMapCacheImpl.createOrRecoverStore(in, config);
+      ChronicleMapStore<K, V> store =
+          ChronicleMapCacheImpl.createOrRecoverStore(in, config, metricMaker);
 
       ChronicleMapCacheLoader<K, V> memLoader =
           new ChronicleMapCacheLoader<>(
@@ -175,8 +173,7 @@
               config,
               metricMaker,
               memLoader,
-              new InMemoryCacheLoadingFromStoreImpl<>(mem, true),
-              store);
+              new InMemoryCacheLoadingFromStoreImpl<>(mem, true));
     } catch (IOException e) {
       throw new UncheckedIOException(e);
     }
diff --git a/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/ChronicleMapCacheImpl.java b/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/ChronicleMapCacheImpl.java
index af35c88..b704a7b 100644
--- a/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/ChronicleMapCacheImpl.java
+++ b/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/ChronicleMapCacheImpl.java
@@ -39,7 +39,7 @@
   private static final FluentLogger logger = FluentLogger.forEnclosingClass();
 
   private final ChronicleMapCacheConfig config;
-  private final ChronicleMap<KeyWrapper<K>, TimedValue<V>> store;
+  private final ChronicleMapStore<K, V> store;
   private final LongAdder hitCount = new LongAdder();
   private final LongAdder missCount = new LongAdder();
   private final LongAdder loadSuccessCount = new LongAdder();
@@ -52,20 +52,21 @@
 
   ChronicleMapCacheImpl(PersistentCacheDef<K, V> def, ChronicleMapCacheConfig config)
       throws IOException {
+    DisabledMetricMaker metricMaker = new DisabledMetricMaker();
 
     this.cacheDefinition = def;
     this.config = config;
     this.hotEntries =
         new InMemoryLRU<>(
             (int) Math.max(config.getMaxEntries() * config.getpercentageHotKeys() / 100, 1));
-    this.store = createOrRecoverStore(def, config);
+    this.store = createOrRecoverStore(def, config, metricMaker);
     this.memLoader =
         new ChronicleMapCacheLoader<>(
             MoreExecutors.directExecutor(), store, config.getExpireAfterWrite());
     this.mem = memLoader.asInMemoryCacheBypass();
 
-    ChronicleMapStorageMetrics metrics = new ChronicleMapStorageMetrics(new DisabledMetricMaker());
-    metrics.registerCallBackMetrics(def.name(), store, hotEntries);
+    ChronicleMapCacheMetrics metrics = new ChronicleMapCacheMetrics(metricMaker);
+    metrics.registerCallBackMetrics(def.name(), this);
   }
 
   ChronicleMapCacheImpl(
@@ -73,8 +74,7 @@
       ChronicleMapCacheConfig config,
       MetricMaker metricMaker,
       ChronicleMapCacheLoader<K, V> memLoader,
-      InMemoryCache<K, V> mem,
-      ChronicleMap<KeyWrapper<K>, TimedValue<V>> store) {
+      InMemoryCache<K, V> mem) {
 
     this.cacheDefinition = def;
     this.config = config;
@@ -83,15 +83,16 @@
             (int) Math.max(config.getMaxEntries() * config.getpercentageHotKeys() / 100, 1));
     this.memLoader = memLoader;
     this.mem = mem;
-    this.store = store;
+    this.store = memLoader.getStore();
 
-    ChronicleMapStorageMetrics metrics = new ChronicleMapStorageMetrics(metricMaker);
-    metrics.registerCallBackMetrics(def.name(), store, hotEntries);
+    ChronicleMapCacheMetrics metrics = new ChronicleMapCacheMetrics(metricMaker);
+    metrics.registerCallBackMetrics(def.name(), this);
   }
 
   @SuppressWarnings({"unchecked", "cast", "rawtypes"})
-  static <K, V> ChronicleMap<KeyWrapper<K>, TimedValue<V>> createOrRecoverStore(
-      PersistentCacheDef<K, V> def, ChronicleMapCacheConfig config) throws IOException {
+  static <K, V> ChronicleMapStore<K, V> createOrRecoverStore(
+      PersistentCacheDef<K, V> def, ChronicleMapCacheConfig config, MetricMaker metricMaker)
+      throws IOException {
     CacheSerializers.registerCacheDef(def);
 
     final Class<KeyWrapper<K>> keyWrapperClass = (Class<KeyWrapper<K>>) (Class) KeyWrapper.class;
@@ -137,49 +138,29 @@
         store.remainingAutoResizes(),
         store.percentageFreeSpace());
 
-    return store;
+    return new ChronicleMapStore<>(store, config, metricMaker);
   }
 
   protected PersistentCacheDef<K, V> getCacheDefinition() {
     return cacheDefinition;
   }
 
-  private static class ChronicleMapStorageMetrics {
+  private static class ChronicleMapCacheMetrics {
 
     private final MetricMaker metricMaker;
 
-    ChronicleMapStorageMetrics(MetricMaker metricMaker) {
+    ChronicleMapCacheMetrics(MetricMaker metricMaker) {
       this.metricMaker = metricMaker;
     }
 
-    <K, V> void registerCallBackMetrics(
-        String name, ChronicleMap<KeyWrapper<K>, TimedValue<V>> store, InMemoryLRU<K> hotEntries) {
+    <K, V> void registerCallBackMetrics(String name, ChronicleMapCacheImpl<K, V> cache) {
       String sanitizedName = metricMaker.sanitizeMetricName(name);
-      String PERCENTAGE_FREE_SPACE_METRIC =
-          "cache/chroniclemap/percentage_free_space_" + sanitizedName;
-      String REMAINING_AUTORESIZES_METRIC =
-          "cache/chroniclemap/remaining_autoresizes_" + sanitizedName;
       String HOT_KEYS_CAPACITY_METRIC = "cache/chroniclemap/hot_keys_capacity_" + sanitizedName;
       String HOT_KEYS_SIZE_METRIC = "cache/chroniclemap/hot_keys_size_" + sanitizedName;
 
-      metricMaker.newCallbackMetric(
-          PERCENTAGE_FREE_SPACE_METRIC,
-          Long.class,
-          new Description(
-              String.format("The amount of free space in the %s cache as a percentage", name)),
-          () -> (long) store.percentageFreeSpace());
-
-      metricMaker.newCallbackMetric(
-          REMAINING_AUTORESIZES_METRIC,
-          Integer.class,
-          new Description(
-              String.format(
-                  "The number of times the %s cache can automatically expand its capacity", name)),
-          store::remainingAutoResizes);
-
       metricMaker.newConstantMetric(
           HOT_KEYS_CAPACITY_METRIC,
-          hotEntries.getCapacity(),
+          cache.hotEntries.getCapacity(),
           new Description(
               String.format(
                   "The number of hot cache keys for %s cache that can be kept in memory", name)));
@@ -190,7 +171,7 @@
           new Description(
               String.format(
                   "The number of hot cache keys for %s cache that are currently in memory", name)),
-          hotEntries::size);
+          cache.hotEntries::size);
     }
   }
 
@@ -276,8 +257,9 @@
   public void putUnchecked(Object key, Object value, Timestamp created) {
     TimedValue<?> wrappedValue = new TimedValue<>(value, created.toInstant().toEpochMilli());
     KeyWrapper<?> wrappedKey = new KeyWrapper<>(key);
-    store.put((KeyWrapper<K>) wrappedKey, (TimedValue<V>) wrappedValue);
-    mem.put((K) key, (TimedValue<V>) wrappedValue);
+    if (store.tryPut((KeyWrapper<K>) wrappedKey, (TimedValue<V>) wrappedValue)) {
+      mem.put((K) key, (TimedValue<V>) wrappedValue);
+    }
   }
 
   /**
@@ -292,21 +274,26 @@
    */
   @SuppressWarnings("unchecked")
   public void putUnchecked(KeyWrapper<Object> wrappedKey, TimedValue<Object> wrappedValue) {
-    store.put((KeyWrapper<K>) wrappedKey, (TimedValue<V>) wrappedValue);
-    mem.put((K) wrappedKey.getValue(), (TimedValue<V>) wrappedValue);
+    if (store.tryPut((KeyWrapper<K>) wrappedKey, (TimedValue<V>) wrappedValue)) {
+      mem.put((K) wrappedKey.getValue(), (TimedValue<V>) wrappedValue);
+    }
   }
 
   @Override
   public void put(K key, V val) {
     TimedValue<V> timedVal = new TimedValue<>(val);
-    mem.put(key, timedVal);
-    putTimedToStore(key, timedVal);
+    if (putTimedToStore(key, timedVal)) {
+      mem.put(key, timedVal);
+    }
   }
 
-  void putTimedToStore(K key, TimedValue<V> timedVal) {
+  boolean putTimedToStore(K key, TimedValue<V> timedVal) {
     KeyWrapper<K> wrappedKey = new KeyWrapper<>(key);
-    store.put(wrappedKey, timedVal);
-    hotEntries.add(key);
+    boolean putSuccess = store.tryPut(wrappedKey, timedVal);
+    if (putSuccess) {
+      hotEntries.add(key);
+    }
+    return putSuccess;
   }
 
   public void prune() {
@@ -392,4 +379,12 @@
   public void close() {
     store.close();
   }
+
+  public double percentageUsedAutoResizes() {
+    return store.percentageUsedAutoResizes();
+  }
+
+  public String name() {
+    return store.name();
+  }
 }
diff --git a/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/ChronicleMapCacheLoader.java b/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/ChronicleMapCacheLoader.java
index 1da3257..9a68780 100644
--- a/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/ChronicleMapCacheLoader.java
+++ b/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/ChronicleMapCacheLoader.java
@@ -31,14 +31,13 @@
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Executor;
 import java.util.concurrent.atomic.LongAdder;
-import net.openhft.chronicle.map.ChronicleMap;
 
 class ChronicleMapCacheLoader<K, V> extends CacheLoader<K, TimedValue<V>> {
   private static final FluentLogger logger = FluentLogger.forEnclosingClass();
 
   private final Executor storePersistenceExecutor;
   private final Optional<CacheLoader<K, V>> loader;
-  private final ChronicleMap<KeyWrapper<K>, TimedValue<V>> store;
+  private final ChronicleMapStore<K, V> store;
   private final LongAdder loadSuccessCount = new LongAdder();
   private final LongAdder loadExceptionCount = new LongAdder();
   private final LongAdder totalLoadTime = new LongAdder();
@@ -56,7 +55,7 @@
    */
   ChronicleMapCacheLoader(
       Executor storePersistenceExecutor,
-      ChronicleMap<KeyWrapper<K>, TimedValue<V>> store,
+      ChronicleMapStore<K, V> store,
       CacheLoader<K, V> loader,
       Duration expireAfterWrite) {
     this.storePersistenceExecutor = storePersistenceExecutor;
@@ -73,9 +72,7 @@
    * @param expireAfterWrite maximum lifetime of the data loaded into ChronicleMap
    */
   ChronicleMapCacheLoader(
-      Executor storePersistenceExecutor,
-      ChronicleMap<KeyWrapper<K>, TimedValue<V>> store,
-      Duration expireAfterWrite) {
+      Executor storePersistenceExecutor, ChronicleMapStore<K, V> store, Duration expireAfterWrite) {
     this.storePersistenceExecutor = storePersistenceExecutor;
     this.store = store;
     this.loader = Optional.empty();
@@ -96,9 +93,16 @@
         missCount.increment();
         long start = System.nanoTime();
         TimedValue<V> loadedValue = new TimedValue<>(loader.get().load(key));
-        loadSuccessCount.increment();
         totalLoadTime.add(System.nanoTime() - start);
-        storePersistenceExecutor.execute(() -> store.put(new KeyWrapper<>(key), loadedValue));
+        storePersistenceExecutor.execute(
+            () -> {
+              // Note that we return a loadedValue, even when we
+              // we fail populating the cache with it, to make clients more
+              // resilient to storage cache failures
+              if (store.tryPut(new KeyWrapper<>(key), loadedValue)) {
+                loadSuccessCount.increment();
+              }
+            });
         return loadedValue;
       }
 
@@ -110,6 +114,10 @@
     }
   }
 
+  public ChronicleMapStore<K, V> getStore() {
+    return store;
+  }
+
   TimedValue<V> loadIfPresent(K key) {
     TimedValue<V> h = store.get(new KeyWrapper<>(key));
     if (h != null && !expired(h.getCreated())) {
@@ -133,8 +141,9 @@
         new FutureCallback<V>() {
           @Override
           public void onSuccess(V result) {
-            store.put(new KeyWrapper<>(key), new TimedValue<>(result));
-            loadSuccessCount.increment();
+            if (store.tryPut(new KeyWrapper<>(key), new TimedValue<>(result))) {
+              loadSuccessCount.increment();
+            }
             totalLoadTime.add(System.nanoTime() - start);
           }
 
@@ -175,7 +184,7 @@
 
       @Override
       public void put(K key, TimedValue<V> value) {
-        store.put(new KeyWrapper<>(key), value);
+        store.tryPut(new KeyWrapper<>(key), value);
       }
 
       @Override
diff --git a/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/ChronicleMapStore.java b/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/ChronicleMapStore.java
new file mode 100644
index 0000000..0419ca4
--- /dev/null
+++ b/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/ChronicleMapStore.java
@@ -0,0 +1,329 @@
+// Copyright (C) 2022 The Android Open Source Project
+//
+// 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 com.googlesource.gerrit.modules.cache.chroniclemap;
+
+import com.google.common.flogger.FluentLogger;
+import com.google.gerrit.metrics.MetricMaker;
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.Type;
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.function.Predicate;
+import net.openhft.chronicle.bytes.BytesStore;
+import net.openhft.chronicle.core.io.Closeable;
+import net.openhft.chronicle.core.util.SerializableFunction;
+import net.openhft.chronicle.hash.Data;
+import net.openhft.chronicle.map.ChronicleMap;
+import net.openhft.chronicle.map.ExternalMapQueryContext;
+import net.openhft.chronicle.map.MapEntry;
+import net.openhft.chronicle.map.MapSegmentContext;
+import net.openhft.chronicle.map.VanillaChronicleMap;
+
+class ChronicleMapStore<K, V> implements ChronicleMap<KeyWrapper<K>, TimedValue<V>> {
+  private static final FluentLogger logger = FluentLogger.forEnclosingClass();
+
+  private final ChronicleMap<KeyWrapper<K>, TimedValue<V>> store;
+  private final ChronicleMapCacheConfig config;
+  private final ChronicleMapStoreMetrics metrics;
+
+  ChronicleMapStore(
+      ChronicleMap<KeyWrapper<K>, TimedValue<V>> store,
+      ChronicleMapCacheConfig config,
+      MetricMaker metricMaker) {
+
+    this.store = store;
+    this.config = config;
+    this.metrics = new ChronicleMapStoreMetrics(store.name(), metricMaker);
+    metrics.registerCallBackMetrics(this);
+  }
+
+  /**
+   * Attempt to put the key/value pair into the chronicle-map store. Also catches and warns on disk
+   * allocation errors, so that such failures result in non-cached entries rather than throwing.
+   *
+   * @param wrappedKey the wrapped key value
+   * @param timedVal the timed value
+   * @return true when the value was successfully inserted in chronicle-map, false otherwise
+   */
+  public boolean tryPut(KeyWrapper<K> wrappedKey, TimedValue<V> timedVal) {
+    try {
+      store.put(wrappedKey, timedVal);
+    } catch (IllegalArgumentException | IllegalStateException e) {
+      metrics.incrementPutFailures();
+      logger.atWarning().withCause(e).log(
+          "[cache %s] Caught exception when inserting entry '%s' in chronicle-map",
+          store.name(), wrappedKey.getValue());
+      return false;
+    }
+    return true;
+  }
+
+  @SuppressWarnings("rawtypes")
+  public double percentageUsedAutoResizes() {
+    /*
+     * Chronicle-map already exposes the number of _remaining_ auto-resizes, but
+     * this is an absolute value, and it is not enough to understand the
+     * percentage of auto-resizes that have been utilized.
+     *
+     * For that, we fist need to understand the _maximum_ number of possible
+     * resizes (inclusive of the resizes allowed by the max-bloat factor).
+     * This information is exposed at low level, by the VanillaChronicleMap,
+     * which has access to the number of allocated segments.
+     *
+     * So we proceed as follows:
+     *
+     * Calculate the maximum number of segments by multiplying the allocated
+     * segments (`actualSegments`) by the configured max-bloat-factor.
+     *
+     * The ratio between this value and the _current_ segment utilization
+     * (`getExtraTiersInUse`) shows the overall percentage.
+     */
+    VanillaChronicleMap vanillaStore = (VanillaChronicleMap) store;
+    long usedResizes = vanillaStore.globalMutableState().getExtraTiersInUse();
+    return usedResizes * 100 / maxAutoResizes();
+  }
+
+  @SuppressWarnings("rawtypes")
+  public double maxAutoResizes() {
+    return config.getMaxBloatFactor() * ((VanillaChronicleMap) store).actualSegments;
+  }
+
+  @Override
+  public int size() {
+    return store.size();
+  }
+
+  @Override
+  public boolean isEmpty() {
+    return store.isEmpty();
+  }
+
+  @Override
+  public boolean containsKey(Object key) {
+    return store.containsKey(key);
+  }
+
+  @Override
+  public boolean containsValue(Object value) {
+    return store.containsValue(value);
+  }
+
+  @Override
+  public TimedValue<V> get(Object key) {
+    return store.get(key);
+  }
+
+  @Override
+  public TimedValue<V> put(KeyWrapper<K> key, TimedValue<V> value) {
+    return store.put(key, value);
+  }
+
+  @Override
+  public TimedValue<V> remove(Object key) {
+    return store.remove(key);
+  }
+
+  @Override
+  public void putAll(Map<? extends KeyWrapper<K>, ? extends TimedValue<V>> m) {
+    store.putAll(m);
+  }
+
+  @Override
+  public void clear() {
+    store.clear();
+  }
+
+  @Override
+  public Set<KeyWrapper<K>> keySet() {
+    return store.keySet();
+  }
+
+  @Override
+  public Collection<TimedValue<V>> values() {
+    return store.values();
+  }
+
+  @Override
+  public Set<Entry<KeyWrapper<K>, TimedValue<V>>> entrySet() {
+    return store.entrySet();
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    return store.equals(o);
+  }
+
+  @Override
+  public int hashCode() {
+    return store.hashCode();
+  }
+
+  @Override
+  public TimedValue<V> getUsing(KeyWrapper<K> key, TimedValue<V> usingValue) {
+    return store.getUsing(key, usingValue);
+  }
+
+  @Override
+  public TimedValue<V> acquireUsing(KeyWrapper<K> key, TimedValue<V> usingValue) {
+    return store.acquireUsing(key, usingValue);
+  }
+
+  @Override
+  public Closeable acquireContext(KeyWrapper<K> key, TimedValue<V> usingValue) {
+    return store.acquireContext(key, usingValue);
+  }
+
+  @Override
+  public <R> R getMapped(
+      KeyWrapper<K> key, SerializableFunction<? super TimedValue<V>, R> function) {
+    return store.getMapped(key, function);
+  }
+
+  @Override
+  public void getAll(File toFile) throws IOException {
+    store.getAll(toFile);
+  }
+
+  @Override
+  public void putAll(File fromFile) throws IOException {
+    store.putAll(fromFile);
+  }
+
+  @Override
+  public Class<TimedValue<V>> valueClass() {
+    return store.valueClass();
+  }
+
+  @Override
+  public Type valueType() {
+    return store.valueType();
+  }
+
+  @Override
+  public short percentageFreeSpace() {
+    return store.percentageFreeSpace();
+  }
+
+  @Override
+  public int remainingAutoResizes() {
+    return store.remainingAutoResizes();
+  }
+
+  @Override
+  public TimedValue<V> putIfAbsent(KeyWrapper<K> key, TimedValue<V> value) {
+    return store.putIfAbsent(key, value);
+  }
+
+  @Override
+  public boolean remove(Object key, Object value) {
+    return store.remove(key, value);
+  }
+
+  @Override
+  public boolean replace(KeyWrapper<K> key, TimedValue<V> oldValue, TimedValue<V> newValue) {
+    return store.replace(key, oldValue, newValue);
+  }
+
+  @Override
+  public TimedValue<V> replace(KeyWrapper<K> key, TimedValue<V> value) {
+    return store.replace(key, value);
+  }
+
+  @Override
+  public File file() {
+    return store.file();
+  }
+
+  @Override
+  public String name() {
+    return store.name();
+  }
+
+  @Override
+  public String toIdentityString() {
+    return store.toIdentityString();
+  }
+
+  @Override
+  public long longSize() {
+    return store.longSize();
+  }
+
+  @Override
+  public long offHeapMemoryUsed() {
+    return store.offHeapMemoryUsed();
+  }
+
+  @Override
+  public Class<KeyWrapper<K>> keyClass() {
+    return store.keyClass();
+  }
+
+  @Override
+  public Type keyType() {
+    return store.keyType();
+  }
+
+  @Override
+  public ExternalMapQueryContext<KeyWrapper<K>, TimedValue<V>, ?> queryContext(KeyWrapper<K> key) {
+    return store.queryContext(key);
+  }
+
+  @Override
+  public ExternalMapQueryContext<KeyWrapper<K>, TimedValue<V>, ?> queryContext(
+      Data<KeyWrapper<K>> key) {
+    return store.queryContext(key);
+  }
+
+  @Override
+  public ExternalMapQueryContext<KeyWrapper<K>, TimedValue<V>, ?> queryContext(
+      @SuppressWarnings("rawtypes") BytesStore keyBytes, long offset, long size) {
+    return store.queryContext(keyBytes, offset, size);
+  }
+
+  @Override
+  public MapSegmentContext<KeyWrapper<K>, TimedValue<V>, ?> segmentContext(int segmentIndex) {
+    return store.segmentContext(segmentIndex);
+  }
+
+  @Override
+  public int segments() {
+    return store.segments();
+  }
+
+  @Override
+  public boolean forEachEntryWhile(
+      Predicate<? super MapEntry<KeyWrapper<K>, TimedValue<V>>> predicate) {
+    return store.forEachEntryWhile(predicate);
+  }
+
+  @Override
+  public void forEachEntry(Consumer<? super MapEntry<KeyWrapper<K>, TimedValue<V>>> action) {
+    store.forEachEntry(action);
+  }
+
+  @Override
+  public void close() {
+    store.close();
+  }
+
+  @Override
+  public boolean isOpen() {
+    return store.isOpen();
+  }
+}
diff --git a/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/ChronicleMapStoreMetrics.java b/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/ChronicleMapStoreMetrics.java
new file mode 100644
index 0000000..cf758b5
--- /dev/null
+++ b/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/ChronicleMapStoreMetrics.java
@@ -0,0 +1,76 @@
+// Copyright (C) 2022 The Android Open Source Project
+//
+// 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 com.googlesource.gerrit.modules.cache.chroniclemap;
+
+import com.google.gerrit.metrics.Counter0;
+import com.google.gerrit.metrics.Description;
+import com.google.gerrit.metrics.MetricMaker;
+
+class ChronicleMapStoreMetrics {
+  private final String sanitizedName;
+  private final MetricMaker metricMaker;
+  private final String name;
+  private final Counter0 storePutFailures;
+
+  ChronicleMapStoreMetrics(String name, MetricMaker metricMaker) {
+    this.name = name;
+    this.sanitizedName = metricMaker.sanitizeMetricName(name);
+    this.metricMaker = metricMaker;
+
+    this.storePutFailures =
+        metricMaker.newCounter(
+            "cache/chroniclemap/store_put_failures_" + sanitizedName,
+            new Description(
+                    "The number of errors caught when inserting entries in chronicle-map store: "
+                        + name)
+                .setCumulative()
+                .setUnit("errors"));
+  }
+
+  void incrementPutFailures() {
+    storePutFailures.increment();
+  }
+
+  <K, V> void registerCallBackMetrics(ChronicleMapStore<K, V> store) {
+    String PERCENTAGE_FREE_SPACE_METRIC =
+        "cache/chroniclemap/percentage_free_space_" + sanitizedName;
+    String REMAINING_AUTORESIZES_METRIC =
+        "cache/chroniclemap/remaining_autoresizes_" + sanitizedName;
+    String MAX_AUTORESIZES_METRIC = "cache/chroniclemap/max_autoresizes_" + sanitizedName;
+
+    metricMaker.newCallbackMetric(
+        PERCENTAGE_FREE_SPACE_METRIC,
+        Long.class,
+        new Description(
+            String.format("The amount of free space in the %s cache as a percentage", name)),
+        () -> (long) store.percentageFreeSpace());
+
+    metricMaker.newCallbackMetric(
+        REMAINING_AUTORESIZES_METRIC,
+        Integer.class,
+        new Description(
+            String.format(
+                "The number of times the %s cache can automatically expand its capacity", name)),
+        store::remainingAutoResizes);
+
+    metricMaker.newConstantMetric(
+        MAX_AUTORESIZES_METRIC,
+        store.maxAutoResizes(),
+        new Description(
+            String.format(
+                "The maximum number of times the %s cache can automatically expand its capacity",
+                name)));
+  }
+}
diff --git a/src/main/resources/Documentation/build.md b/src/main/resources/Documentation/build.md
index 7e23698..652b328 100644
--- a/src/main/resources/Documentation/build.md
+++ b/src/main/resources/Documentation/build.md
@@ -14,8 +14,8 @@
 git clone https://gerrit.googlesource.com/gerrit
 git clone https://gerrit.googlesource.com/modules/cache-chroniclemap
 cd gerrit/plugins
-ln -s ../../cache-chroniclemap cache-chroniclemap
-ln -sf ../../external_plugin_deps.bzl .
+ln -s ../../cache-chroniclemap .
+ln -sf ../../cache-chroniclemap/external_plugin_deps.bzl .
 ```
 
 From the Gerrit source tree issue the command `bazelsk build plugins/cache-chroniclemap`.
diff --git a/src/main/resources/Documentation/metrics.md b/src/main/resources/Documentation/metrics.md
index 8e176ef..2928708 100644
--- a/src/main/resources/Documentation/metrics.md
+++ b/src/main/resources/Documentation/metrics.md
@@ -17,8 +17,14 @@
   See the [official documentation](https://javadoc.io/static/net.openhft/chronicle-map/3.20.83/net/openhft/chronicle/map/ChronicleMap.html#remainingAutoResizes--)
   for more information.
 
+* cache/chroniclemap/max_autoresizes_<cache-name>
+  : The maximum number of times the cache can automatically expand its capacity.
+
 * cache/chroniclemap/hot_keys_capacity_<cache-name>
   : Constant number of hot keys for the cache that can be kept in memory.
 
 * cache/chroniclemap/hot_keys_size_<cache-name>
-  : The number of hot keys for the cache that are currently in memory.
\ No newline at end of file
+  : The number of hot keys for the cache that are currently in memory.
+
+* "cache/chroniclemap/store_put_failures_<cache-name>
+  : The number of errors caught when inserting entries in chronicle-map store
\ No newline at end of file
diff --git a/src/main/resources/Documentation/tuning.md b/src/main/resources/Documentation/tuning.md
index feb98fc..77078a4 100644
--- a/src/main/resources/Documentation/tuning.md
+++ b/src/main/resources/Documentation/tuning.md
@@ -12,7 +12,7 @@
 to analyze your existing H2 caches [here](#analyze-h2-caches).
 
 In case you have already migrated to chronicle-map please follow instructions on
-how to further tune existing .dat caches [here](#tune-chronicle-map-caches).
+how to further tune existing .dat caches [here](#auto-adjust-chronicle-map-caches).
 
 ## Analyze H2 caches
 
@@ -148,7 +148,7 @@
 * You can now run an the tuning command:
 
 ```bash
-ssh -p 29418 admin@<gerrit-server> cache-chroniclemap auto-adjust-caches [--dry-run]
+ssh -p 29418 admin@<gerrit-server> cache-chroniclemap auto-adjust-caches [--dry-run] [cache-name]
 ```
 
 * You can also use the REST-API:
@@ -162,6 +162,28 @@
 Calculate the average key and value size, but do not migrate current cache
 data into new files
 
+* `--max-entries` or `-m` (SSH), `?max-entries` or `?m` (REST-API) optional parameter
+
+The number of entries the tuned cache file is going to hold. This is typically
+useful when the auto-tuning is executed with the intent to increase the number
+of entries that the current cache can hold. When not specified, the
+auto-adjust-cache command checks the percentage utilization of the current
+cache.
+
+If the current utilization of the cache is higher than 50%, then `maxEntries`
+for the tuned cache will be increased by a factor of *2*.
+
+To _decrease_ the number of max entries during auto-tuning, the `max-entries`
+value should be passed _explicitly_.
+
+Note that this parameter will be used globally across all caches, so if you want
+to increase the size of a particular cache only you should be using this
+together with the `cache-name` parameter.
+
+* `cache-name` (SSH), `?CACHE_NAME=cache-name` (REST-API) optional restriction of the caches
+  to analyze and auto-tune. The parameter can be repeated multiple times for analyzing
+  multiple caches. By default, analyze and adjust all persistent caches.
+
 For each chronicle-map cache that needs tuning (i.e. `foo_1.dat` file) in
 the `cache` directory, a new one will be created (i.e. `foo_1_tuned_<timestamp>.dat`).
 The new cache will have these characteristics:
diff --git a/src/test/java/com/googlesource/gerrit/modules/cache/chroniclemap/AutoAdjustCachesIT.java b/src/test/java/com/googlesource/gerrit/modules/cache/chroniclemap/AutoAdjustCachesIT.java
index 087e804..776ee45 100644
--- a/src/test/java/com/googlesource/gerrit/modules/cache/chroniclemap/AutoAdjustCachesIT.java
+++ b/src/test/java/com/googlesource/gerrit/modules/cache/chroniclemap/AutoAdjustCachesIT.java
@@ -15,6 +15,8 @@
 package com.googlesource.gerrit.modules.cache.chroniclemap;
 
 import static com.google.common.truth.Truth.assertThat;
+import static com.googlesource.gerrit.modules.cache.chroniclemap.AutoAdjustCaches.MAX_ENTRIES_MULTIPLIER;
+import static com.googlesource.gerrit.modules.cache.chroniclemap.AutoAdjustCaches.PERCENTAGE_SIZE_INCREASE_THRESHOLD;
 import static com.googlesource.gerrit.modules.cache.chroniclemap.AutoAdjustCachesCommand.CONFIG_HEADER;
 import static com.googlesource.gerrit.modules.cache.chroniclemap.AutoAdjustCachesCommand.TUNED_INFIX;
 import static com.googlesource.gerrit.modules.cache.chroniclemap.ChronicleMapCacheConfig.Defaults.maxBloatFactorFor;
@@ -41,6 +43,7 @@
 import java.nio.file.Path;
 import java.util.Objects;
 import java.util.Set;
+import java.util.UUID;
 import java.util.function.Function;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
@@ -129,6 +132,43 @@
   }
 
   @Test
+  @GerritConfig(name = "cache.test_cache.maxEntries", value = "10")
+  @GerritConfig(name = "cache.test_cache.maxBloatFactor", value = "1")
+  public void shouldIncreaseCacheSizeWhenIsGettingFull() throws Exception {
+    ChronicleMapCacheImpl<String, String> chronicleMapCache =
+        (ChronicleMapCacheImpl<String, String>) testCache;
+
+    while (chronicleMapCache.percentageUsedAutoResizes() < PERCENTAGE_SIZE_INCREASE_THRESHOLD) {
+      String aString = UUID.randomUUID().toString();
+      testCache.put(aString, aString);
+    }
+
+    String tuneResult = adminSshSession.exec(SSH_CMD + " " + TEST_CACHE_NAME);
+    adminSshSession.assertSuccess();
+
+    Config tunedConfig = configResult(tuneResult, CONFIG_HEADER);
+    assertThat(tunedConfig.getSubsections("cache")).contains(TEST_CACHE_NAME);
+    assertThat(tunedConfig.getLong("cache", TEST_CACHE_NAME, "maxEntries", 0))
+        .isEqualTo(chronicleMapCache.getConfig().getMaxEntries() * MAX_ENTRIES_MULTIPLIER);
+  }
+
+  @Test
+  public void shouldHonourMaxEntriesParameter() throws Exception {
+    createChange();
+    Long wantedMaxEntries = 100L;
+
+    String result =
+        adminSshSession.exec(String.format("%s --max-entries %s", SSH_CMD, wantedMaxEntries));
+
+    adminSshSession.assertSuccess();
+    Config configResult = configResult(result, CONFIG_HEADER);
+
+    for (String cache : EXPECTED_CACHES) {
+      assertThat(configResult.getLong("cache", cache, "maxEntries", 0)).isEqualTo(wantedMaxEntries);
+    }
+  }
+
+  @Test
   public void shouldCreateNewCacheFiles() throws Exception {
     createChange();
 
@@ -142,12 +182,42 @@
   }
 
   @Test
+  public void shouldCreateNewCacheFileForSingleGitFileDiffCache() throws Exception {
+    createChange();
+
+    adminSshSession.exec(SSH_CMD + " " + GIT_FILE_DIFF);
+
+    adminSshSession.assertSuccess();
+    Set<String> tunedCaches =
+        tunedFileNamesSet(n -> n.matches(".*" + AutoAdjustCaches.TUNED_INFIX + ".*"));
+
+    assertThat(tunedCaches.size()).isEqualTo(1);
+  }
+
+  @Test
+  public void shouldCreateNewCacheFileForSingleGerritFileDiffCache() throws Exception {
+    createChange();
+
+    adminSshSession.exec(SSH_CMD + " " + GERRIT_FILE_DIFF);
+
+    adminSshSession.assertSuccess();
+    Set<String> tunedCaches =
+        tunedFileNamesSet(n -> n.matches(".*" + AutoAdjustCaches.TUNED_INFIX + ".*"));
+
+    assertThat(tunedCaches.size()).isEqualTo(1);
+  }
+
+  @Test
   @GerritConfig(name = "cache.test_cache.avgKeySize", value = "207")
   @GerritConfig(name = "cache.test_cache.avgValueSize", value = "207")
   public void shouldNotRecreateTestCacheFileWhenAlreadyTuned() throws Exception {
     testCache.get(TEST_CACHE_KEY_100_CHARS);
 
-    String tuneResult = adminSshSession.exec(SSH_CMD);
+    String tuneResult =
+        adminSshSession.exec(
+            String.format(
+                "%s --max-entries %s",
+                SSH_CMD, ChronicleMapCacheConfig.Defaults.maxEntriesFor(TEST_CACHE_KEY_100_CHARS)));
     adminSshSession.assertSuccess();
 
     assertThat(configResult(tuneResult, CONFIG_HEADER).getSubsections("cache"))
@@ -190,6 +260,47 @@
     assertThat(tunedFileNamesSet(MATCH_ALL)).isNotEmpty();
   }
 
+  @Test
+  public void shouldHonourMaxEntriesOverRestForAdmin() throws Exception {
+    Long wantedMaxEntries = 100L;
+
+    RestResponse resp =
+        adminRestSession.put(String.format("%s?max-entries=%s", REST_CMD, wantedMaxEntries));
+
+    resp.assertCreated();
+
+    assertThat(
+            configResult(resp.getEntityContent(), null)
+                .getLong("cache", ACCOUNTS, "maxEntries", 0L))
+        .isEqualTo(wantedMaxEntries);
+  }
+
+  @Test
+  public void shouldAllowTuningOfSingleGitFileDiffCacheOverRestForAdmin() throws Exception {
+    createChange();
+
+    RestResponse resp = adminRestSession.put(REST_CMD + "?CACHE_NAME=" + GIT_FILE_DIFF);
+
+    resp.assertCreated();
+
+    assertThat(configResult(resp.getEntityContent(), null).getSubsections("cache")).isNotEmpty();
+    assertThat(tunedFileNamesSet(n -> n.matches(".*" + AutoAdjustCaches.TUNED_INFIX + ".*")))
+        .hasSize(1);
+  }
+
+  @Test
+  public void shouldAllowTuningOfSingleGerritFileDiffCacheOverRestForAdmin() throws Exception {
+    createChange();
+
+    RestResponse resp = adminRestSession.put(REST_CMD + "?CACHE_NAME=" + GERRIT_FILE_DIFF);
+
+    resp.assertCreated();
+
+    assertThat(configResult(resp.getEntityContent(), null).getSubsections("cache")).isNotEmpty();
+    assertThat(tunedFileNamesSet(n -> n.matches(".*" + AutoAdjustCaches.TUNED_INFIX + ".*")))
+        .hasSize(1);
+  }
+
   private Config configResult(String result, @Nullable String configHeader)
       throws ConfigInvalidException {
     Config configResult = new Config();
diff --git a/src/test/java/com/googlesource/gerrit/modules/cache/chroniclemap/ChronicleMapCacheTest.java b/src/test/java/com/googlesource/gerrit/modules/cache/chroniclemap/ChronicleMapCacheTest.java
index 91ddbe3..0041c53 100644
--- a/src/test/java/com/googlesource/gerrit/modules/cache/chroniclemap/ChronicleMapCacheTest.java
+++ b/src/test/java/com/googlesource/gerrit/modules/cache/chroniclemap/ChronicleMapCacheTest.java
@@ -17,6 +17,7 @@
 import static com.google.common.truth.Truth.assertWithMessage;
 import static com.google.gerrit.testing.GerritJUnit.assertThrows;
 
+import com.codahale.metrics.Counter;
 import com.codahale.metrics.Gauge;
 import com.codahale.metrics.MetricRegistry;
 import com.google.gerrit.acceptance.AbstractDaemonTest;
@@ -355,11 +356,50 @@
     while (!cache.runningOutOfFreeSpace()) {
       cache.put(UUID.randomUUID().toString(), UUID.randomUUID().toString());
     }
-    assertThat(cache.runningOutOfFreeSpace()).isTrue();
+  }
 
-    cache.prune();
+  @Test
+  public void shouldRecoverWhenPutFailsBecauseEntryIsTooBig() throws Exception {
+    String key = UUID.randomUUID().toString();
+    String value = UUID.randomUUID().toString();
+    int uuidSize = valueSize(value);
+    gerritConfig.setInt("cache", testCacheName, "maxEntries", 1);
+    gerritConfig.setInt("cache", testCacheName, "maxBloatFactor", 1);
+    gerritConfig.setInt("cache", testCacheName, "avgKeySize", uuidSize / 2);
+    gerritConfig.setInt("cache", testCacheName, "avgValueSize", uuidSize / 2);
+    gerritConfig.save();
 
-    assertThat(cache.runningOutOfFreeSpace()).isFalse();
+    ChronicleMapCacheImpl<String, String> cache = newCacheWithMetrics(testCacheName, value);
+
+    cache.put(key, value);
+
+    assertThat(cache.getStore().size()).isEqualTo(0);
+    assertThat(cache.getIfPresent(key)).isNull();
+    assertThat(getCounter("cache/chroniclemap/store_put_failures_" + testCacheName).getCount())
+        .isEqualTo(1L);
+  }
+
+  @Test
+  public void shouldRecoverWhenPutFailsBecauseCacheCannotExpand() throws Exception {
+    String key = UUID.randomUUID().toString();
+    String value = UUID.randomUUID().toString();
+    int uuidSize = valueSize(value);
+    gerritConfig.setInt("cache", testCacheName, "maxEntries", 1);
+    gerritConfig.setInt("cache", testCacheName, "maxBloatFactor", 1);
+    gerritConfig.setInt("cache", testCacheName, "avgKeySize", uuidSize);
+    gerritConfig.setInt("cache", testCacheName, "avgValueSize", uuidSize);
+    gerritConfig.save();
+
+    ChronicleMapCacheImpl<String, String> cache = newCacheWithoutLoader();
+
+    cache.put(UUID.randomUUID().toString(), UUID.randomUUID().toString());
+    cache.put(UUID.randomUUID().toString(), UUID.randomUUID().toString());
+    cache.put(key, value);
+
+    assertThat(cache.getStore().size()).isEqualTo(2);
+    assertThat(cache.getIfPresent(key)).isNull();
+    assertThat(getCounter("cache/chroniclemap/store_put_failures_" + testCacheName).getCount())
+        .isEqualTo(1L);
   }
 
   @Test
@@ -403,6 +443,21 @@
   }
 
   @Test
+  public void shouldTriggerMaxAutoResizeMetric() throws Exception {
+    String cachedValue = UUID.randomUUID().toString();
+    String maxAutoResizeMetricName = "cache/chroniclemap/max_autoresizes_" + testCacheName;
+    gerritConfig.setInt("cache", testCacheName, "maxEntries", 2);
+    gerritConfig.setInt("cache", testCacheName, "avgKeySize", cachedValue.getBytes().length);
+    gerritConfig.setInt("cache", testCacheName, "avgValueSize", valueSize(cachedValue));
+    gerritConfig.setInt("cache", testCacheName, "maxBloatFactor", 3);
+    gerritConfig.save();
+
+    newCacheWithMetrics(testCacheName, cachedValue);
+
+    assertThat(getMetric(maxAutoResizeMetricName).getValue()).isEqualTo(3);
+  }
+
+  @Test
   public void shouldTriggerHotKeysCapacityCacheMetric() throws Exception {
     String cachedValue = UUID.randomUUID().toString();
     int percentageHotKeys = 60;
@@ -487,6 +542,7 @@
     String hotKeySizeMetricName = "cache/chroniclemap/hot_keys_size_" + sanitized;
     String percentageFreeMetricName = "cache/chroniclemap/percentage_free_space_" + sanitized;
     String autoResizeMetricName = "cache/chroniclemap/remaining_autoresizes_" + sanitized;
+    String maxAutoResizeMetricName = "cache/chroniclemap/max_autoresizes_" + sanitized;
     String hotKeyCapacityMetricName = "cache/chroniclemap/hot_keys_capacity_" + sanitized;
 
     newCacheWithMetrics(cacheName, null);
@@ -494,6 +550,7 @@
     getMetric(hotKeySizeMetricName);
     getMetric(percentageFreeMetricName);
     getMetric(autoResizeMetricName);
+    getMetric(maxAutoResizeMetricName);
     getMetric(hotKeyCapacityMetricName);
   }
 
@@ -591,4 +648,10 @@
     assertWithMessage(name).that(gauge).isNotNull();
     return gauge;
   }
+
+  private Counter getCounter(String name) {
+    Counter counter = (Counter) metricRegistry.getMetrics().get(name);
+    assertWithMessage(name).that(counter).isNotNull();
+    return counter;
+  }
 }