Improve the performance of branch extraction

Improve the performance of contributors endpoint by using
an in-memory/on-disk cache to avoid computing file diff for the same
objectIds over and over again but for different branches.

Remove also the use of parallel collection because of
greatly diminishing the throughput caused by increasing the overhead
in memory management.

The performance improvement is higher for larget repositories with
a lot of branches.

Examples:

- Extracting analytics, including branches, since 2000-01-01 from the
Gerrit Code Review repo, which at the time of writing has a size of 130Mb
and has only 15 branches might save up to 10 seconds. Arguably not that
much.

- Extracting analytics, including branches, since 2017-01-01 from the
platform/prebuilts/tools repo (part of AOSP), which at the time of
writing has a size of 22Gb and 215 branchea, might help latency to go
down from 7minutes to 20 seconds (20 times faster).

Extracting from this very same repo since 2000-01-01, allows to get a
response in a matter of minutes rather several hours.

Bug: Issue 10729
Change-Id: I991a5fc82d7c32c6e035da8b90ba4bebeab50188
diff --git a/build.sbt b/build.sbt
index f0f4745..74cfc91 100644
--- a/build.sbt
+++ b/build.sbt
@@ -6,6 +6,8 @@
 
 git.useGitDescribe := true
 
+concurrentRestrictions in Global += Tags.limit(Tags.Test, 1)
+
 lazy val root = (project in file("."))
   .settings(
     name := pluginName,
@@ -23,6 +25,8 @@
       "com.google.code.gson" % "gson" % "2.8.5" % Provided,
       "joda-time" % "joda-time" % "2.9.9",
 
+      "com.google.gerrit" % "gerrit-acceptance-framework" % gerritApiVersion % Test,
+      "org.bouncycastle" % "bcprov-jdk15on" % "1.60" % Test,
       "org.scalatest" %% "scalatest" % "3.0.1" % Test,
       "net.codingwell" %% "scala-guice" % "4.1.0" % Test),
 
diff --git a/config.md b/config.md
new file mode 100644
index 0000000..4d617ae
--- /dev/null
+++ b/config.md
@@ -0,0 +1,28 @@
+# Analytics extraction plugin configuration
+
+Once installed, this plugin requires no additional configuration and can just be
+ used out of the box.
+
+You can however, tweak gerrit.config, as follows
+
+### Commits Statistics Cache
+
+To increase throughput and minimize resource consumption this plugin makes use
+of an internal cache to store statistics associated to specific object ids.
+
+Depending on your needs you can reserve more memory to it, set expiration
+policies and decide whether you want to persist it on disk.
+
+To achieve this, this plugin makes use of Gerrit built-in cache mechanism as
+ described [here](https://gerrit-review.googlesource.com/Documentation/config-gerrit.html#cache).
+As an example, you could add the following stanza to the `gerrit.config` file:
+
+```
+[cache "analytics.commits_statistics_cache"]
+        memoryLimit = 100000
+        diskLimit = 52428800
+```
+
+Defaults:
+* `diskLimit`: disk storage for the cache is disabled
+* `memoryLimit`: 100000
\ No newline at end of file
diff --git a/src/main/scala/com/googlesource/gerrit/plugins/analytics/Contributors.scala b/src/main/scala/com/googlesource/gerrit/plugins/analytics/Contributors.scala
index 932e04c..8e048e2 100644
--- a/src/main/scala/com/googlesource/gerrit/plugins/analytics/Contributors.scala
+++ b/src/main/scala/com/googlesource/gerrit/plugins/analytics/Contributors.scala
@@ -14,6 +14,7 @@
 
 package com.googlesource.gerrit.plugins.analytics
 
+import com.google.common.cache.{Cache, LoadingCache}
 import com.google.gerrit.extensions.api.projects.CommentLinkInfo
 import com.google.gerrit.extensions.restapi.{BadRequestException, Response, RestReadView}
 import com.google.gerrit.server.git.GitRepositoryManager
@@ -21,8 +22,11 @@
 import com.google.gerrit.server.restapi.project.ProjectsCollection
 import com.google.gerrit.sshd.{CommandMetaData, SshCommand}
 import com.google.inject.Inject
+import com.google.inject.name.Named
 import com.googlesource.gerrit.plugins.analytics.common.DateConversions._
 import com.googlesource.gerrit.plugins.analytics.common._
+import com.googlesource.gerrit.plugins.analytics.common.CommitsStatisticsCache.COMMITS_STATISTICS_CACHE
+import org.eclipse.jgit.lib.ObjectId
 import org.kohsuke.args4j.{Option => ArgOption}
 
 
@@ -141,7 +145,7 @@
                                     projectCache:ProjectCache,
                                     histogram: UserActivityHistogram,
                                     gsonFmt: GsonFormatter,
-                                    botLikeExtractor: BotLikeExtractor) {
+                                    commitsStatisticsCache: CommitsStatisticsCache) {
   import RichBoolean._
 
   import scala.collection.JavaConverters._
@@ -155,11 +159,10 @@
     }.toList.flatten
 
     ManagedResource.use(repoManager.openRepository(projectRes.getNameKey)) { repo =>
-      val stats = new Statistics(repo, botLikeExtractor, commentLinks)
+      val stats  = new Statistics(projectRes.getNameKey, commitsStatisticsCache)
       val branchesExtractor = extractBranches.option(new BranchesExtractor(repo))
 
       histogram.get(repo, new AggregatedHistogramFilterByDates(startDate, stopDate, branchesExtractor, aggregationStrategy))
-        .par
         .flatMap(aggregatedCommitActivity => UserActivitySummary.apply(stats)(aggregatedCommitActivity))
         .toStream
     }
diff --git a/src/main/scala/com/googlesource/gerrit/plugins/analytics/Module.scala b/src/main/scala/com/googlesource/gerrit/plugins/analytics/Module.scala
index ca8167b..b0c643e 100644
--- a/src/main/scala/com/googlesource/gerrit/plugins/analytics/Module.scala
+++ b/src/main/scala/com/googlesource/gerrit/plugins/analytics/Module.scala
@@ -18,12 +18,14 @@
 import com.google.gerrit.server.project.ProjectResource.PROJECT_KIND
 import com.google.inject.AbstractModule
 import com.googlesource.gerrit.plugins.analytics.common.{BotLikeExtractor, BotLikeExtractorImpl}
+import com.googlesource.gerrit.plugins.analytics.common.CommitsStatisticsCacheModule
 
 class Module extends AbstractModule {
 
   override protected def configure() {
     bind(classOf[BotLikeExtractor]).to(classOf[BotLikeExtractorImpl])
 
+    install(new CommitsStatisticsCacheModule())
     install(new RestApiModule() {
       override protected def configure() = {
         get(PROJECT_KIND, "contributors").to(classOf[ContributorsResource])
diff --git a/src/main/scala/com/googlesource/gerrit/plugins/analytics/common/BotLikeExtractor.scala b/src/main/scala/com/googlesource/gerrit/plugins/analytics/common/BotLikeExtractor.scala
index ac67647..a3efffc 100644
--- a/src/main/scala/com/googlesource/gerrit/plugins/analytics/common/BotLikeExtractor.scala
+++ b/src/main/scala/com/googlesource/gerrit/plugins/analytics/common/BotLikeExtractor.scala
@@ -14,11 +14,12 @@
 
 package com.googlesource.gerrit.plugins.analytics.common
 
-import com.google.inject.Inject
+import com.google.inject.{ImplementedBy, Inject}
 import com.googlesource.gerrit.plugins.analytics.AnalyticsConfig
 
 import scala.util.matching.Regex
 
+@ImplementedBy(classOf[BotLikeExtractorImpl])
 trait BotLikeExtractor {
   def isBotLike(files: Set[String]): Boolean
 }
diff --git a/src/main/scala/com/googlesource/gerrit/plugins/analytics/common/CommitsStatistics.scala b/src/main/scala/com/googlesource/gerrit/plugins/analytics/common/CommitsStatistics.scala
index ae1509c..685fc59 100644
--- a/src/main/scala/com/googlesource/gerrit/plugins/analytics/common/CommitsStatistics.scala
+++ b/src/main/scala/com/googlesource/gerrit/plugins/analytics/common/CommitsStatistics.scala
@@ -14,18 +14,9 @@
 
 package com.googlesource.gerrit.plugins.analytics.common
 
-import com.google.gerrit.extensions.api.projects.CommentLinkInfo
+import com.google.gerrit.reviewdb.client.Project
 import com.googlesource.gerrit.plugins.analytics.{CommitInfo, IssueInfo}
-import com.googlesource.gerrit.plugins.analytics.common.ManagedResource.use
-import org.eclipse.jgit.diff.{DiffFormatter, RawTextComparator}
-import org.eclipse.jgit.lib.{ObjectId, Repository}
-import org.eclipse.jgit.revwalk.RevWalk
-import org.eclipse.jgit.treewalk.{CanonicalTreeParser, EmptyTreeIterator}
-import org.eclipse.jgit.util.io.DisabledOutputStream
-import org.slf4j.LoggerFactory
-
-import scala.collection.JavaConverters._
-import scala.util.matching.Regex
+import org.eclipse.jgit.lib.ObjectId
 
 /**
   * Collects overall stats on a series of commits and provides some basic info on the included commits
@@ -82,13 +73,7 @@
   val EmptyBotMerge = EmptyMerge.copy(isForBotLike = true)
 }
 
-class Statistics(repo: Repository, botLikeExtractor: BotLikeExtractor, commentInfoList: List[CommentLinkInfo] = Nil) {
-
-  val log = LoggerFactory.getLogger(classOf[Statistics])
-  val replacers = commentInfoList.map(info =>
-    Replacer(
-      info.`match`.r,
-      Option(info.link).getOrElse(info.html)))
+class Statistics(projectNameKey: Project.NameKey, commitStatsCache: CommitsStatisticsCache) {
 
   /**
     * Returns up to four different CommitsStatistics object grouping the stats into:
@@ -102,7 +87,7 @@
     */
   def forCommits(commits: ObjectId*): Iterable[CommitsStatistics] = {
 
-    val stats = commits.map(forSingleCommit)
+    val stats = commits.map(commitStatsCache.get(projectNameKey.get(), _))
 
     val (mergeStatsSeq, nonMergeStatsSeq) = stats.partition(_.isForMergeCommits)
 
@@ -118,57 +103,4 @@
     .filterNot(_.isEmpty)
   }
 
-  protected def forSingleCommit(objectId: ObjectId): CommitsStatistics = {
-    import RevisionBrowsingSupport._
-
-    // I can imagine this kind of statistics is already being available in Gerrit but couldn't understand how to access it
-    // which Injection can be useful for this task?
-    use(new RevWalk(repo)) { rw =>
-      val reader = repo.newObjectReader()
-      val commit = rw.parseCommit(objectId)
-      val commitMessage = commit.getFullMessage
-
-      val oldTree = {
-        // protects against initial commit
-        if (commit.getParentCount == 0)
-          new EmptyTreeIterator
-        else
-          new CanonicalTreeParser(null, reader, rw.parseCommit(commit.getParent(0).getId).getTree)
-      }
-
-      val newTree = new CanonicalTreeParser(null, reader, commit.getTree)
-
-      val df = new DiffFormatter(DisabledOutputStream.INSTANCE)
-      df.setRepository(repo)
-      df.setDiffComparator(RawTextComparator.DEFAULT)
-      df.setDetectRenames(true)
-      val diffs = df.scan(oldTree, newTree).asScala
-      case class Lines(deleted: Int, added: Int) {
-        def +(other: Lines) = Lines(deleted + other.deleted, added + other.added)
-      }
-      val lines = (for {
-        diff <- diffs
-        edit <- df.toFileHeader(diff).toEditList.asScala
-      } yield Lines(edit.getEndA - edit.getBeginA, edit.getEndB - edit.getBeginB)).fold(Lines(0, 0))(_ + _)
-
-      val files: Set[String] = diffs.map(df.toFileHeader(_).getNewPath).toSet
-
-      val commitInfo = CommitInfo(objectId.getName, commit.getAuthorIdent.getWhen.getTime, commit.isMerge, botLikeExtractor.isBotLike(files), files)
-
-      CommitsStatistics(lines.added, lines.deleted, commitInfo.merge, commitInfo.botLike, List(commitInfo), extractIssues(commitMessage))
-    }
-  }
-
-  def extractIssues(commitMessage: String): List[IssueInfo] =
-    replacers.flatMap {
-      case Replacer(pattern, replaced) =>
-        pattern.findAllIn(commitMessage)
-          .map(code => {
-            val transformed = pattern.replaceAllIn(code, replaced)
-            IssueInfo(code, transformed)
-          })
-    }
-
-  case class Replacer(pattern: Regex, replaced: String)
-
 }
diff --git a/src/main/scala/com/googlesource/gerrit/plugins/analytics/common/CommitsStatisticsCache.scala b/src/main/scala/com/googlesource/gerrit/plugins/analytics/common/CommitsStatisticsCache.scala
new file mode 100644
index 0000000..8d573f0
--- /dev/null
+++ b/src/main/scala/com/googlesource/gerrit/plugins/analytics/common/CommitsStatisticsCache.scala
@@ -0,0 +1,39 @@
+// Copyright (C) 2019 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.plugins.analytics.common
+
+import com.google.common.cache.LoadingCache
+import com.google.inject.{ImplementedBy, Inject}
+import com.google.inject.name.Named
+import com.googlesource.gerrit.plugins.analytics.common.CommitsStatisticsCache.COMMITS_STATISTICS_CACHE
+import org.eclipse.jgit.lib.ObjectId
+
+@ImplementedBy(classOf[CommitsStatisticsCacheImpl])
+trait CommitsStatisticsCache {
+  def get(project: String, objectId: ObjectId): CommitsStatistics
+}
+
+case class CommitsStatisticsCacheKey(projectName: String, commitId: ObjectId)
+
+object CommitsStatisticsCache {
+  final val COMMITS_STATISTICS_CACHE = "commits_statistics_cache"
+}
+
+class CommitsStatisticsCacheImpl @Inject() (@Named(COMMITS_STATISTICS_CACHE) commitStatsCache: LoadingCache[CommitsStatisticsCacheKey, CommitsStatistics]
+) extends CommitsStatisticsCache {
+
+  override def get(project: String, objectId: ObjectId): CommitsStatistics =
+    commitStatsCache.get(CommitsStatisticsCacheKey(project, objectId))
+}
\ No newline at end of file
diff --git a/src/main/scala/com/googlesource/gerrit/plugins/analytics/common/CommitsStatisticsCacheModule.scala b/src/main/scala/com/googlesource/gerrit/plugins/analytics/common/CommitsStatisticsCacheModule.scala
new file mode 100644
index 0000000..9b2e358
--- /dev/null
+++ b/src/main/scala/com/googlesource/gerrit/plugins/analytics/common/CommitsStatisticsCacheModule.scala
@@ -0,0 +1,32 @@
+// Copyright (C) 2019 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.plugins.analytics.common
+
+import com.google.gerrit.server.cache.CacheModule
+import com.google.gerrit.server.cache.serialize.ObjectIdCacheSerializer
+import org.eclipse.jgit.lib.ObjectId
+
+class CommitsStatisticsCacheModule extends CacheModule() {
+
+  override protected def configure(): Unit = {
+    persist(CommitsStatisticsCache.COMMITS_STATISTICS_CACHE, classOf[CommitsStatisticsCacheKey], classOf[CommitsStatistics])
+      .version(1)
+      .diskLimit(-1)
+      .maximumWeight(100000)
+      .keySerializer(CommitsStatisticsCacheKeySerializer)
+      .valueSerializer(CommitsStatisticsCacheSerializer)
+      .loader(classOf[CommitsStatisticsLoader])
+  }
+}
diff --git a/src/main/scala/com/googlesource/gerrit/plugins/analytics/common/CommitsStatisticsCacheSerializer.scala b/src/main/scala/com/googlesource/gerrit/plugins/analytics/common/CommitsStatisticsCacheSerializer.scala
new file mode 100644
index 0000000..30a2a76
--- /dev/null
+++ b/src/main/scala/com/googlesource/gerrit/plugins/analytics/common/CommitsStatisticsCacheSerializer.scala
@@ -0,0 +1,54 @@
+// Copyright (C) 2019 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.plugins.analytics.common
+
+import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectInputStream, ObjectOutputStream}
+
+import com.google.gerrit.server.cache.serialize.{CacheSerializer, ObjectIdCacheSerializer}
+import org.eclipse.jgit.lib.{Constants, ObjectId}
+
+object CommitsStatisticsCacheSerializer extends CacheSerializer[CommitsStatistics] {
+
+  override def serialize(`object`: CommitsStatistics): Array[Byte] = {
+    val stream: ByteArrayOutputStream = new ByteArrayOutputStream()
+    val oos = new ObjectOutputStream(stream)
+    oos.writeObject(`object`)
+    oos.close()
+    stream.toByteArray
+  }
+
+  override def deserialize(in: Array[Byte]): CommitsStatistics = {
+    val ois = new ObjectInputStream(new ByteArrayInputStream(in))
+    val value = ois.readObject.asInstanceOf[CommitsStatistics]
+    ois.close()
+    value
+  }
+}
+
+object CommitsStatisticsCacheKeySerializer extends CacheSerializer[CommitsStatisticsCacheKey] {
+
+  override def serialize(obj: CommitsStatisticsCacheKey): Array[Byte] = {
+    val objectIdBin = ObjectIdCacheSerializer.INSTANCE.serialize(obj.commitId)
+    val projectNameBin = obj.projectName.getBytes
+    return objectIdBin ++ projectNameBin
+  }
+
+  override def deserialize(in: Array[Byte]): CommitsStatisticsCacheKey = {
+    val objectIdBin = in.take(Constants.OBJECT_ID_LENGTH)
+    val projectNameBin = in.drop(Constants.OBJECT_ID_LENGTH)
+
+    return CommitsStatisticsCacheKey(new String(projectNameBin), ObjectId.fromRaw(objectIdBin))
+  }
+}
diff --git a/src/main/scala/com/googlesource/gerrit/plugins/analytics/common/CommitsStatisticsLoader.scala b/src/main/scala/com/googlesource/gerrit/plugins/analytics/common/CommitsStatisticsLoader.scala
new file mode 100644
index 0000000..36a175b
--- /dev/null
+++ b/src/main/scala/com/googlesource/gerrit/plugins/analytics/common/CommitsStatisticsLoader.scala
@@ -0,0 +1,103 @@
+// Copyright (C) 2019 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.plugins.analytics.common
+
+import com.google.common.cache.CacheLoader
+import com.google.gerrit.extensions.api.projects.CommentLinkInfo
+import com.google.gerrit.reviewdb.client.Project
+import com.google.gerrit.server.git.GitRepositoryManager
+import com.google.gerrit.server.project.ProjectCache
+import com.google.inject.Inject
+import com.googlesource.gerrit.plugins.analytics.{CommitInfo, IssueInfo}
+import com.googlesource.gerrit.plugins.analytics.common.ManagedResource.use
+import org.eclipse.jgit.diff.{DiffFormatter, RawTextComparator}
+import org.eclipse.jgit.revwalk.RevWalk
+import org.eclipse.jgit.treewalk.{CanonicalTreeParser, EmptyTreeIterator}
+import org.eclipse.jgit.util.io.DisabledOutputStream
+
+import scala.collection.JavaConverters._
+import scala.util.matching.Regex
+
+class CommitsStatisticsLoader @Inject() (gitRepositoryManager: GitRepositoryManager, projectCache: ProjectCache, botLikeExtractor: BotLikeExtractor) extends CacheLoader[CommitsStatisticsCacheKey, CommitsStatistics] {
+
+  override def load(cacheKey: CommitsStatisticsCacheKey): CommitsStatistics = {
+    import RevisionBrowsingSupport._
+
+    val objectId = cacheKey.commitId
+    val nameKey = new Project.NameKey(cacheKey.projectName)
+    val commentInfoList: Seq[CommentLinkInfo] = projectCache.get(nameKey).getCommentLinks.asScala
+    val replacers = commentInfoList.map(info =>
+      Replacer(
+        info.`match`.r,
+        Option(info.link).getOrElse(info.html)))
+
+    use(gitRepositoryManager.openRepository(nameKey)) { repo =>
+
+      // I can imagine this kind of statistics is already being available in Gerrit but couldn't understand how to access it
+      // which Injection can be useful for this task?
+        use(new RevWalk(repo)) { rw =>
+          val reader = repo.newObjectReader()
+          val commit = rw.parseCommit(objectId)
+          val commitMessage = commit.getFullMessage
+
+          val oldTree = {
+            // protects against initial commit
+            if (commit.getParentCount == 0)
+              new EmptyTreeIterator
+            else
+              new CanonicalTreeParser(null, reader, rw.parseCommit(commit.getParent(0).getId).getTree)
+          }
+
+          val newTree = new CanonicalTreeParser(null, reader, commit.getTree)
+
+          val df = new DiffFormatter(DisabledOutputStream.INSTANCE)
+          df.setRepository(repo)
+          df.setDiffComparator(RawTextComparator.DEFAULT)
+          df.setDetectRenames(true)
+          val diffs = df.scan(oldTree, newTree).asScala
+
+          val lines = (for {
+            diff <- diffs
+            edit <- df.toFileHeader(diff).toEditList.asScala
+          } yield Lines(edit.getEndA - edit.getBeginA, edit.getEndB - edit.getBeginB)).fold(Lines(0, 0))(_ + _)
+
+          val files: Set[String] = diffs.map(df.toFileHeader(_).getNewPath).toSet
+
+          val commitInfo = CommitInfo(objectId.getName, commit.getAuthorIdent.getWhen.getTime, commit.isMerge, botLikeExtractor.isBotLike(files), files)
+          val commitsStats = CommitsStatistics(lines.added, lines.deleted, commitInfo.merge, commitInfo.botLike, List(commitInfo), extractIssues(commitMessage, replacers).toList)
+
+          commitsStats
+        }
+    }
+  }
+
+  private def extractIssues(commitMessage: String, replacers: Seq[Replacer]): Seq[IssueInfo] =
+    replacers.flatMap {
+      case Replacer(pattern, replaced) =>
+        pattern.findAllIn(commitMessage)
+          .map(code => {
+            val transformed = pattern.replaceAllIn(code, replaced)
+            IssueInfo(code, transformed)
+          })
+    }
+
+
+
+  final private case class Replacer(pattern: Regex, replaced: String)
+
+  final private case class Lines(deleted: Int, added: Int) {
+    def +(other: Lines) = Lines(deleted + other.deleted, added + other.added)
+  }
+}
diff --git a/src/test/scala/com/googlesource/gerrit/plugins/analytics/test/CommitStatisticsCommentLinkSpec.scala b/src/test/scala/com/googlesource/gerrit/plugins/analytics/test/CommitStatisticsCommentLinkSpec.scala
index d263577..5f1fe84 100644
--- a/src/test/scala/com/googlesource/gerrit/plugins/analytics/test/CommitStatisticsCommentLinkSpec.scala
+++ b/src/test/scala/com/googlesource/gerrit/plugins/analytics/test/CommitStatisticsCommentLinkSpec.scala
@@ -16,13 +16,15 @@
 
 import com.google.gerrit.acceptance.UseLocalDisk
 import com.google.gerrit.extensions.api.projects.CommentLinkInfo
+import com.google.gerrit.reviewdb.client.Project
+import com.google.gerrit.server.git.GitRepositoryManager
 import com.googlesource.gerrit.plugins.analytics.IssueInfo
 import com.googlesource.gerrit.plugins.analytics.common.{CommitsStatistics, Statistics}
 import org.eclipse.jgit.lib.Repository
 import org.scalatest.{FlatSpec, Inside, Matchers}
 
 @UseLocalDisk
-class CommitStatisticsCommentLinkSpec extends FlatSpec with GerritTestDaemon with Matchers with Inside {
+class CommitStatisticsCommentLinkSpec extends FlatSpec with GerritTestDaemon with TestCommitStatisticsNoCache with Matchers with Inside {
 
   def createCommentLinkInfo(pattern: String, link: Option[String] = None, html: Option[String] = None) = {
     val info = new CommentLinkInfo
@@ -32,14 +34,20 @@
     info
   }
 
-  class TestEnvironment(val repo: Repository = fileRepository,
-                        val commentLinks: List[CommentLinkInfo] = List(
-                          createCommentLinkInfo(pattern = "(bug\\s+#?)(\\d+)",
-                            link = Some("http://bugs.example.com/show_bug.cgi?id=$2")),
-                          createCommentLinkInfo(pattern = "([Bb]ug:\\s+)(\\d+)",
-                            html = Some("$1<a href=\"http://trak.example.com/$2\">$2</a>")))) {
-
-    lazy val stats = new Statistics(repo, TestBotLikeExtractor, commentLinks)
+  class TestEnvironment(val repo: Repository = fileRepository) {
+    lazy val stats = new Statistics(fileRepositoryName, commitsStatisticsNoCache)
+    testFileRepository.commitFile("project.config",
+      """
+        |[access]
+        |       inheritFrom = All-Projects
+        |[submit]
+        |       action = inherit
+        |[commentlink "link1"]
+        |       match = "(bug\\s+#?)(\\d+)"
+        |       link = "http://bugs.example.com/show_bug.cgi?id=$2"
+        |[commentlink "link2"]
+        |       match = "([Bb]ug:\\s+)(\\d+)"
+        |       link = "http://trak.example.com/$2" """.stripMargin, branch = "refs/meta/config")
   }
 
   it should "collect no commentslink if no matching" in new TestEnvironment {
@@ -67,7 +75,7 @@
     inside(stats.forCommits(simpleTrackComment)) {
       case List(s: CommitsStatistics) =>
         s.issues should have size 1
-        s.issues should contain(IssueInfo("Bug: 1234", "Bug: <a href=\"http://trak.example.com/1234\">1234</a>"))
+        s.issues should contain(IssueInfo("Bug: 1234", "http://trak.example.com/1234"))
     }
   }
 
@@ -79,7 +87,7 @@
       case List(s: CommitsStatistics) =>
         s.issues should contain allOf(
           IssueInfo("bug 12", "http://bugs.example.com/show_bug.cgi?id=12"),
-          IssueInfo("Bug: 23", "Bug: <a href=\"http://trak.example.com/23\">23</a>")
+          IssueInfo("Bug: 23", "http://trak.example.com/23")
         )
     }
   }
diff --git a/src/test/scala/com/googlesource/gerrit/plugins/analytics/test/CommitStatisticsSpec.scala b/src/test/scala/com/googlesource/gerrit/plugins/analytics/test/CommitStatisticsSpec.scala
index 4b56f7c..b2ae9e7 100644
--- a/src/test/scala/com/googlesource/gerrit/plugins/analytics/test/CommitStatisticsSpec.scala
+++ b/src/test/scala/com/googlesource/gerrit/plugins/analytics/test/CommitStatisticsSpec.scala
@@ -16,15 +16,14 @@
 
 import com.google.gerrit.acceptance.UseLocalDisk
 import com.googlesource.gerrit.plugins.analytics.CommitInfo
-import com.googlesource.gerrit.plugins.analytics.common.{CommitsStatistics, Statistics}
+import com.googlesource.gerrit.plugins.analytics.common.{CommitsStatistics, CommitsStatisticsLoader, Statistics}
 import org.scalatest.{FlatSpec, Inside, Matchers}
 
 @UseLocalDisk
-class CommitStatisticsSpec extends FlatSpec with GerritTestDaemon with Matchers with Inside {
-
+class CommitStatisticsSpec extends FlatSpec with GerritTestDaemon with TestCommitStatisticsNoCache with Matchers with Inside {
   class TestEnvironment {
     val repo = fileRepository
-    val stats = new Statistics(repo, TestBotLikeExtractor)
+    val stats = new Statistics(fileRepositoryName, commitsStatisticsNoCache)
   }
 
   "CommitStatistics" should "stats a single file added" in new TestEnvironment {
diff --git a/src/test/scala/com/googlesource/gerrit/plugins/analytics/test/GerritTestDaemon.scala b/src/test/scala/com/googlesource/gerrit/plugins/analytics/test/GerritTestDaemon.scala
index 6e6f457..f10c772 100644
--- a/src/test/scala/com/googlesource/gerrit/plugins/analytics/test/GerritTestDaemon.scala
+++ b/src/test/scala/com/googlesource/gerrit/plugins/analytics/test/GerritTestDaemon.scala
@@ -18,7 +18,9 @@
 import java.util.{Date, UUID}
 
 import com.google.gerrit.acceptance.{AbstractDaemonTest, GitUtil}
+import com.google.gerrit.extensions.annotations.PluginName
 import com.google.gerrit.reviewdb.client.Project
+import com.google.inject.{AbstractModule, Module}
 import org.eclipse.jgit.api.MergeCommand.FastForwardMode
 import org.eclipse.jgit.api.{Git, MergeResult}
 import org.eclipse.jgit.internal.storage.file.FileRepository
@@ -55,6 +57,8 @@
 
   implicit var fileRepository: FileRepository = _
 
+  implicit var fileRepositoryName: Project.NameKey = null
+
   protected lazy val author = newPersonIdent("Test Author", "author@test.com")
 
   protected lazy val committer = newPersonIdent("Test Committer", "committer@test.com")
@@ -63,9 +67,8 @@
     new PersonIdent(new PersonIdent(name, email), ts)
 
   override def beforeEach {
-    fileRepository = daemonTest.getRepository(
-      daemonTest.newProject(testSpecificRepositoryName))
-
+    fileRepositoryName = daemonTest.newProject(testSpecificRepositoryName)
+    fileRepository = daemonTest.getRepository(fileRepositoryName)
     testFileRepository = GitUtil.newTestRepository(fileRepository)
   }
 
@@ -143,4 +146,13 @@
     repoManager.openRepository(projectName).asInstanceOf[FileRepository]
 
   def adminAuthor = admin.getIdent
+
+  def getInstance[T](clazz: Class[T]): T =
+    server.getTestInjector.getInstance(clazz)
+
+  override def createModule(): Module = new AbstractModule {
+    override def configure(): Unit = {
+      bind(classOf[String]).annotatedWith(classOf[PluginName]).toInstance("analytics")
+    }
+  }
 }
\ No newline at end of file
diff --git a/src/test/scala/com/googlesource/gerrit/plugins/analytics/test/TestCommitStatisticsNoCache.scala b/src/test/scala/com/googlesource/gerrit/plugins/analytics/test/TestCommitStatisticsNoCache.scala
new file mode 100644
index 0000000..8834418
--- /dev/null
+++ b/src/test/scala/com/googlesource/gerrit/plugins/analytics/test/TestCommitStatisticsNoCache.scala
@@ -0,0 +1,29 @@
+// Copyright (C) 2017 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.plugins.analytics.test
+
+import com.googlesource.gerrit.plugins.analytics.common._
+import org.eclipse.jgit.lib.ObjectId
+
+trait TestCommitStatisticsNoCache {
+  self: GerritTestDaemon =>
+
+  lazy val commitsStatisticsNoCache  = CommitsStatisticsNoCache(daemonTest.getInstance(classOf[CommitsStatisticsLoader]))
+}
+
+case class CommitsStatisticsNoCache(commitsStatisticsLoader: CommitsStatisticsLoader) extends CommitsStatisticsCache {
+  override def get(project: String, objectId: ObjectId): CommitsStatistics =
+    commitsStatisticsLoader.load(CommitsStatisticsCacheKey(project, objectId))
+}
diff --git a/src/test/scala/com/googlesource/gerrit/plugins/analytics/test/UserActivitySummarySpec.scala b/src/test/scala/com/googlesource/gerrit/plugins/analytics/test/UserActivitySummarySpec.scala
index a783290..3ecaa36 100644
--- a/src/test/scala/com/googlesource/gerrit/plugins/analytics/test/UserActivitySummarySpec.scala
+++ b/src/test/scala/com/googlesource/gerrit/plugins/analytics/test/UserActivitySummarySpec.scala
@@ -8,7 +8,7 @@
 import org.scalatest.{FlatSpec, Matchers}
 
 @UseLocalDisk
-class UserActivitySummarySpec extends FlatSpec with GerritTestDaemon with TestUtils with Matchers {
+class UserActivitySummarySpec extends FlatSpec with GerritTestDaemon with TestCommitStatisticsNoCache with TestUtils with Matchers {
 
   "numCommits" should "count only comments filtered by their merge status" in {
     val personEmail = "aCommitter@aCompany.com"
@@ -23,7 +23,7 @@
     cloneRepo.push
 
     val aggregatedCommits = aggregateBy(EMAIL)
-    val summary = UserActivitySummary.apply(new Statistics(fileRepository, TestBotLikeExtractor))(aggregatedCommits.head)
+    val summary = UserActivitySummary.apply(new Statistics(fileRepositoryName, commitsStatisticsNoCache))(aggregatedCommits.head)
 
     val nonMergeSummary = summary.head
     val mergeSummary = summary.drop(1).head