Merge branch 'stable-3.3' into master

* stable-3.3:
  Consolidated documentation under resources
  Remove command sub-package
  Introduce migrate-h2-caches command
  Introduce base command class

Change-Id: I0fff3a20f064cdfac2c4559536ac4d98ed797eb2
diff --git a/BUILD b/BUILD
index 52f23ec..22dd6f0 100644
--- a/BUILD
+++ b/BUILD
@@ -11,7 +11,7 @@
     name = "cache-chroniclemap",
     srcs = glob(["src/main/java/**/*.java"]),
     manifest_entries = [
-        "Gerrit-SshModule: com.googlesource.gerrit.modules.cache.chroniclemap.command.SSHCommandModule",
+        "Gerrit-SshModule: com.googlesource.gerrit.modules.cache.chroniclemap.SSHCommandModule",
     ],
     resources = glob(["src/main/resources/**/*"]),
     deps = [
@@ -50,6 +50,9 @@
     deps = [
         ":cache-chroniclemap__plugin",
         ":chroniclemap-test-lib",
+        "//java/com/google/gerrit/server/cache/h2",
+        "//java/com/google/gerrit/server/cache/serialize",
+        "//proto:cache_java_proto",
     ],
 )
 
diff --git a/README.md b/README.md
index 2befb1a..870dd90 100644
--- a/README.md
+++ b/README.md
@@ -6,7 +6,7 @@
 ## How to build
 
 This libModule is built like a Gerrit in-tree plugin, using Bazelisk. See the
-[build instructions](build.md) for more details.
+[build instructions](src/main/resources/Documentation/build.md) for more details.
 
 
 ## Setup
@@ -22,5 +22,9 @@
   installModule = com.googlesource.gerrit.modules.cache.chroniclemap.ChronicleMapCacheModule
 ```
 
-For further information and supported options, refer to [config](config.md)
-documentation.
\ No newline at end of file
+For further information and supported options, refer to [config](src/main/resources/Documentation/config.md)
+documentation.
+
+## Migration from H2 caches
+
+You can check how to migrate from H2 to chronicle-map [here](src/main/resources/Documentation/migration.md).
\ No newline at end of file
diff --git a/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/AnalyzeH2Caches.java b/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/AnalyzeH2Caches.java
new file mode 100644
index 0000000..bf86e78
--- /dev/null
+++ b/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/AnalyzeH2Caches.java
@@ -0,0 +1,79 @@
+// Copyright (C) 2020 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.server.config.GerritServerConfig;
+import com.google.gerrit.server.config.SitePaths;
+import com.google.inject.Inject;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Collections;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.eclipse.jgit.lib.Config;
+
+public class AnalyzeH2Caches extends H2CacheSshCommand {
+
+  @Inject
+  AnalyzeH2Caches(@GerritServerConfig Config cfg, SitePaths site) {
+    this.gerritConfig = cfg;
+    this.site = site;
+  }
+
+  @Override
+  protected void run() throws UnloggedFailure, Failure, Exception {
+    Set<Path> h2Files = getH2CacheFiles();
+    stdout.println("Extracting information from H2 caches...");
+
+    Config config = new Config();
+    for (Path h2 : h2Files) {
+      H2AggregateData stats = getStats(h2);
+      String baseName = baseName(h2);
+
+      if (stats.isEmpty()) {
+        stdout.println(String.format("WARN: Cache %s is empty, skipping.", baseName));
+        continue;
+      }
+      appendToConfig(config, stats);
+    }
+    stdout.println();
+    stdout.println("****************************");
+    stdout.println("** Chronicle-map template **");
+    stdout.println("****************************");
+    stdout.println();
+    stdout.println(config.toText());
+  }
+
+  private Set<Path> getH2CacheFiles() throws UnloggedFailure {
+
+    try {
+      return getCacheDir()
+          .map(
+              cacheDir -> {
+                try {
+                  return Files.walk(cacheDir)
+                      .filter(path -> path.toString().endsWith(H2_SUFFIX))
+                      .collect(Collectors.toSet());
+                } catch (IOException e) {
+                  logger.atSevere().withCause(e).log("Could not read H2 files");
+                  return Collections.<Path>emptySet();
+                }
+              })
+          .orElse(Collections.emptySet());
+    } catch (IOException e) {
+      throw die(e);
+    }
+  }
+}
diff --git a/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/ChronicleMapCacheConfig.java b/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/ChronicleMapCacheConfig.java
index 2fc54d3..6cdc8a7 100644
--- a/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/ChronicleMapCacheConfig.java
+++ b/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/ChronicleMapCacheConfig.java
@@ -43,6 +43,16 @@
         @Assisted File persistedFile,
         @Nullable @Assisted("ExpireAfterWrite") Duration expireAfterWrite,
         @Nullable @Assisted("RefreshAfterWrite") Duration refreshAfterWrite);
+
+    ChronicleMapCacheConfig createWithValues(
+        @Assisted("ConfigKey") String configKey,
+        @Assisted File persistedFile,
+        @Nullable @Assisted("ExpireAfterWrite") Duration expireAfterWrite,
+        @Nullable @Assisted("RefreshAfterWrite") Duration refreshAfterWrite,
+        @Assisted("maxEntries") long maxEntries,
+        @Assisted("avgKeySize") long avgKeySize,
+        @Assisted("avgValueSize") long avgValueSize,
+        @Assisted("maxBloatFactor") int maxBloatFactor);
   }
 
   @AssistedInject
@@ -52,14 +62,37 @@
       @Assisted File persistedFile,
       @Nullable @Assisted("ExpireAfterWrite") Duration expireAfterWrite,
       @Nullable @Assisted("RefreshAfterWrite") Duration refreshAfterWrite) {
+
+    this(
+        cfg,
+        configKey,
+        persistedFile,
+        expireAfterWrite,
+        refreshAfterWrite,
+        cfg.getLong("cache", configKey, "maxEntries", Defaults.maxEntriesFor(configKey)),
+        cfg.getLong("cache", configKey, "avgKeySize", Defaults.averageKeySizeFor(configKey)),
+        cfg.getLong("cache", configKey, "avgValueSize", Defaults.avgValueSizeFor(configKey)),
+        cfg.getInt("cache", configKey, "maxBloatFactor", Defaults.maxBloatFactorFor(configKey)));
+  }
+
+  @AssistedInject
+  ChronicleMapCacheConfig(
+      @GerritServerConfig Config cfg,
+      @Assisted("ConfigKey") String configKey,
+      @Assisted File persistedFile,
+      @Nullable @Assisted("ExpireAfterWrite") Duration expireAfterWrite,
+      @Nullable @Assisted("RefreshAfterWrite") Duration refreshAfterWrite,
+      @Assisted("maxEntries") long maxEntries,
+      @Assisted("avgKeySize") long avgKeySize,
+      @Assisted("avgValueSize") long avgValueSize,
+      @Assisted("maxBloatFactor") int maxBloatFactor) {
     this.persistedFile = persistedFile;
 
-    this.maxEntries =
-        cfg.getLong("cache", configKey, "maxEntries", Defaults.maxEntriesFor(configKey));
-    this.averageKeySize =
-        cfg.getLong("cache", configKey, "avgKeySize", Defaults.averageKeySizeFor(configKey));
-    this.averageValueSize =
-        cfg.getLong("cache", configKey, "avgValueSize", Defaults.avgValueSizeFor(configKey));
+    this.maxEntries = maxEntries;
+    this.averageKeySize = avgKeySize;
+    this.averageValueSize = avgValueSize;
+    this.maxBloatFactor = maxBloatFactor;
+
     this.expireAfterWrite =
         Duration.ofSeconds(
             ConfigUtil.getTimeUnit(
@@ -74,9 +107,6 @@
                 toSeconds(refreshAfterWrite),
                 SECONDS));
 
-    this.maxBloatFactor =
-        cfg.getInt("cache", configKey, "maxBloatFactor", Defaults.maxBloatFactorFor(configKey));
-
     this.percentageFreeSpaceEvictionThreshold =
         cfg.getInt(
             "cache",
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 999775d..e85e553 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
@@ -23,6 +23,7 @@
 import com.google.gerrit.server.cache.PersistentCacheDef;
 import com.google.gerrit.server.util.time.TimeUtil;
 import java.io.IOException;
+import java.sql.Timestamp;
 import java.time.Duration;
 import java.time.Instant;
 import java.util.concurrent.Callable;
@@ -229,6 +230,12 @@
     return v;
   }
 
+  @SuppressWarnings("unchecked")
+  public void putUnchecked(Object key, Object value, Timestamp created) {
+    TimedValue<?> wrapped = new TimedValue<>(value, created.toInstant().toEpochMilli());
+    store.put((K) key, (TimedValue<V>) wrapped);
+  }
+
   @Override
   public void put(K key, V val) {
     TimedValue<V> wrapped = new TimedValue<>(val);
diff --git a/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/H2AggregateData.java b/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/H2AggregateData.java
new file mode 100644
index 0000000..8e1ca68
--- /dev/null
+++ b/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/H2AggregateData.java
@@ -0,0 +1,40 @@
+// Copyright (C) 2021 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.auto.value.AutoValue;
+
+@AutoValue
+abstract class H2AggregateData {
+  protected abstract String cacheName();
+
+  protected abstract long size();
+
+  protected abstract long avgKeySize();
+
+  protected abstract long avgValueSize();
+
+  protected static H2AggregateData create(
+      String cacheName, long size, long avgKeySize, long avgValueSize) {
+    return new AutoValue_H2AggregateData(cacheName, size, avgKeySize, avgValueSize);
+  }
+
+  protected static H2AggregateData empty(String cacheName) {
+    return new AutoValue_H2AggregateData(cacheName, 0L, 0L, 0L);
+  }
+
+  protected boolean isEmpty() {
+    return size() == 0L;
+  }
+}
diff --git a/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/H2CacheSshCommand.java b/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/H2CacheSshCommand.java
new file mode 100644
index 0000000..ab964ce
--- /dev/null
+++ b/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/H2CacheSshCommand.java
@@ -0,0 +1,97 @@
+// Copyright (C) 2021 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.server.config.SitePaths;
+import com.google.gerrit.sshd.SshCommand;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Optional;
+import org.apache.commons.io.FilenameUtils;
+import org.eclipse.jgit.lib.Config;
+import org.h2.Driver;
+
+public abstract class H2CacheSshCommand extends SshCommand {
+  protected static final FluentLogger logger = FluentLogger.forEnclosingClass();
+  protected static final String H2_SUFFIX = "h2.db";
+
+  protected Config gerritConfig;
+  protected SitePaths site;
+
+  protected static String baseName(Path h2File) {
+    return FilenameUtils.removeExtension(FilenameUtils.getBaseName(h2File.toString()));
+  }
+
+  protected static H2AggregateData getStats(Path h2File) throws UnloggedFailure {
+    String url = jdbcUrl(h2File);
+    String baseName = baseName(h2File);
+    try {
+
+      try (Connection conn = Driver.load().connect(url, null);
+          Statement s = conn.createStatement();
+          ResultSet r =
+              s.executeQuery(
+                  "SELECT COUNT(*), AVG(OCTET_LENGTH(k)), AVG(OCTET_LENGTH(v)) FROM data")) {
+        if (r.next()) {
+          long size = r.getLong(1);
+          long avgKeySize = r.getLong(2);
+          long avgValueSize = r.getLong(3);
+
+          // Account for extra serialization bytes of TimedValue entries.
+          short TIMED_VALUE_WRAPPER_OVERHEAD = Long.BYTES + Integer.BYTES;
+          return H2AggregateData.create(
+              baseName, size, avgKeySize, avgValueSize + TIMED_VALUE_WRAPPER_OVERHEAD);
+        }
+        return H2AggregateData.empty(baseName);
+      }
+    } catch (SQLException e) {
+      throw new UnloggedFailure(1, "fatal: " + e.getMessage(), e);
+    }
+  }
+
+  protected static String jdbcUrl(Path h2FilePath) {
+    final String normalized =
+        FilenameUtils.removeExtension(FilenameUtils.removeExtension(h2FilePath.toString()));
+    return "jdbc:h2:" + normalized + ";AUTO_SERVER=TRUE";
+  }
+
+  protected Optional<Path> getCacheDir() throws IOException {
+    String name = gerritConfig.getString("cache", null, "directory");
+    if (name == null) {
+      return Optional.empty();
+    }
+    Path loc = site.resolve(name);
+    if (!Files.exists(loc)) {
+      throw new IOException(
+          String.format("disk cache is configured but doesn't exist: %s", loc.toAbsolutePath()));
+    }
+    if (!Files.isReadable(loc)) {
+      throw new IOException(String.format("Can't read from disk cache: %s", loc.toAbsolutePath()));
+    }
+    logger.atFine().log("Enabling disk cache %s", loc.toAbsolutePath());
+    return Optional.of(loc);
+  }
+
+  protected void appendToConfig(Config config, H2AggregateData stats) {
+    config.setLong("cache", stats.cacheName(), "maxEntries", stats.size());
+    config.setLong("cache", stats.cacheName(), "avgKeySize", stats.avgKeySize());
+    config.setLong("cache", stats.cacheName(), "avgValueSize", stats.avgValueSize());
+  }
+}
diff --git a/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/MigrateH2Caches.java b/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/MigrateH2Caches.java
new file mode 100644
index 0000000..d61fb85
--- /dev/null
+++ b/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/MigrateH2Caches.java
@@ -0,0 +1,214 @@
+// Copyright (C) 2021 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.common.data.GlobalCapability;
+import com.google.gerrit.extensions.annotations.RequiresCapability;
+import com.google.gerrit.metrics.DisabledMetricMaker;
+import com.google.gerrit.server.cache.PersistentCacheDef;
+import com.google.gerrit.server.cache.serialize.CacheSerializer;
+import com.google.gerrit.server.cache.serialize.StringCacheSerializer;
+import com.google.gerrit.server.config.GerritServerConfig;
+import com.google.gerrit.server.config.SitePaths;
+import com.google.gerrit.sshd.CommandMetaData;
+import com.google.inject.Binding;
+import com.google.inject.Inject;
+import com.google.inject.Injector;
+import com.google.inject.Key;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import org.eclipse.jgit.lib.Config;
+import org.eclipse.jgit.lib.TextProgressMonitor;
+import org.h2.Driver;
+import org.kohsuke.args4j.Option;
+
+@RequiresCapability(GlobalCapability.ADMINISTRATE_SERVER)
+@CommandMetaData(name = "migrate-h2-caches", description = "Migrate H2 caches to Chronicle-Map")
+public class MigrateH2Caches extends H2CacheSshCommand {
+
+  private final Injector injector;
+  private final ChronicleMapCacheConfig.Factory configFactory;
+
+  protected static int DEFAULT_SIZE_MULTIPLIER = 3;
+  protected static int DEFAULT_MAX_BLOAT_FACTOR = 3;
+
+  @Option(
+      name = "--size-multiplier",
+      aliases = {"-s"},
+      metaVar = "MULTIPLIER",
+      usage = "Multiplicative factor for the number of entries allowed in chronicle-map")
+  private int sizeMultiplier = DEFAULT_SIZE_MULTIPLIER;
+
+  @Option(
+      name = "--max-bloat-factor",
+      aliases = {"-m"},
+      metaVar = "FACTOR",
+      usage = "maximum number of times chronicle-map cache is allowed to grow in size")
+  private int maxBloatFactor = DEFAULT_MAX_BLOAT_FACTOR;
+
+  @Inject
+  MigrateH2Caches(
+      @GerritServerConfig Config cfg,
+      SitePaths site,
+      Injector injector,
+      ChronicleMapCacheConfig.Factory configFactory) {
+    this.injector = injector;
+    this.configFactory = configFactory;
+    this.site = site;
+    this.gerritConfig = cfg;
+  }
+
+  @Override
+  protected void run() throws Exception {
+    Optional<Path> cacheDir = getCacheDir();
+
+    if (!cacheDir.isPresent()) {
+      throw die("Cannot run migration, cache directory is not configured");
+    }
+
+    stdout.println("Migrating H2 caches to Chronicle-Map...");
+    stdout.println("* Size multiplier: " + sizeMultiplier);
+    stdout.println("* Max Bloat Factor: " + maxBloatFactor);
+    Set<PersistentCacheDef<?, ?>> cacheDefs = getAllBoundPersistentCacheDefs();
+
+    Config outputChronicleMapConfig = new Config();
+
+    for (PersistentCacheDef<?, ?> in : cacheDefs) {
+      Optional<Path> h2CacheFile = getH2CacheFile(cacheDir.get(), in.name());
+
+      if (h2CacheFile.isPresent()) {
+        H2AggregateData stats = getStats(h2CacheFile.get());
+
+        if (!stats.isEmpty()) {
+          ChronicleMapCacheImpl<?, ?> chronicleMapCache =
+              new ChronicleMapCacheImpl<>(
+                  in,
+                  makeChronicleMapConfig(
+                      configFactory, cacheDir.get(), in, stats, sizeMultiplier, maxBloatFactor),
+                  null,
+                  new DisabledMetricMaker());
+          doMigrate(h2CacheFile.get(), in, chronicleMapCache, stats.size());
+          chronicleMapCache.close();
+          appendBloatedConfig(outputChronicleMapConfig, stats);
+        }
+      }
+    }
+    stdout.println("Complete!");
+    stdout.println();
+    stdout.println("****************************");
+    stdout.println("** Chronicle-map template **");
+    stdout.println("****************************");
+    stdout.println();
+    stdout.println(outputChronicleMapConfig.toText());
+  }
+
+  protected static ChronicleMapCacheConfig makeChronicleMapConfig(
+      ChronicleMapCacheConfig.Factory configFactory,
+      Path cacheDir,
+      PersistentCacheDef<?, ?> in,
+      H2AggregateData stats,
+      int sizeMultiplier,
+      int maxBloatFactor) {
+    return configFactory.createWithValues(
+        in.configKey(),
+        ChronicleMapCacheFactory.fileName(cacheDir, in.name(), in.version()),
+        in.expireAfterWrite(),
+        in.refreshAfterWrite(),
+        stats.size() * sizeMultiplier,
+        stats.avgKeySize(),
+        stats.avgValueSize(),
+        maxBloatFactor);
+  }
+
+  private void doMigrate(
+      Path h2File,
+      PersistentCacheDef<?, ?> in,
+      ChronicleMapCacheImpl<?, ?> chronicleMapCache,
+      long totalEntries)
+      throws UnloggedFailure {
+
+    TextProgressMonitor cacheProgress = new TextProgressMonitor(stdout);
+    cacheProgress.beginTask(String.format("[%s]", in.name()), (int) totalEntries);
+
+    String url = jdbcUrl(h2File);
+    try (Connection conn = Driver.load().connect(url, null)) {
+      PreparedStatement preparedStatement =
+          conn.prepareStatement("SELECT k, v, created FROM data WHERE version=?");
+      preparedStatement.setInt(1, in.version());
+
+      try (ResultSet r = preparedStatement.executeQuery()) {
+        while (r.next()) {
+          Object key = in.keySerializer().deserialize(getBytes(r, 1, in.keySerializer()));
+          Object value = in.valueSerializer().deserialize(getBytes(r, 2, in.valueSerializer()));
+          Timestamp created = r.getTimestamp(3);
+          chronicleMapCache.putUnchecked(key, value, created);
+          cacheProgress.update(1);
+        }
+      }
+
+    } catch (Exception e) {
+      String message = String.format("FATAL: error migrating %s H2 cache", in.name());
+      logger.atSevere().withCause(e).log(message);
+      stderr.println(message);
+      throw die(e);
+    }
+    cacheProgress.endTask();
+  }
+
+  private Set<PersistentCacheDef<?, ?>> getAllBoundPersistentCacheDefs() {
+    Set<PersistentCacheDef<?, ?>> cacheDefs = new HashSet<>();
+    for (Map.Entry<Key<?>, Binding<?>> entry : injector.getParent().getAllBindings().entrySet()) {
+      final Class<?> rawType = entry.getKey().getTypeLiteral().getRawType();
+      if ("PersistentCacheDef".equals(rawType.getSimpleName())) {
+        cacheDefs.add((PersistentCacheDef<?, ?>) entry.getValue().getProvider().get());
+      }
+    }
+    return cacheDefs;
+  }
+
+  private byte[] getBytes(ResultSet r, int columnIndex, CacheSerializer<?> serializer)
+      throws SQLException {
+    return (serializer instanceof StringCacheSerializer)
+        ? r.getString(columnIndex).getBytes()
+        : r.getBytes(columnIndex);
+  }
+
+  private Optional<Path> getH2CacheFile(Path cacheDir, String name) {
+    Path h2CacheFile = cacheDir.resolve(String.format("%s.%s", name, H2_SUFFIX));
+    if (Files.exists(h2CacheFile)) {
+      return Optional.of(h2CacheFile);
+    }
+    return Optional.empty();
+  }
+
+  private void appendBloatedConfig(Config config, H2AggregateData stats) {
+    appendToConfig(
+        config,
+        H2AggregateData.create(
+            stats.cacheName(),
+            stats.size() * sizeMultiplier,
+            stats.avgKeySize(),
+            stats.avgValueSize()));
+    config.setLong("cache", stats.cacheName(), "maxBloatFactor", maxBloatFactor);
+  }
+}
diff --git a/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/command/SSHCommandModule.java b/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/SSHCommandModule.java
similarity index 82%
rename from src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/command/SSHCommandModule.java
rename to src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/SSHCommandModule.java
index 038c177..81c2a62 100644
--- a/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/command/SSHCommandModule.java
+++ b/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/SSHCommandModule.java
@@ -11,13 +11,15 @@
 // 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.command;
+package com.googlesource.gerrit.modules.cache.chroniclemap;
 
 import com.google.gerrit.sshd.PluginCommandModule;
 
 public class SSHCommandModule extends PluginCommandModule {
   @Override
   protected void configureCommands() {
+    factory(ChronicleMapCacheConfig.Factory.class);
     command("analyze-h2-caches").to(AnalyzeH2Caches.class);
+    command("migrate-h2-caches").to(MigrateH2Caches.class);
   }
 }
diff --git a/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/command/AnalyzeH2Caches.java b/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/command/AnalyzeH2Caches.java
deleted file mode 100644
index 89dfeef..0000000
--- a/src/main/java/com/googlesource/gerrit/modules/cache/chroniclemap/command/AnalyzeH2Caches.java
+++ /dev/null
@@ -1,141 +0,0 @@
-// Copyright (C) 2020 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.command;
-
-import com.google.common.flogger.FluentLogger;
-import com.google.gerrit.server.config.GerritServerConfig;
-import com.google.gerrit.server.config.SitePaths;
-import com.google.gerrit.sshd.SshCommand;
-import com.google.inject.Inject;
-import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.sql.Connection;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.Collections;
-import java.util.Optional;
-import java.util.Set;
-import java.util.stream.Collectors;
-import org.apache.commons.io.FilenameUtils;
-import org.eclipse.jgit.lib.Config;
-import org.h2.Driver;
-
-public class AnalyzeH2Caches extends SshCommand {
-  private static final FluentLogger logger = FluentLogger.forEnclosingClass();
-
-  private String cacheDirectory;
-  private SitePaths site;
-
-  @Inject
-  AnalyzeH2Caches(@GerritServerConfig Config cfg, SitePaths site) {
-    this.cacheDirectory = cfg.getString("cache", null, "directory");
-    this.site = site;
-  }
-
-  @Override
-  protected void run() throws UnloggedFailure, Failure, Exception {
-    Set<Path> h2Files = getH2CacheFiles();
-    stdout.println("Extracting information from H2 caches...");
-
-    Config config = new Config();
-    for (Path h2 : h2Files) {
-      final String url = jdbcUrl(h2);
-      final String baseName =
-          FilenameUtils.removeExtension(FilenameUtils.getBaseName(h2.toString()));
-      try {
-
-        try (Connection conn = Driver.load().connect(url, null);
-            Statement s = conn.createStatement();
-            ResultSet r =
-                s.executeQuery(
-                    "SELECT COUNT(*), AVG(OCTET_LENGTH(k)), AVG(OCTET_LENGTH(v)) FROM data")) {
-          if (r.next()) {
-            long size = r.getLong(1);
-            long avgKeySize = r.getLong(2);
-            long avgValueSize = r.getLong(3);
-
-            if (size == 0) {
-              stdout.println(String.format("WARN: Cache %s is empty, skipping.", baseName));
-              continue;
-            }
-
-            config.setLong("cache", baseName, "maxEntries", size);
-            config.setLong("cache", baseName, "avgKeySize", avgKeySize);
-
-            // Account for extra serialization bytes of TimedValue entries.
-            short TIMED_VALUE_WRAPPER_OVERHEAD = Long.BYTES + Integer.BYTES;
-            config.setLong(
-                "cache", baseName, "avgValueSize", avgValueSize + TIMED_VALUE_WRAPPER_OVERHEAD);
-          }
-        }
-      } catch (SQLException e) {
-        stderr.println(String.format("Could not get information from %s", baseName));
-        throw die(e);
-      }
-    }
-    stdout.println();
-    stdout.println("****************************");
-    stdout.println("** Chronicle-map template **");
-    stdout.println("****************************");
-    stdout.println();
-    stdout.println(config.toText());
-  }
-
-  private Set<Path> getH2CacheFiles() throws UnloggedFailure {
-
-    try {
-      final Optional<Path> maybeCacheDir = getCacheDir(site, cacheDirectory);
-
-      return maybeCacheDir
-          .map(
-              cacheDir -> {
-                try {
-                  return Files.walk(cacheDir)
-                      .filter(path -> path.toString().endsWith("h2.db"))
-                      .collect(Collectors.toSet());
-                } catch (IOException e) {
-                  logger.atSevere().withCause(e).log("Could not read H2 files");
-                  return Collections.<Path>emptySet();
-                }
-              })
-          .orElse(Collections.emptySet());
-    } catch (IOException e) {
-      throw die(e);
-    }
-  }
-
-  private String jdbcUrl(Path h2FilePath) {
-    final String normalized =
-        FilenameUtils.removeExtension(FilenameUtils.removeExtension(h2FilePath.toString()));
-    return "jdbc:h2:" + normalized + ";AUTO_SERVER=TRUE";
-  }
-
-  private static Optional<Path> getCacheDir(SitePaths site, String name) throws IOException {
-    if (name == null) {
-      return Optional.empty();
-    }
-    Path loc = site.resolve(name);
-    if (!Files.exists(loc)) {
-      throw new IOException(
-          String.format("disk cache is configured but doesn't exist: %s", loc.toAbsolutePath()));
-    }
-    if (!Files.isReadable(loc)) {
-      throw new IOException(String.format("Can't read from disk cache: %s", loc.toAbsolutePath()));
-    }
-    logger.atFine().log("Enabling disk cache %s", loc.toAbsolutePath());
-    return Optional.of(loc);
-  }
-}
diff --git a/build.md b/src/main/resources/Documentation/build.md
similarity index 100%
rename from build.md
rename to src/main/resources/Documentation/build.md
diff --git a/config.md b/src/main/resources/Documentation/config.md
similarity index 100%
rename from config.md
rename to src/main/resources/Documentation/config.md
diff --git a/metrics.md b/src/main/resources/Documentation/metrics.md
similarity index 100%
rename from metrics.md
rename to src/main/resources/Documentation/metrics.md
diff --git a/src/main/resources/Documentation/migration.md b/src/main/resources/Documentation/migration.md
new file mode 100644
index 0000000..e182d85
--- /dev/null
+++ b/src/main/resources/Documentation/migration.md
@@ -0,0 +1,89 @@
+## Migration from H2 Caches
+
+This module provides an SSH command to help converting existing cache from H2 to
+chronicle-map, which requires the `Administrate Server` capability to be
+executed.
+
+The migration must be executed _before_ switching to use chronicle-map, while
+Gerrit cache is still backed by H2.
+
+The migration can be run online without any disruption of the Gerrit server.
+However note that since the migration perform many, sequential reads from the H2
+database, it will participate to the overall database load, so you should test
+accordingly.
+
+The migration should be performed as follows:
+
+* Copy `cache-chroniclemap.jar` file in the `plugins/` directory.
+* Wait for the pluginLoader to load the new plugin. You will see an entry in
+the `error_log`:
+
+```
+INFO  com.google.gerrit.server.plugins.PluginLoader : Loaded plugin cache-chroniclemap
+```
+
+* You can now run the migration
+
+```bash
+ssh -p 29418 admin@<gerrit-server> cache-chroniclemap migrate-h2-caches \
+    [--max-bloat-factor FACTOR] \
+    [--size-multiplier MULTIPLIER]
+```
+
+This might require some time, depending on the size of the H2 caches and it will
+terminate with the output of the configuration that should be places in
+`etc/gerrit.config`in order to leverage the newly created caches correctly.
+
+For example:
+
+```Migrating H2 caches to Chronicle-Map...
+   * Size multiplier: 1
+   * Max Bloat Factor: 1
+   [diff]:                 100% (216/216)
+   [persisted_projects]:   100% (3/3)
+   [diff_summary]:         100% (216/216)
+   [accounts]:             100% (2/2)
+   [mergeability]:         100% (2444/2444)
+   Complete!
+
+   ****************************
+   ** Chronicle-map template **
+   ****************************
+
+   [cache "diff"]
+   	maxEntries = 216
+   	avgKeySize = 188
+   	avgValueSize = 796
+   	maxBloatFactor = 1
+   [cache "persisted_projects"]
+   	maxEntries = 3
+   	avgKeySize = 80
+   	avgValueSize = 4087
+   	maxBloatFactor = 1
+   [cache "diff_summary"]
+   	maxEntries = 216
+   	avgKeySize = 192
+   	avgValueSize = 254
+   	maxBloatFactor = 1
+   [cache "accounts"]
+   	maxEntries = 2
+   	avgKeySize = 52
+   	avgValueSize = 194
+   	maxBloatFactor = 1
+   [cache "mergeability"]
+   	maxEntries = 2444
+   	avgKeySize = 150
+   	avgValueSize = 20
+   	maxBloatFactor = 1
+```
+
+Optionally the SSH command can receive the following additional arguments:
+
+* --max-bloat-factor (-m) FACTOR
+
+maximum number of times chronicle-map cache is allowed to grow in size.
+*default:3*
+
+*  --size-multiplier (-s) MULTIPLIER
+Multiplicative factor for the number of entries allowed in chronicle-map.
+*default:3*
\ No newline at end of file
diff --git a/tuning.md b/src/main/resources/Documentation/tuning.md
similarity index 100%
rename from tuning.md
rename to src/main/resources/Documentation/tuning.md
diff --git a/src/test/java/com/googlesource/gerrit/modules/cache/chroniclemap/command/AnalyzeH2CachesIT.java b/src/test/java/com/googlesource/gerrit/modules/cache/chroniclemap/AnalyzeH2CachesIT.java
similarity index 97%
rename from src/test/java/com/googlesource/gerrit/modules/cache/chroniclemap/command/AnalyzeH2CachesIT.java
rename to src/test/java/com/googlesource/gerrit/modules/cache/chroniclemap/AnalyzeH2CachesIT.java
index d6ae02d..e7074e0 100644
--- a/src/test/java/com/googlesource/gerrit/modules/cache/chroniclemap/command/AnalyzeH2CachesIT.java
+++ b/src/test/java/com/googlesource/gerrit/modules/cache/chroniclemap/AnalyzeH2CachesIT.java
@@ -12,7 +12,7 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
-package com.googlesource.gerrit.modules.cache.chroniclemap.command;
+package com.googlesource.gerrit.modules.cache.chroniclemap;
 
 import static com.google.common.truth.Truth.assertThat;
 
@@ -32,7 +32,7 @@
 @UseSsh
 @TestPlugin(
     name = "cache-chroniclemap",
-    sshModule = "com.googlesource.gerrit.modules.cache.chroniclemap.command.SSHCommandModule")
+    sshModule = "com.googlesource.gerrit.modules.cache.chroniclemap.SSHCommandModule")
 public class AnalyzeH2CachesIT extends LightweightPluginDaemonTest {
 
   @Inject private SitePaths sitePaths;
diff --git a/src/test/java/com/googlesource/gerrit/modules/cache/chroniclemap/MigrateH2CachesIT.java b/src/test/java/com/googlesource/gerrit/modules/cache/chroniclemap/MigrateH2CachesIT.java
new file mode 100644
index 0000000..89fe5f6
--- /dev/null
+++ b/src/test/java/com/googlesource/gerrit/modules/cache/chroniclemap/MigrateH2CachesIT.java
@@ -0,0 +1,226 @@
+// Copyright (C) 2021 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 static com.google.common.truth.Truth.assertThat;
+import static com.googlesource.gerrit.modules.cache.chroniclemap.H2CacheSshCommand.H2_SUFFIX;
+import static com.googlesource.gerrit.modules.cache.chroniclemap.MigrateH2Caches.DEFAULT_MAX_BLOAT_FACTOR;
+import static com.googlesource.gerrit.modules.cache.chroniclemap.MigrateH2Caches.DEFAULT_SIZE_MULTIPLIER;
+
+import com.google.common.base.Joiner;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.gerrit.acceptance.LightweightPluginDaemonTest;
+import com.google.gerrit.acceptance.Sandboxed;
+import com.google.gerrit.acceptance.TestPlugin;
+import com.google.gerrit.acceptance.UseLocalDisk;
+import com.google.gerrit.acceptance.UseSsh;
+import com.google.gerrit.acceptance.WaitUtil;
+import com.google.gerrit.entities.CachedProjectConfig;
+import com.google.gerrit.entities.Project;
+import com.google.gerrit.entities.RefNames;
+import com.google.gerrit.metrics.DisabledMetricMaker;
+import com.google.gerrit.server.account.CachedAccountDetails;
+import com.google.gerrit.server.cache.PersistentCacheDef;
+import com.google.gerrit.server.cache.h2.H2CacheImpl;
+import com.google.gerrit.server.cache.proto.Cache;
+import com.google.gerrit.server.cache.serialize.ObjectIdConverter;
+import com.google.gerrit.server.config.SitePaths;
+import com.google.gerrit.server.git.GitRepositoryManager;
+import com.google.gerrit.sshd.BaseCommand;
+import com.google.inject.Binding;
+import com.google.inject.Inject;
+import com.google.inject.Key;
+import java.io.IOException;
+import java.lang.annotation.Annotation;
+import java.nio.file.Path;
+import java.time.Duration;
+import java.util.Map;
+import org.eclipse.jgit.lib.Repository;
+import org.junit.Before;
+import org.junit.Test;
+
+@Sandboxed
+@UseSsh
+@TestPlugin(
+    name = "cache-chroniclemap",
+    sshModule = "com.googlesource.gerrit.modules.cache.chroniclemap.SSHCommandModule")
+public class MigrateH2CachesIT extends LightweightPluginDaemonTest {
+  private final Duration LOAD_CACHE_WAIT_TIMEOUT = Duration.ofSeconds(4);
+  private String ACCOUNTS_CACHE_NAME = "accounts";
+  private String PERSISTED_PROJECTS_CACHE_NAME = "persisted_projects";
+
+  @Inject protected GitRepositoryManager repoManager;
+  @Inject private SitePaths sitePaths;
+
+  private ChronicleMapCacheConfig.Factory chronicleMapCacheConfigFactory;
+
+  private String cmd = Joiner.on(" ").join("cache-chroniclemap", "migrate-h2-caches");
+
+  @Before
+  public void setUp() {
+    chronicleMapCacheConfigFactory =
+        plugin.getSshInjector().getInstance(ChronicleMapCacheConfig.Factory.class);
+  }
+
+  @Test
+  @UseLocalDisk
+  public void shouldRunAndCompleteSuccessfullyWhenCacheDirectoryIsDefined() throws Exception {
+    String result = adminSshSession.exec(cmd);
+    adminSshSession.assertSuccess();
+    assertThat(result).contains("Complete");
+  }
+
+  @Test
+  @UseLocalDisk
+  public void shouldOutputChronicleMapBloatedConfiguration() throws Exception {
+    waitForCacheToLoad(ACCOUNTS_CACHE_NAME);
+    waitForCacheToLoad(PERSISTED_PROJECTS_CACHE_NAME);
+
+    String result = adminSshSession.exec(cmd);
+    adminSshSession.assertSuccess();
+
+    assertThat(result)
+        .contains(
+            "[cache \""
+                + ACCOUNTS_CACHE_NAME
+                + "\"]\n"
+                + "\tmaxEntries = "
+                + H2CacheFor(ACCOUNTS_CACHE_NAME).diskStats().size() * DEFAULT_SIZE_MULTIPLIER);
+
+    assertThat(result)
+        .contains(
+            "[cache \""
+                + PERSISTED_PROJECTS_CACHE_NAME
+                + "\"]\n"
+                + "\tmaxEntries = "
+                + H2CacheFor(PERSISTED_PROJECTS_CACHE_NAME).diskStats().size()
+                    * DEFAULT_SIZE_MULTIPLIER);
+  }
+
+  @Test
+  public void shouldFailWhenCacheDirectoryIsNotDefined() throws Exception {
+    adminSshSession.exec(cmd);
+    adminSshSession.assertFailure("fatal: Cannot run migration, cache directory is not configured");
+  }
+
+  @Test
+  public void shouldFailWhenUserHasNoAdminServerCapability() throws Exception {
+    userSshSession.exec(cmd);
+    userSshSession.assertFailure("administrateServer for plugin cache-chroniclemap not permitted");
+  }
+
+  @Test
+  @UseLocalDisk
+  public void shouldMigrateAccountsCache() throws Exception {
+    waitForCacheToLoad(ACCOUNTS_CACHE_NAME);
+
+    adminSshSession.exec(cmd);
+    adminSshSession.assertSuccess();
+
+    ChronicleMapCacheImpl<CachedAccountDetails.Key, CachedAccountDetails> chronicleMapCache =
+        chronicleCacheFor(ACCOUNTS_CACHE_NAME);
+    H2CacheImpl<CachedAccountDetails.Key, CachedAccountDetails> h2Cache =
+        H2CacheFor(ACCOUNTS_CACHE_NAME);
+
+    assertThat(chronicleMapCache.diskStats().size()).isEqualTo(h2Cache.diskStats().size());
+  }
+
+  @Test
+  @UseLocalDisk
+  public void shouldMigratePersistentProjects() throws Exception {
+    waitForCacheToLoad(PERSISTED_PROJECTS_CACHE_NAME);
+
+    adminSshSession.exec(cmd);
+    adminSshSession.assertSuccess();
+
+    H2CacheImpl<Cache.ProjectCacheKeyProto, CachedProjectConfig> h2Cache =
+        H2CacheFor(PERSISTED_PROJECTS_CACHE_NAME);
+    ChronicleMapCacheImpl<Cache.ProjectCacheKeyProto, CachedProjectConfig> chronicleMapCache =
+        chronicleCacheFor(PERSISTED_PROJECTS_CACHE_NAME);
+
+    Cache.ProjectCacheKeyProto allUsersProto = projectCacheKey(allUsers);
+    Cache.ProjectCacheKeyProto allProjectsProto = projectCacheKey(allProjects);
+
+    assertThat(chronicleMapCache.get(allUsersProto)).isEqualTo(h2Cache.get(allUsersProto));
+    assertThat(chronicleMapCache.get(allProjectsProto)).isEqualTo(h2Cache.get(allProjectsProto));
+  }
+
+  private Cache.ProjectCacheKeyProto projectCacheKey(Project.NameKey key) throws IOException {
+    try (Repository git = repoManager.openRepository(key)) {
+      return Cache.ProjectCacheKeyProto.newBuilder()
+          .setProject(key.get())
+          .setRevision(
+              ObjectIdConverter.create()
+                  .toByteString(git.exactRef(RefNames.REFS_CONFIG).getObjectId()))
+          .build();
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  private <K, V> PersistentCacheDef<K, V> getPersistentCacheDef(String named) {
+    return findClassBoundWithName(PersistentCacheDef.class, named);
+  }
+
+  @SuppressWarnings("unchecked")
+  private <K, V> H2CacheImpl<K, V> H2CacheFor(String named) {
+    return (H2CacheImpl<K, V>) findClassBoundWithName(LoadingCache.class, named);
+  }
+
+  @SuppressWarnings("unchecked")
+  private <K, V> CacheLoader<K, V> cacheLoaderFor(String named) {
+    return findClassBoundWithName(CacheLoader.class, named);
+  }
+
+  private <T> T findClassBoundWithName(Class<T> clazz, String named) {
+    return plugin.getSysInjector().getAllBindings().entrySet().stream()
+        .filter(entry -> isClassBoundWithName(entry, clazz.getSimpleName(), named))
+        .findFirst()
+        .map(entry -> clazz.cast(entry.getValue().getProvider().get()))
+        .get();
+  }
+
+  private boolean isClassBoundWithName(
+      Map.Entry<Key<?>, Binding<?>> entry, String classNameMatch, String named) {
+    String className = entry.getKey().getTypeLiteral().getRawType().getSimpleName();
+    Annotation annotation = entry.getKey().getAnnotation();
+    return className.equals(classNameMatch)
+        && annotation != null
+        && annotation.toString().endsWith(String.format("Named(value=\"%s\")", named));
+  }
+
+  private <K, V> ChronicleMapCacheImpl<K, V> chronicleCacheFor(String cacheName)
+      throws BaseCommand.UnloggedFailure, IOException {
+    Path cacheDirectory = sitePaths.resolve(cfg.getString("cache", null, "directory"));
+
+    PersistentCacheDef<K, V> persistentDef = getPersistentCacheDef(cacheName);
+    ChronicleMapCacheConfig config =
+        MigrateH2Caches.makeChronicleMapConfig(
+            chronicleMapCacheConfigFactory,
+            cacheDirectory,
+            persistentDef,
+            H2CacheSshCommand.getStats(
+                cacheDirectory.resolve(String.format("%s.%s", cacheName, H2_SUFFIX))),
+            DEFAULT_SIZE_MULTIPLIER,
+            DEFAULT_MAX_BLOAT_FACTOR);
+
+    return new ChronicleMapCacheImpl<>(
+        persistentDef, config, cacheLoaderFor(cacheName), new DisabledMetricMaker());
+  }
+
+  private void waitForCacheToLoad(String cacheName) throws InterruptedException {
+    WaitUtil.waitUntil(() -> H2CacheFor(cacheName).diskStats().size() > 0, LOAD_CACHE_WAIT_TIMEOUT);
+  }
+}