Cache the comment context
This change introduces a cache interface for the comment context
(CommentContextCache). The request execution flow is as follows:
ListChangeComments loads all comments from the change notes and passes
them to the CommentJson for formatting. If the "context" flag is set,
CommentJson uses the CommentContextCache to evaluate and retrieve the
context for all comments.
The comment context cache interface uses a persisted cache and provides
two methods: get() and getAll(). The getAll() method is the one used by
CommentJson. The cache loader also overrides the load() and loadAll()
methods. The cache loader uses CommentContextLoader.java to load the
context by opening the files at the exact commits specified for each
comment.
The loadAll() method groups all comments by project and change ID (both
specify a certain commit). All comments in the same group are loaded
simultaneouly.
Change-Id: I4bc24c0556974dac99d14fef3128f6db8da18e54
diff --git a/Documentation/config-gerrit.txt b/Documentation/config-gerrit.txt
index 6b89d67..6d238f7 100644
--- a/Documentation/config-gerrit.txt
+++ b/Documentation/config-gerrit.txt
@@ -989,6 +989,11 @@
be expensive to compute (60 or more seconds for a large history
like the Linux kernel repository).
+cache `"comment_context"`::
++
+Caches the context lines of comments, which are the lines of the source file
+highlighted by the user when the comment was written.
+
cache `"groups"`::
+
Caches the basic group information of internal groups by group ID,
diff --git a/java/com/google/gerrit/entities/CommentContext.java b/java/com/google/gerrit/entities/CommentContext.java
new file mode 100644
index 0000000..183f6d0
--- /dev/null
+++ b/java/com/google/gerrit/entities/CommentContext.java
@@ -0,0 +1,29 @@
+// 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.google.gerrit.entities;
+
+import com.google.auto.value.AutoValue;
+import com.google.common.collect.ImmutableMap;
+
+/** An entity class representing all context lines of a comment. */
+@AutoValue
+public abstract class CommentContext {
+ public static CommentContext create(ImmutableMap<Integer, String> lines) {
+ return new AutoValue_CommentContext(lines);
+ }
+
+ /** Map of {line number, line text} of the context lines of a comment */
+ public abstract ImmutableMap<Integer, String> lines();
+}
diff --git a/java/com/google/gerrit/server/CommentContextLoader.java b/java/com/google/gerrit/server/CommentContextLoader.java
index 813dad7..01a78a7 100644
--- a/java/com/google/gerrit/server/CommentContextLoader.java
+++ b/java/com/google/gerrit/server/CommentContextLoader.java
@@ -17,18 +17,20 @@
import static java.util.stream.Collectors.groupingBy;
import com.google.auto.value.AutoValue;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Streams;
import com.google.common.flogger.FluentLogger;
+import com.google.gerrit.entities.Comment;
+import com.google.gerrit.entities.CommentContext;
import com.google.gerrit.entities.Project;
import com.google.gerrit.exceptions.StorageException;
-import com.google.gerrit.extensions.common.CommentInfo;
import com.google.gerrit.extensions.common.ContextLineInfo;
import com.google.gerrit.server.git.GitRepositoryManager;
import com.google.gerrit.server.patch.Text;
import com.google.inject.Inject;
import com.google.inject.assistedinject.Assisted;
import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
@@ -48,7 +50,6 @@
private final GitRepositoryManager repoManager;
private final Project.NameKey project;
- private Map<ContextData, List<ContextLineInfo>> candidates;
public interface Factory {
CommentContextLoader create(Project.NameKey project);
@@ -58,80 +59,66 @@
public CommentContextLoader(GitRepositoryManager repoManager, @Assisted Project.NameKey project) {
this.repoManager = repoManager;
this.project = project;
- this.candidates = new HashMap<>();
}
/**
- * Returns an empty list of {@link ContextLineInfo}. Clients are expected to call this method one
- * or more times. Each call returns a reference to an empty {@link List<ContextLineInfo>}.
+ * Load the comment context for multiple comments at once. This method will open the repository
+ * and read the source files for all necessary comments' file paths.
*
- * <p>A single call to {@link #fill()} will cause all list references returned from this method to
- * be populated. If a client calls this method again with a comment that was passed before calling
- * {@link #fill()}, the new populated list will be returned.
- *
- * @param comment the comment entity for which we want to load the context
- * @return a list of {@link ContextLineInfo}
+ * @param comments a list of comments.
+ * @return a Map where all entries consist of the input comments and the values are their
+ * corresponding {@link CommentContext}.
*/
- public List<ContextLineInfo> getContext(CommentInfo comment) {
- ContextData key =
- ContextData.create(
- comment.id,
- ObjectId.fromString(comment.commitId),
- comment.path,
- getStartAndEndLines(comment));
- List<ContextLineInfo> context = candidates.get(key);
- if (context == null) {
- context = new ArrayList<>();
- candidates.put(key, context);
- }
- return context;
- }
+ public Map<Comment, CommentContext> getContext(Iterable<Comment> comments) {
+ ImmutableMap.Builder<Comment, CommentContext> result =
+ ImmutableMap.builderWithExpectedSize(Iterables.size(comments));
- /**
- * A call to this method loads the context for all comments stored in {@link
- * CommentContextLoader#candidates}. This is useful so that the repository is opened once for all
- * comments.
- */
- public void fill() {
// Group comments by commit ID so that each commit is parsed only once
- Map<ObjectId, List<ContextData>> commentsByCommitId =
- candidates.keySet().stream().collect(groupingBy(ContextData::commitId));
+ Map<ObjectId, List<Comment>> commentsByCommitId =
+ Streams.stream(comments).collect(groupingBy(Comment::getCommitId));
try (Repository repo = repoManager.openRepository(project);
RevWalk rw = new RevWalk(repo)) {
for (ObjectId commitId : commentsByCommitId.keySet()) {
RevCommit commit = rw.parseCommit(commitId);
- for (ContextData k : commentsByCommitId.get(commitId)) {
- if (!k.range().isPresent()) {
+ for (Comment comment : commentsByCommitId.get(commitId)) {
+ Optional<Range> range = getStartAndEndLines(comment);
+ if (!range.isPresent()) {
continue;
}
- try (TreeWalk tw = TreeWalk.forPath(rw.getObjectReader(), k.path(), commit.getTree())) {
+ // TODO(ghareeb): We can further group the comments by file paths to avoid opening
+ // the same file multiple times.
+ try (TreeWalk tw =
+ TreeWalk.forPath(rw.getObjectReader(), comment.key.filename, commit.getTree())) {
if (tw == null) {
logger.atWarning().log(
"Failed to find path %s in the git tree of ID %s.",
- k.path(), commit.getTree().getId());
+ comment.key.filename, commit.getTree().getId());
continue;
}
ObjectId id = tw.getObjectId(0);
Text src = new Text(repo.open(id, Constants.OBJ_BLOB));
- List<ContextLineInfo> contextLines = candidates.get(k);
- Range r = k.range().get();
- for (int i = r.start(); i <= r.end(); i++) {
- contextLines.add(new ContextLineInfo(i, src.getString(i - 1)));
+ Range r = range.get();
+ ImmutableMap.Builder<Integer, String> context =
+ ImmutableMap.builderWithExpectedSize(r.end() - r.start());
+ for (int i = r.start(); i < r.end(); i++) {
+ context.put(i, src.getString(i - 1));
}
+ result.put(comment, CommentContext.create(context.build()));
}
}
}
+ return result.build();
} catch (IOException e) {
throw new StorageException("Failed to load the comment context", e);
}
}
- private static Optional<Range> getStartAndEndLines(CommentInfo comment) {
+ private static Optional<Range> getStartAndEndLines(Comment comment) {
if (comment.range != null) {
- return Optional.of(Range.create(comment.range.startLine, comment.range.endLine));
- } else if (comment.line != null) {
- return Optional.of(Range.create(comment.line, comment.line));
+ return Optional.of(Range.create(comment.range.startLine, comment.range.endLine + 1));
+ } else if (comment.lineNbr > 0) {
+ return Optional.of(Range.create(comment.lineNbr, comment.lineNbr + 1));
}
return Optional.empty();
}
@@ -142,23 +129,10 @@
return new AutoValue_CommentContextLoader_Range(start, end);
}
+ /** Start line of the comment (inclusive). */
abstract int start();
+ /** End line of the comment (exclusive). */
abstract int end();
}
-
- @AutoValue
- abstract static class ContextData {
- static ContextData create(String id, ObjectId commitId, String path, Optional<Range> range) {
- return new AutoValue_CommentContextLoader_ContextData(id, commitId, path, range);
- }
-
- abstract String id();
-
- abstract ObjectId commitId();
-
- abstract String path();
-
- abstract Optional<Range> range();
- }
}
diff --git a/java/com/google/gerrit/server/comment/CommentContextCache.java b/java/com/google/gerrit/server/comment/CommentContextCache.java
new file mode 100644
index 0000000..8c40763
--- /dev/null
+++ b/java/com/google/gerrit/server/comment/CommentContextCache.java
@@ -0,0 +1,42 @@
+// 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.google.gerrit.server.comment;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.gerrit.entities.CommentContext;
+
+/**
+ * Caches the context lines of comments (source file content surrounding and including the lines
+ * where the comment was written)
+ */
+public interface CommentContextCache {
+
+ /**
+ * Returns the context lines for a single comment.
+ *
+ * @param key a key representing a subset of fields for a comment that serves as an identifier.
+ * @return a {@link CommentContext} object containing all line numbers and text of the context.
+ */
+ CommentContext get(CommentContextKey key);
+
+ /**
+ * Returns the context lines for multiple comments - identified by their {@code keys}.
+ *
+ * @param keys list of keys, where each key represents a single comment through its project,
+ * change ID, patchset, path and ID. The keys can belong to different projects and changes.
+ * @return {@code Map} of {@code CommentContext} containing the context for all comments.
+ */
+ ImmutableMap<CommentContextKey, CommentContext> getAll(Iterable<CommentContextKey> keys);
+}
diff --git a/java/com/google/gerrit/server/comment/CommentContextCacheImpl.java b/java/com/google/gerrit/server/comment/CommentContextCacheImpl.java
new file mode 100644
index 0000000..c4e29d8
--- /dev/null
+++ b/java/com/google/gerrit/server/comment/CommentContextCacheImpl.java
@@ -0,0 +1,256 @@
+// 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.google.gerrit.server.comment;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Streams;
+import com.google.common.hash.Hashing;
+import com.google.gerrit.entities.Change;
+import com.google.gerrit.entities.Comment;
+import com.google.gerrit.entities.CommentContext;
+import com.google.gerrit.entities.HumanComment;
+import com.google.gerrit.entities.Project;
+import com.google.gerrit.exceptions.StorageException;
+import com.google.gerrit.proto.Protos;
+import com.google.gerrit.server.CommentContextLoader;
+import com.google.gerrit.server.CommentsUtil;
+import com.google.gerrit.server.cache.CacheModule;
+import com.google.gerrit.server.cache.proto.Cache.AllCommentContextProto;
+import com.google.gerrit.server.cache.proto.Cache.AllCommentContextProto.CommentContextProto;
+import com.google.gerrit.server.cache.serialize.CacheSerializer;
+import com.google.gerrit.server.notedb.ChangeNotes;
+import com.google.inject.Inject;
+import com.google.inject.Module;
+import com.google.inject.name.Named;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/** Implementation of {@link CommentContextCache}. */
+public class CommentContextCacheImpl implements CommentContextCache {
+ private static final String CACHE_NAME = "comment_context";
+
+ public static Module module() {
+ return new CacheModule() {
+ @Override
+ protected void configure() {
+ persist(CACHE_NAME, CommentContextKey.class, CommentContext.class)
+ .version(1)
+ .diskLimit(1 << 30) // limit the total cache size to 1 GB
+ .maximumWeight(1 << 23) // Limit the size of the in-memory cache to 8 MB
+ .weigher(CommentContextWeigher.class)
+ .keySerializer(CommentContextKey.Serializer.INSTANCE)
+ .valueSerializer(CommentContextSerializer.INSTANCE)
+ .loader(Loader.class);
+
+ bind(CommentContextCache.class).to(CommentContextCacheImpl.class);
+ }
+ };
+ }
+
+ private final LoadingCache<CommentContextKey, CommentContext> contextCache;
+
+ @Inject
+ CommentContextCacheImpl(
+ @Named(CACHE_NAME) LoadingCache<CommentContextKey, CommentContext> contextCache) {
+ this.contextCache = contextCache;
+ }
+
+ @Override
+ public CommentContext get(CommentContextKey comment) {
+ return getAll(ImmutableList.of(comment)).get(comment);
+ }
+
+ @Override
+ public ImmutableMap<CommentContextKey, CommentContext> getAll(
+ Iterable<CommentContextKey> inputKeys) {
+ ImmutableMap.Builder<CommentContextKey, CommentContext> result = ImmutableMap.builder();
+
+ // Convert the input keys to the same keys but with their file paths hashed
+ Map<CommentContextKey, CommentContextKey> keysToCacheKeys =
+ Streams.stream(inputKeys)
+ .collect(
+ Collectors.toMap(
+ Function.identity(),
+ k -> k.toBuilder().path(Loader.hashPath(k.path())).build()));
+
+ try {
+ ImmutableMap<CommentContextKey, CommentContext> allContext =
+ contextCache.getAll(keysToCacheKeys.values());
+
+ for (CommentContextKey inputKey : inputKeys) {
+ CommentContextKey cacheKey = keysToCacheKeys.get(inputKey);
+ result.put(inputKey, allContext.get(cacheKey));
+ }
+ return result.build();
+ } catch (ExecutionException e) {
+ throw new StorageException("Failed to retrieve comments' context", e);
+ }
+ }
+
+ public enum CommentContextSerializer implements CacheSerializer<CommentContext> {
+ INSTANCE;
+
+ @Override
+ public byte[] serialize(CommentContext commentContext) {
+ AllCommentContextProto.Builder allBuilder = AllCommentContextProto.newBuilder();
+
+ commentContext
+ .lines()
+ .entrySet()
+ .forEach(
+ c ->
+ allBuilder.addContext(
+ CommentContextProto.newBuilder()
+ .setLineNumber(c.getKey())
+ .setContextLine(c.getValue())));
+ return Protos.toByteArray(allBuilder.build());
+ }
+
+ @Override
+ public CommentContext deserialize(byte[] in) {
+ ImmutableMap.Builder<Integer, String> contextLinesMap = ImmutableMap.builder();
+ Protos.parseUnchecked(AllCommentContextProto.parser(), in).getContextList().stream()
+ .forEach(c -> contextLinesMap.put(c.getLineNumber(), c.getContextLine()));
+ return CommentContext.create(contextLinesMap.build());
+ }
+ }
+
+ static class Loader extends CacheLoader<CommentContextKey, CommentContext> {
+ private final ChangeNotes.Factory notesFactory;
+ private final CommentsUtil commentsUtil;
+ private final CommentContextLoader.Factory factory;
+
+ @Inject
+ Loader(
+ CommentsUtil commentsUtil,
+ ChangeNotes.Factory notesFactory,
+ CommentContextLoader.Factory factory) {
+ this.commentsUtil = commentsUtil;
+ this.notesFactory = notesFactory;
+ this.factory = factory;
+ }
+
+ @Override
+ public CommentContext load(CommentContextKey key) {
+ return loadAll(ImmutableList.of(key)).get(key);
+ }
+
+ @Override
+ public Map<CommentContextKey, CommentContext> loadAll(
+ Iterable<? extends CommentContextKey> keys) {
+ ImmutableMap.Builder<CommentContextKey, CommentContext> result =
+ ImmutableMap.builderWithExpectedSize(Iterables.size(keys));
+
+ Map<Project.NameKey, Map<Change.Id, List<CommentContextKey>>> groupedKeys =
+ Streams.stream(keys)
+ .distinct()
+ .map(k -> (CommentContextKey) k)
+ .collect(
+ Collectors.groupingBy(
+ CommentContextKey::project,
+ Collectors.groupingBy(CommentContextKey::changeId)));
+
+ for (Map.Entry<Project.NameKey, Map<Change.Id, List<CommentContextKey>>> perProject :
+ groupedKeys.entrySet()) {
+ Map<Change.Id, List<CommentContextKey>> keysPerProject = perProject.getValue();
+
+ for (Map.Entry<Change.Id, List<CommentContextKey>> perChange : keysPerProject.entrySet()) {
+ Map<CommentContextKey, CommentContext> context =
+ loadForSameChange(perChange.getValue(), perProject.getKey(), perChange.getKey());
+ result.putAll(context);
+ }
+ }
+ return result.build();
+ }
+
+ /**
+ * Load the comment context for comments of the same project and change ID.
+ *
+ * @param keys a list of keys corresponding to some comments
+ * @param project a gerrit project/repository
+ * @param changeId an identifier for a change
+ * @return a map of the input keys to their corresponding {@link CommentContext}
+ */
+ private Map<CommentContextKey, CommentContext> loadForSameChange(
+ List<CommentContextKey> keys, Project.NameKey project, Change.Id changeId) {
+ ChangeNotes notes = notesFactory.createChecked(project, changeId);
+ List<HumanComment> humanComments = commentsUtil.publishedHumanCommentsByChange(notes);
+ CommentContextLoader loader = factory.create(project);
+ Map<Comment, CommentContextKey> commentsToKeys = new HashMap<>();
+ for (CommentContextKey key : keys) {
+ commentsToKeys.put(getCommentForKey(humanComments, key), key);
+ }
+ Map<Comment, CommentContext> allContext = loader.getContext(commentsToKeys.keySet());
+ return allContext.entrySet().stream()
+ .collect(Collectors.toMap(e -> commentsToKeys.get(e.getKey()), Map.Entry::getValue));
+ }
+
+ /**
+ * Return the single comment from the {@code allComments} input list corresponding to the key
+ * parameter.
+ *
+ * @param allComments a list of comments.
+ * @param key a key representing a single comment.
+ * @return the single comment corresponding to the key parameter.
+ */
+ private Comment getCommentForKey(List<HumanComment> allComments, CommentContextKey key) {
+ return allComments.stream()
+ .filter(
+ c ->
+ key.id().equals(c.key.uuid)
+ && key.patchset() == c.key.patchSetId
+ && key.path().equals(hashPath(c.key.filename)))
+ .findFirst()
+ .orElseThrow(() -> new IllegalArgumentException("Unable to find comment for key " + key));
+ }
+
+ /**
+ * Hash an input String using the general {@link Hashing#murmur3_128()} hash.
+ *
+ * @param input the input String
+ * @return a hashed representation of the input String
+ */
+ static String hashPath(String input) {
+ return Hashing.murmur3_128().hashString(input, UTF_8).toString();
+ }
+ }
+
+ private static class CommentContextWeigher implements Weigher<CommentContextKey, CommentContext> {
+ @Override
+ public int weigh(CommentContextKey key, CommentContext commentContext) {
+ int size = 0;
+ size += key.id().length();
+ size += key.path().length();
+ size += key.project().get().length();
+ size += 4;
+ for (String line : commentContext.lines().values()) {
+ size += 4; // line number
+ size += line.length(); // number of characters in the context line
+ }
+ return size;
+ }
+ }
+}
diff --git a/java/com/google/gerrit/server/comment/CommentContextKey.java b/java/com/google/gerrit/server/comment/CommentContextKey.java
new file mode 100644
index 0000000..e4a927a
--- /dev/null
+++ b/java/com/google/gerrit/server/comment/CommentContextKey.java
@@ -0,0 +1,82 @@
+package com.google.gerrit.server.comment;
+
+import com.google.auto.value.AutoValue;
+import com.google.gerrit.entities.Change;
+import com.google.gerrit.entities.Project;
+import com.google.gerrit.proto.Protos;
+import com.google.gerrit.server.cache.proto.Cache;
+import com.google.gerrit.server.cache.serialize.CacheSerializer;
+import java.util.Collection;
+
+/**
+ * An identifier of a comment that should be used to load the comment context using {@link
+ * CommentContextCache#get(CommentContextKey)}, or {@link CommentContextCache#getAll(Collection)}.
+ *
+ * <p>The {@link CommentContextCacheImpl} implementation uses this class as the cache key, while
+ * replacing the {@link #path()} field with the hashed path.
+ */
+@AutoValue
+public abstract class CommentContextKey {
+ abstract Project.NameKey project();
+
+ abstract Change.Id changeId();
+
+ /** The unique comment ID. */
+ abstract String id();
+
+ /** File path at which the comment was written. */
+ abstract String path();
+
+ abstract Integer patchset();
+
+ abstract Builder toBuilder();
+
+ public static Builder builder() {
+ return new AutoValue_CommentContextKey.Builder();
+ }
+
+ @AutoValue.Builder
+ public abstract static class Builder {
+
+ public abstract Builder project(Project.NameKey nameKey);
+
+ public abstract Builder changeId(Change.Id changeId);
+
+ public abstract Builder id(String id);
+
+ public abstract Builder path(String path);
+
+ public abstract Builder patchset(Integer patchset);
+
+ public abstract CommentContextKey build();
+ }
+
+ public enum Serializer implements CacheSerializer<CommentContextKey> {
+ INSTANCE;
+
+ @Override
+ public byte[] serialize(CommentContextKey key) {
+ return Protos.toByteArray(
+ Cache.CommentContextKeyProto.newBuilder()
+ .setProject(key.project().get())
+ .setChangeId(key.changeId().toString())
+ .setPatchset(key.patchset())
+ .setPathHash(key.path())
+ .setCommentId(key.id())
+ .build());
+ }
+
+ @Override
+ public CommentContextKey deserialize(byte[] in) {
+ Cache.CommentContextKeyProto proto =
+ Protos.parseUnchecked(Cache.CommentContextKeyProto.parser(), in);
+ return CommentContextKey.builder()
+ .project(Project.NameKey.parse(proto.getProject()))
+ .changeId(Change.Id.tryParse(proto.getChangeId()).get())
+ .patchset(proto.getPatchset())
+ .id(proto.getCommentId())
+ .path(proto.getPathHash())
+ .build();
+ }
+ }
+}
diff --git a/java/com/google/gerrit/server/config/GerritGlobalModule.java b/java/com/google/gerrit/server/config/GerritGlobalModule.java
index 0023395..054f490 100644
--- a/java/com/google/gerrit/server/config/GerritGlobalModule.java
+++ b/java/com/google/gerrit/server/config/GerritGlobalModule.java
@@ -114,6 +114,7 @@
import com.google.gerrit.server.change.MergeabilityCacheImpl;
import com.google.gerrit.server.change.ReviewerSuggestion;
import com.google.gerrit.server.change.RevisionJson;
+import com.google.gerrit.server.comment.CommentContextCacheImpl;
import com.google.gerrit.server.events.EventFactory;
import com.google.gerrit.server.events.EventListener;
import com.google.gerrit.server.events.EventsMetrics;
@@ -248,6 +249,7 @@
install(TagCache.module());
install(OAuthTokenCache.module());
install(PureRevertCache.module());
+ install(CommentContextCacheImpl.module());
install(new AccessControlModule());
install(new CmdLineParserModule());
diff --git a/java/com/google/gerrit/server/restapi/change/CommentJson.java b/java/com/google/gerrit/server/restapi/change/CommentJson.java
index 67049e8..4a9cc51 100644
--- a/java/com/google/gerrit/server/restapi/change/CommentJson.java
+++ b/java/com/google/gerrit/server/restapi/change/CommentJson.java
@@ -20,9 +20,12 @@
import com.google.common.base.Strings;
import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Streams;
import com.google.gerrit.common.Nullable;
+import com.google.gerrit.entities.Change;
import com.google.gerrit.entities.Comment;
+import com.google.gerrit.entities.CommentContext;
import com.google.gerrit.entities.FixReplacement;
import com.google.gerrit.entities.FixSuggestion;
import com.google.gerrit.entities.HumanComment;
@@ -31,15 +34,18 @@
import com.google.gerrit.extensions.client.Comment.Range;
import com.google.gerrit.extensions.client.Side;
import com.google.gerrit.extensions.common.CommentInfo;
+import com.google.gerrit.extensions.common.ContextLineInfo;
import com.google.gerrit.extensions.common.FixReplacementInfo;
import com.google.gerrit.extensions.common.FixSuggestionInfo;
import com.google.gerrit.extensions.common.RobotCommentInfo;
import com.google.gerrit.extensions.restapi.Url;
-import com.google.gerrit.server.CommentContextLoader;
import com.google.gerrit.server.account.AccountLoader;
+import com.google.gerrit.server.comment.CommentContextCache;
+import com.google.gerrit.server.comment.CommentContextKey;
import com.google.gerrit.server.permissions.PermissionBackendException;
import com.google.inject.Inject;
import java.util.ArrayList;
+import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
@@ -47,18 +53,19 @@
public class CommentJson {
private final AccountLoader.Factory accountLoaderFactory;
+ private final CommentContextCache commentContextCache;
+
+ private Project.NameKey project;
+ private Change.Id changeId;
private boolean fillAccounts = true;
private boolean fillPatchSet;
- private CommentContextLoader.Factory commentContextLoaderFactory;
- private CommentContextLoader commentContextLoader;
+ private boolean fillCommentContext;
@Inject
- CommentJson(
- AccountLoader.Factory accountLoaderFactory,
- CommentContextLoader.Factory commentContextLoaderFactory) {
+ CommentJson(AccountLoader.Factory accountLoaderFactory, CommentContextCache commentContextCache) {
this.accountLoaderFactory = accountLoaderFactory;
- this.commentContextLoaderFactory = commentContextLoaderFactory;
+ this.commentContextCache = commentContextCache;
}
CommentJson setFillAccounts(boolean fillAccounts) {
@@ -71,10 +78,18 @@
return this;
}
- CommentJson setEnableContext(boolean enableContext, Project.NameKey project) {
- if (enableContext) {
- this.commentContextLoader = commentContextLoaderFactory.create(project);
- }
+ CommentJson setFillCommentContext(boolean fillCommentContext) {
+ this.fillCommentContext = fillCommentContext;
+ return this;
+ }
+
+ CommentJson setProjectKey(Project.NameKey project) {
+ this.project = project;
+ return this;
+ }
+
+ CommentJson setChangeId(Change.Id changeId) {
+ this.changeId = changeId;
return this;
}
@@ -93,9 +108,6 @@
if (loader != null) {
loader.fill();
}
- if (commentContextLoader != null) {
- commentContextLoader.fill();
- }
return info;
}
@@ -120,8 +132,10 @@
if (loader != null) {
loader.fill();
}
- if (commentContextLoader != null) {
- commentContextLoader.fill();
+
+ if (fillCommentContext) {
+ List<T> allComments = out.values().stream().flatMap(Collection::stream).collect(toList());
+ addCommentContext(allComments);
}
return out;
}
@@ -138,12 +152,41 @@
if (loader != null) {
loader.fill();
}
- if (commentContextLoader != null) {
- commentContextLoader.fill();
+
+ if (fillCommentContext) {
+ addCommentContext(out);
}
+
return out;
}
+ protected void addCommentContext(List<T> allComments) {
+ List<CommentContextKey> keys =
+ allComments.stream().map(this::createCommentContextKey).collect(toList());
+ ImmutableMap<CommentContextKey, CommentContext> allContext = commentContextCache.getAll(keys);
+ for (T c : allComments) {
+ c.contextLines = toContextLineInfoList(allContext.get(createCommentContextKey(c)));
+ }
+ }
+
+ protected List<ContextLineInfo> toContextLineInfoList(CommentContext commentContext) {
+ List<ContextLineInfo> result = new ArrayList<>();
+ for (Map.Entry<Integer, String> e : commentContext.lines().entrySet()) {
+ result.add(new ContextLineInfo(e.getKey(), e.getValue()));
+ }
+ return result;
+ }
+
+ protected CommentContextKey createCommentContextKey(T r) {
+ return CommentContextKey.builder()
+ .project(project)
+ .changeId(changeId)
+ .id(r.id)
+ .path(r.path)
+ .patchset(r.patchSet)
+ .build();
+ }
+
protected abstract T toInfo(F comment, AccountLoader loader);
protected void fillCommentInfo(Comment c, CommentInfo r, AccountLoader loader) {
@@ -171,9 +214,6 @@
r.author = loader.get(c.author.getId());
}
r.commitId = c.getCommitId().getName();
- if (commentContextLoader != null) {
- r.contextLines = commentContextLoader.getContext(r);
- }
}
protected Range toRange(Comment.Range commentRange) {
diff --git a/java/com/google/gerrit/server/restapi/change/ListChangeComments.java b/java/com/google/gerrit/server/restapi/change/ListChangeComments.java
index fec7bdc..fa7c1f5 100644
--- a/java/com/google/gerrit/server/restapi/change/ListChangeComments.java
+++ b/java/com/google/gerrit/server/restapi/change/ListChangeComments.java
@@ -19,14 +19,12 @@
import com.google.common.collect.ImmutableList;
import com.google.gerrit.entities.ChangeMessage;
import com.google.gerrit.entities.HumanComment;
-import com.google.gerrit.entities.Project;
import com.google.gerrit.extensions.common.CommentInfo;
import com.google.gerrit.extensions.common.ContextLineInfo;
import com.google.gerrit.extensions.restapi.AuthException;
import com.google.gerrit.extensions.restapi.Response;
import com.google.gerrit.extensions.restapi.RestReadView;
import com.google.gerrit.server.ChangeMessagesUtil;
-import com.google.gerrit.server.CommentContextLoader;
import com.google.gerrit.server.CommentsUtil;
import com.google.gerrit.server.change.ChangeResource;
import com.google.gerrit.server.permissions.PermissionBackendException;
@@ -42,7 +40,6 @@
private final ChangeData.Factory changeDataFactory;
private final Provider<CommentJson> commentJson;
private final CommentsUtil commentsUtil;
- private final CommentContextLoader.Factory commentContextFactory;
private boolean includeContext;
@@ -62,13 +59,11 @@
ChangeData.Factory changeDataFactory,
Provider<CommentJson> commentJson,
CommentsUtil commentsUtil,
- ChangeMessagesUtil changeMessagesUtil,
- CommentContextLoader.Factory commentContextFactory) {
+ ChangeMessagesUtil changeMessagesUtil) {
this.changeDataFactory = changeDataFactory;
this.commentJson = commentJson;
this.commentsUtil = commentsUtil;
this.changeMessagesUtil = changeMessagesUtil;
- this.commentContextFactory = commentContextFactory;
}
@Override
@@ -88,8 +83,7 @@
private ImmutableList<CommentInfo> getAsList(Iterable<HumanComment> comments, ChangeResource rsrc)
throws PermissionBackendException {
- ImmutableList<CommentInfo> commentInfos =
- getCommentFormatter(rsrc.getProject()).formatAsList(comments);
+ ImmutableList<CommentInfo> commentInfos = getCommentFormatter(rsrc).formatAsList(comments);
List<ChangeMessage> changeMessages = changeMessagesUtil.byChange(rsrc.getNotes());
CommentsUtil.linkCommentsToChangeMessages(commentInfos, changeMessages, true);
return commentInfos;
@@ -97,8 +91,7 @@
private Map<String, List<CommentInfo>> getAsMap(
Iterable<HumanComment> comments, ChangeResource rsrc) throws PermissionBackendException {
- Map<String, List<CommentInfo>> commentInfosMap =
- getCommentFormatter(rsrc.getProject()).format(comments);
+ Map<String, List<CommentInfo>> commentInfosMap = getCommentFormatter(rsrc).format(comments);
List<CommentInfo> commentInfos =
commentInfosMap.values().stream().flatMap(List::stream).collect(toList());
List<ChangeMessage> changeMessages = changeMessagesUtil.byChange(rsrc.getNotes());
@@ -106,12 +99,14 @@
return commentInfosMap;
}
- private CommentJson.HumanCommentFormatter getCommentFormatter(Project.NameKey project) {
+ private CommentJson.HumanCommentFormatter getCommentFormatter(ChangeResource rsrc) {
return commentJson
.get()
.setFillAccounts(true)
.setFillPatchSet(true)
- .setEnableContext(includeContext, project)
+ .setFillCommentContext(includeContext)
+ .setProjectKey(rsrc.getProject())
+ .setChangeId(rsrc.getId())
.newHumanCommentFormatter();
}
}
diff --git a/javatests/com/google/gerrit/acceptance/server/change/CommentsIT.java b/javatests/com/google/gerrit/acceptance/server/change/CommentsIT.java
index 72453fd..b72e202 100644
--- a/javatests/com/google/gerrit/acceptance/server/change/CommentsIT.java
+++ b/javatests/com/google/gerrit/acceptance/server/change/CommentsIT.java
@@ -1091,6 +1091,76 @@
contextLines("2", "line_2", "3", "line_3", "4", "line_4", "5", "line_5"));
}
+ @Test
+ public void commentContextForCommentsOnDifferentPatchsets() throws Exception {
+ PushOneCommit.Result r1 = createChange();
+
+ ImmutableList.Builder<String> content = ImmutableList.builder();
+ for (int i = 1; i <= 10; i++) {
+ content.add("line_" + i);
+ }
+
+ PushOneCommit.Result r2 =
+ pushFactory
+ .create(
+ admin.newIdent(),
+ testRepo,
+ SUBJECT,
+ FILE_NAME,
+ String.join("\n", content.build()),
+ r1.getChangeId())
+ .to("refs/for/master");
+
+ PushOneCommit.Result r3 =
+ pushFactory
+ .create(
+ admin.newIdent(),
+ testRepo,
+ SUBJECT,
+ FILE_NAME,
+ content.build().stream().collect(Collectors.joining("\n")),
+ r1.getChangeId())
+ .to("refs/for/master");
+
+ addCommentOnLine(r2, "r2: please fix", 1);
+ addCommentOnRange(r2, "r2: looks good", commentRangeInLines(2, 3));
+ addCommentOnLine(r3, "r3: please fix", 6);
+ addCommentOnRange(r3, "r3: looks good", commentRangeInLines(7, 8));
+
+ List<CommentInfo> comments =
+ gApi.changes().id(r2.getChangeId()).commentsRequest().withContext(true).getAsList();
+
+ assertThat(comments).hasSize(4);
+
+ assertThat(
+ comments.stream()
+ .filter(c -> c.message.equals("r2: please fix"))
+ .collect(MoreCollectors.onlyElement())
+ .contextLines)
+ .containsExactlyElementsIn(contextLines("1", "line_1"));
+
+ assertThat(
+ comments.stream()
+ .filter(c -> c.message.equals("r2: looks good"))
+ .collect(MoreCollectors.onlyElement())
+ .contextLines)
+ .containsExactlyElementsIn(contextLines("2", "line_2", "3", "line_3"));
+
+ assertThat(
+ comments.stream()
+ .filter(c -> c.message.equals("r3: please fix"))
+ .collect(MoreCollectors.onlyElement())
+ .contextLines)
+ .containsExactlyElementsIn(contextLines("6", "line_6"));
+
+ assertThat(
+ comments.stream()
+ .filter(c -> c.message.equals("r3: looks good"))
+ .collect(MoreCollectors.onlyElement())
+ .contextLines)
+ .containsExactlyElementsIn(contextLines("7", "line_7", "8", "line_8"));
+ }
+
private List<ContextLineInfo> contextLines(String... args) {
List<ContextLineInfo> result = new ArrayList<>();
for (int i = 0; i < args.length; i += 2) {
@@ -1814,7 +1884,7 @@
Comment.Range range)
throws Exception {
CommentInput c = new CommentInput();
- c.line = 1;
+ c.line = line == null ? 1 : line;
c.message = message;
c.path = FILE_NAME;
c.unresolved = unresolved;
diff --git a/javatests/com/google/gerrit/server/cache/serialize/BUILD b/javatests/com/google/gerrit/server/cache/serialize/BUILD
index fa6a717..6976d19 100644
--- a/javatests/com/google/gerrit/server/cache/serialize/BUILD
+++ b/javatests/com/google/gerrit/server/cache/serialize/BUILD
@@ -5,6 +5,7 @@
srcs = glob(["*.java"]),
deps = [
"//java/com/google/gerrit/entities",
+ "//java/com/google/gerrit/extensions:api",
"//java/com/google/gerrit/server",
"//java/com/google/gerrit/server/cache/serialize",
"//java/com/google/gerrit/server/cache/testing",
diff --git a/javatests/com/google/gerrit/server/cache/serialize/CommentContextSerializerTest.java b/javatests/com/google/gerrit/server/cache/serialize/CommentContextSerializerTest.java
new file mode 100644
index 0000000..84f290c
--- /dev/null
+++ b/javatests/com/google/gerrit/server/cache/serialize/CommentContextSerializerTest.java
@@ -0,0 +1,41 @@
+package com.google.gerrit.server.cache.serialize;
+
+import static com.google.common.truth.Truth.assertThat;
+import static com.google.gerrit.server.comment.CommentContextCacheImpl.CommentContextSerializer.INSTANCE;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.gerrit.entities.Change;
+import com.google.gerrit.entities.CommentContext;
+import com.google.gerrit.entities.Project;
+import com.google.gerrit.server.comment.CommentContextKey;
+import org.junit.Test;
+
+public class CommentContextSerializerTest {
+ @Test
+ public void roundTripValue() {
+ CommentContext commentContext =
+ CommentContext.create(ImmutableMap.of(1, "line_1", 2, "line_2"));
+
+ byte[] serialized = INSTANCE.serialize(commentContext);
+ CommentContext deserialized = INSTANCE.deserialize(serialized);
+
+ assertThat(commentContext).isEqualTo(deserialized);
+ }
+
+ @Test
+ public void roundTripKey() {
+ Project.NameKey proj = Project.NameKey.parse("project");
+ Change.Id changeId = Change.Id.tryParse("1234").get();
+
+ CommentContextKey k =
+ CommentContextKey.builder()
+ .project(proj)
+ .changeId(changeId)
+ .id("commentId")
+ .path("pathHash")
+ .patchset(1)
+ .build();
+ byte[] serialized = CommentContextKey.Serializer.INSTANCE.serialize(k);
+ assertThat(k).isEqualTo(CommentContextKey.Serializer.INSTANCE.deserialize(serialized));
+ }
+}
diff --git a/proto/cache.proto b/proto/cache.proto
index 7924cbd..6e1c1a8 100644
--- a/proto/cache.proto
+++ b/proto/cache.proto
@@ -500,3 +500,25 @@
bytes global_config_revision = 3; // Hash of All-Projects-projects.config. This
// will only be populated for All-Projects.
}
+
+// Serialized form of com.google.gerrit.server.comment.CommentContextCacheImpl.Key
+// Next ID: 6
+message CommentContextKeyProto {
+ string project = 1;
+ string change_id = 2;
+ int32 patchset = 3;
+ string commentId = 4;
+
+ // hashed with the murmur3_128 hash function
+ string path_hash = 5;
+}
+
+// Serialized form of a list of com.google.gerrit.extensions.common.ContextLineInfo
+// Next ID: 2
+message AllCommentContextProto {
+ message CommentContextProto {
+ int32 line_number = 1;
+ string context_line = 2;
+ }
+ repeated CommentContextProto context = 1;
+}