Work around buggy MyersDiff by killing threads

The JGIt MyersDiff class contains a bug that triggers an infinite loop
on only certain input files.  Most source code is able to be processed
in a reasonable time bound, but some just steal a thread and never
return to the caller.

Implement a custom thread pool that is used to invoke MyersDiff for
the intraline difference data.  If the worker thread doesn't end
within the configured time bound (default of 5 seconds), Gerrit
removes the worker from the pool and tries to kill the worker with the
unsafe Thread.stop() method.

A custom thread pool is used to try and make Thread.stop() safe by
having the amount of data accessed by each worker thread be limited to
only the "constant" inputs supplied by the cache lookup request, and
the result that the thread would return.

If any locks are released early as a result of ThreadDeath going up
the worker thread stack at worst only the incoming or outgoing queues
that are private to that worker will be corrupted.  Since these queues
are private to the worker, and to the thread that is currently
borrowing this worker from the pool (and who is also now killing it),
we can safely ensure that the queues won't be touched after the
Thread.stop() request is made.  This wouldn't be true if we reused any
of the java.util.concurrent thread pool utilities.

This change doesn't actually fix the MyersDiff bug, so we're leaving
issue 487 open.  It does however reduce the impact by trying to abort
the runaway thread, and still show the file with intraline difference
support disabled on just that one file.

Bug: issue 487
Change-Id: I6cbfdd0acc6f7e612a29ed789efe9da591a45273
Signed-off-by: Shawn O. Pearce <sop@google.com>
diff --git a/Documentation/config-gerrit.txt b/Documentation/config-gerrit.txt
index 098bfff..2fbe58c 100644
--- a/Documentation/config-gerrit.txt
+++ b/Documentation/config-gerrit.txt
@@ -397,6 +397,35 @@
 [[cache_options]]Cache Options
 ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
 
+cache.diff_intraline.maxIdleWorkers::
++
+Number of idle worker threads to maintain for the intraline difference
+computations.  There is no upper bound on how many concurrent requests
+can occur at once, if additional threads are started to handle a peak
+load, only this many will remaining idle afterwards.
++
+Default is 1.5x number of available CPUs.
+
+cache.diff_intraline.timeout::
++
+Maximum number of milliseconds to wait for intraline difference data
+before giving up and disabling it for a particular file pair.  This is
+a work around for an infinite loop bug in the intraline difference
+implementation.  If computation takes longer than the timeout the
+worker thread is terminated and no intraline difference is displayed.
++
+Values should use common unit suffixes to express their setting:
++
+* ms, milliseconds
+* s, sec, second, seconds
+* m, min, minute, minutes
+* h, hr, hour, hours
+
++
+If a unit suffix is not specified, `milliseconds` is assumed.
++
+Default is 5 seconds.
+
 cache.diff_intraline.enabled::
 +
 Boolean to enable or disable the computation of intraline differences
diff --git a/gerrit-common/src/main/java/com/google/gerrit/common/data/PatchScript.java b/gerrit-common/src/main/java/com/google/gerrit/common/data/PatchScript.java
index e24405f..24cdee4 100644
--- a/gerrit-common/src/main/java/com/google/gerrit/common/data/PatchScript.java
+++ b/gerrit-common/src/main/java/com/google/gerrit/common/data/PatchScript.java
@@ -55,6 +55,7 @@
   protected List<Patch> history;
   protected boolean hugeFile;
   protected boolean intralineDifference;
+  protected boolean intralineFailure;
 
   public PatchScript(final Change.Key ck, final ChangeType ct, final String on,
       final String nn, final FileMode om, final FileMode nm,
@@ -62,7 +63,7 @@
       final SparseFileContent ca, final SparseFileContent cb,
       final List<Edit> e, final DisplayMethod ma, final DisplayMethod mb,
       final CommentDetail cd, final List<Patch> hist, final boolean hf,
-      final boolean id) {
+      final boolean id, final boolean idf) {
     changeId = ck;
     changeType = ct;
     oldName = on;
@@ -80,6 +81,7 @@
     history = hist;
     hugeFile = hf;
     intralineDifference = id;
+    intralineFailure = idf;
   }
 
   protected PatchScript() {
@@ -149,6 +151,10 @@
     return intralineDifference;
   }
 
+  public boolean hasIntralineFailure() {
+    return intralineFailure;
+  }
+
   public SparseFileContent getA() {
     return a;
   }
diff --git a/gerrit-gwtui/src/main/java/com/google/gerrit/client/patches/PatchConstants.java b/gerrit-gwtui/src/main/java/com/google/gerrit/client/patches/PatchConstants.java
index 93e8deb..dbbc9c3 100644
--- a/gerrit-gwtui/src/main/java/com/google/gerrit/client/patches/PatchConstants.java
+++ b/gerrit-gwtui/src/main/java/com/google/gerrit/client/patches/PatchConstants.java
@@ -31,6 +31,7 @@
 
   String patchHistoryTitle();
   String disabledOnLargeFiles();
+  String intralineFailure();
 
   String upToChange();
   String linePrev();
diff --git a/gerrit-gwtui/src/main/java/com/google/gerrit/client/patches/PatchConstants.properties b/gerrit-gwtui/src/main/java/com/google/gerrit/client/patches/PatchConstants.properties
index 90def1d..590007d 100644
--- a/gerrit-gwtui/src/main/java/com/google/gerrit/client/patches/PatchConstants.properties
+++ b/gerrit-gwtui/src/main/java/com/google/gerrit/client/patches/PatchConstants.properties
@@ -13,6 +13,7 @@
 patchHeaderNew = New Version
 patchHistoryTitle = Patch History
 disabledOnLargeFiles = Disabled on very large source files.
+intralineFailure = Intraline difference not available due to server error.
 
 upToChange = Up to change
 linePrev = Previous line
diff --git a/gerrit-gwtui/src/main/java/com/google/gerrit/client/patches/PatchScreen.java b/gerrit-gwtui/src/main/java/com/google/gerrit/client/patches/PatchScreen.java
index 812a74e..cbe037b 100644
--- a/gerrit-gwtui/src/main/java/com/google/gerrit/client/patches/PatchScreen.java
+++ b/gerrit-gwtui/src/main/java/com/google/gerrit/client/patches/PatchScreen.java
@@ -15,6 +15,7 @@
 package com.google.gerrit.client.patches;
 
 import com.google.gerrit.client.Dispatcher;
+import com.google.gerrit.client.ErrorDialog;
 import com.google.gerrit.client.Gerrit;
 import com.google.gerrit.client.RpcStatus;
 import com.google.gerrit.client.changes.CommitMessageBlock;
@@ -140,6 +141,7 @@
   /** Keys that cause an action on this screen */
   private KeyCommandSet keysNavigation;
   private HandlerRegistration regNavigation;
+  private boolean intralineFailure;
 
   /**
    * How this patch should be displayed in the patch screen.
@@ -461,6 +463,17 @@
       settingsPanel.getReviewedCheckBox().setValue(true);
       setReviewedByCurrentUser(true /* reviewed */);
     }
+
+    intralineFailure = isFirst && script.hasIntralineFailure();
+  }
+
+  @Override
+  public void onShowView() {
+    super.onShowView();
+    if (intralineFailure) {
+      intralineFailure = false;
+      new ErrorDialog(PatchUtil.C.intralineFailure()).show();
+    }
   }
 
   private void showPatch(final boolean showPatch) {
diff --git a/gerrit-httpd/src/main/java/com/google/gerrit/httpd/rpc/patch/PatchScriptBuilder.java b/gerrit-httpd/src/main/java/com/google/gerrit/httpd/rpc/patch/PatchScriptBuilder.java
index 2604d92..b7b3dd2 100644
--- a/gerrit-httpd/src/main/java/com/google/gerrit/httpd/rpc/patch/PatchScriptBuilder.java
+++ b/gerrit-httpd/src/main/java/com/google/gerrit/httpd/rpc/patch/PatchScriptBuilder.java
@@ -20,12 +20,14 @@
 import com.google.gerrit.prettify.common.EditList;
 import com.google.gerrit.prettify.common.SparseFileContent;
 import com.google.gerrit.reviewdb.AccountDiffPreference;
+import com.google.gerrit.reviewdb.AccountDiffPreference.Whitespace;
 import com.google.gerrit.reviewdb.Change;
 import com.google.gerrit.reviewdb.Patch;
 import com.google.gerrit.reviewdb.PatchLineComment;
-import com.google.gerrit.reviewdb.AccountDiffPreference.Whitespace;
+import com.google.gerrit.reviewdb.Project;
 import com.google.gerrit.server.FileTypeRegistry;
 import com.google.gerrit.server.patch.IntraLineDiff;
+import com.google.gerrit.server.patch.IntraLineDiffKey;
 import com.google.gerrit.server.patch.PatchListCache;
 import com.google.gerrit.server.patch.PatchListEntry;
 import com.google.gerrit.server.patch.Text;
@@ -65,6 +67,7 @@
   };
 
   private Repository db;
+  private Project.NameKey projectKey;
   private ObjectReader reader;
   private Change change;
   private AccountDiffPreference diffPrefs;
@@ -88,8 +91,9 @@
     patchListCache = plc;
   }
 
-  void setRepository(final Repository r) {
-    db = r;
+  void setRepository(Repository r, Project.NameKey projectKey) {
+    this.db = r;
+    this.projectKey = projectKey;
   }
 
   void setChange(final Change c) {
@@ -128,6 +132,7 @@
       final CommentDetail comments, final List<Patch> history)
       throws IOException {
     boolean intralineDifference = diffPrefs.isIntralineDifference();
+    boolean intralineFailure = false;
 
     a.path = oldName(content);
     b.path = newName(content);
@@ -137,17 +142,29 @@
 
     edits = new ArrayList<Edit>(content.getEdits());
 
-    if (intralineDifference) {
-      if (isModify(content)) {
-        IntraLineDiff d =
-            patchListCache.getIntraLineDiff(a.id, a.src, b.id, b.src, edits);
-        if (d != null) {
-          edits = new ArrayList<Edit>(d.getEdits());
-        } else {
-          intralineDifference = false;
+    if (intralineDifference && isModify(content)) {
+      IntraLineDiff d =
+          patchListCache.getIntraLineDiff(new IntraLineDiffKey(a.id, a.src,
+              b.id, b.src, edits, projectKey, bId, b.path));
+      if (d != null) {
+        switch (d.getStatus()) {
+          case EDIT_LIST:
+            edits = new ArrayList<Edit>(d.getEdits());
+            break;
+
+          case DISABLED:
+            intralineDifference = false;
+            break;
+
+          case ERROR:
+          case TIMEOUT:
+            intralineDifference = false;
+            intralineFailure = true;
+            break;
         }
       } else {
         intralineDifference = false;
+        intralineFailure = true;
       }
     }
 
@@ -188,10 +205,11 @@
       packContent(diffPrefs.getIgnoreWhitespace() != Whitespace.IGNORE_NONE);
     }
 
-    return new PatchScript(change.getKey(), content.getChangeType(), content
-        .getOldName(), content.getNewName(), a.fileMode, b.fileMode, content
-        .getHeaderLines(), diffPrefs, a.dst, b.dst, edits, a.displayMethod,
-        b.displayMethod, comments, history, hugeFile, intralineDifference);
+    return new PatchScript(change.getKey(), content.getChangeType(),
+        content.getOldName(), content.getNewName(), a.fileMode, b.fileMode,
+        content.getHeaderLines(), diffPrefs, a.dst, b.dst, edits,
+        a.displayMethod, b.displayMethod, comments, history, hugeFile,
+        intralineDifference, intralineFailure);
   }
 
   private static boolean isModify(PatchListEntry content) {
diff --git a/gerrit-httpd/src/main/java/com/google/gerrit/httpd/rpc/patch/PatchScriptFactory.java b/gerrit-httpd/src/main/java/com/google/gerrit/httpd/rpc/patch/PatchScriptFactory.java
index 377cf49..fcaa0c5 100644
--- a/gerrit-httpd/src/main/java/com/google/gerrit/httpd/rpc/patch/PatchScriptFactory.java
+++ b/gerrit-httpd/src/main/java/com/google/gerrit/httpd/rpc/patch/PatchScriptFactory.java
@@ -173,7 +173,7 @@
   private PatchScriptBuilder newBuilder(final PatchList list, Repository git) {
     final AccountDiffPreference dp = new AccountDiffPreference(diffPrefs);
     final PatchScriptBuilder b = builderFactory.get();
-    b.setRepository(git);
+    b.setRepository(git, projectKey);
     b.setChange(change);
     b.setDiffPrefs(dp);
     b.setTrees(list.isAgainstParent(), list.getOldId(), list.getNewId());
diff --git a/gerrit-server/src/main/java/com/google/gerrit/server/patch/IntraLineDiff.java b/gerrit-server/src/main/java/com/google/gerrit/server/patch/IntraLineDiff.java
index d694724..3805f8f 100644
--- a/gerrit-server/src/main/java/com/google/gerrit/server/patch/IntraLineDiff.java
+++ b/gerrit-server/src/main/java/com/google/gerrit/server/patch/IntraLineDiff.java
@@ -14,9 +14,13 @@
 
 package com.google.gerrit.server.patch;
 
+import static com.google.gerrit.server.ioutil.BasicSerialization.readEnum;
 import static com.google.gerrit.server.ioutil.BasicSerialization.readVarInt32;
+import static com.google.gerrit.server.ioutil.BasicSerialization.writeEnum;
 import static com.google.gerrit.server.ioutil.BasicSerialization.writeVarInt32;
 
+import com.google.gerrit.reviewdb.CodedEnum;
+
 import org.eclipse.jgit.diff.Edit;
 import org.eclipse.jgit.diff.ReplaceEdit;
 
@@ -33,17 +37,44 @@
 public class IntraLineDiff implements Serializable {
   static final long serialVersionUID = IntraLineDiffKey.serialVersionUID;
 
+  public static enum Status implements CodedEnum {
+    EDIT_LIST('e'), DISABLED('D'), TIMEOUT('T'), ERROR('E');
+
+    private final char code;
+
+    Status(char code) {
+      this.code = code;
+    }
+
+    @Override
+    public char getCode() {
+      return code;
+    }
+  }
+
+  private transient Status status;
   private transient List<Edit> edits;
 
+  IntraLineDiff(Status status) {
+    this.status = status;
+    this.edits = Collections.emptyList();
+  }
+
   IntraLineDiff(List<Edit> edits) {
+    this.status = Status.EDIT_LIST;
     this.edits = Collections.unmodifiableList(edits);
   }
 
+  public Status getStatus() {
+    return status;
+  }
+
   public List<Edit> getEdits() {
     return edits;
   }
 
   private void writeObject(final ObjectOutputStream out) throws IOException {
+    writeEnum(out, status);
     writeVarInt32(out, edits.size());
     for (Edit e : edits) {
       writeEdit(out, e);
@@ -61,6 +92,7 @@
   }
 
   private void readObject(final ObjectInputStream in) throws IOException {
+    status = readEnum(in, Status.values());
     int editCount = readVarInt32(in);
     Edit[] editArray = new Edit[editCount];
     for (int i = 0; i < editCount; i++) {
diff --git a/gerrit-server/src/main/java/com/google/gerrit/server/patch/IntraLineDiffKey.java b/gerrit-server/src/main/java/com/google/gerrit/server/patch/IntraLineDiffKey.java
index 61004dd..a8d62fc 100644
--- a/gerrit-server/src/main/java/com/google/gerrit/server/patch/IntraLineDiffKey.java
+++ b/gerrit-server/src/main/java/com/google/gerrit/server/patch/IntraLineDiffKey.java
@@ -17,6 +17,8 @@
 import static org.eclipse.jgit.lib.ObjectIdSerialization.readNotNull;
 import static org.eclipse.jgit.lib.ObjectIdSerialization.writeNotNull;
 
+import com.google.gerrit.reviewdb.Project;
+
 import org.eclipse.jgit.diff.Edit;
 import org.eclipse.jgit.lib.ObjectId;
 
@@ -27,7 +29,7 @@
 import java.util.List;
 
 public class IntraLineDiffKey implements Serializable {
-  static final long serialVersionUID = 2L;
+  static final long serialVersionUID = 3L;
 
   private transient ObjectId aId;
   private transient ObjectId bId;
@@ -38,14 +40,22 @@
   private transient Text bText;
   private transient List<Edit> edits;
 
-  IntraLineDiffKey(ObjectId aId, Text aText, ObjectId bId, Text bText,
-      List<Edit> edits) {
+  private transient Project.NameKey projectKey;
+  private transient ObjectId commit;
+  private transient String path;
+
+  public IntraLineDiffKey(ObjectId aId, Text aText, ObjectId bId, Text bText,
+      List<Edit> edits, Project.NameKey projectKey, ObjectId commit, String path) {
     this.aId = aId;
     this.bId = bId;
 
     this.aText = aText;
     this.bText = bText;
     this.edits = edits;
+
+    this.projectKey = projectKey;
+    this.commit = commit;
+    this.path = path;
   }
 
   Text getTextA() {
@@ -60,6 +70,26 @@
     return edits;
   }
 
+  ObjectId getBlobA() {
+    return aId;
+  }
+
+  ObjectId getBlobB() {
+    return bId;
+  }
+
+  Project.NameKey getProject() {
+    return projectKey;
+  }
+
+  ObjectId getCommit() {
+    return commit;
+  }
+
+  String getPath() {
+    return path;
+  }
+
   @Override
   public int hashCode() {
     int h = 0;
diff --git a/gerrit-server/src/main/java/com/google/gerrit/server/patch/IntraLineLoader.java b/gerrit-server/src/main/java/com/google/gerrit/server/patch/IntraLineLoader.java
index 0d3afde..0ac1af2 100644
--- a/gerrit-server/src/main/java/com/google/gerrit/server/patch/IntraLineLoader.java
+++ b/gerrit-server/src/main/java/com/google/gerrit/server/patch/IntraLineLoader.java
@@ -16,24 +16,200 @@
 package com.google.gerrit.server.patch;
 
 import com.google.gerrit.server.cache.EntryCreator;
+import com.google.gerrit.server.config.ConfigUtil;
+import com.google.gerrit.server.config.GerritServerConfig;
+import com.google.inject.Inject;
 
 import org.eclipse.jgit.diff.Edit;
 import org.eclipse.jgit.diff.MyersDiff;
 import org.eclipse.jgit.diff.ReplaceEdit;
+import org.eclipse.jgit.lib.Config;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.regex.Pattern;
 
 class IntraLineLoader extends EntryCreator<IntraLineDiffKey, IntraLineDiff> {
+  private static final Logger log = LoggerFactory
+      .getLogger(IntraLineLoader.class);
+
   private static final Pattern BLANK_LINE_RE = Pattern
       .compile("^[ \\t]*(|[{}]|/\\*\\*?|\\*)[ \\t]*$");
 
   private static final Pattern CONTROL_BLOCK_START_RE = Pattern
       .compile("[{:][ \\t]*$");
 
+  private final BlockingQueue<Worker> workerPool;
+  private final long timeoutMillis;
+
+  @Inject
+  IntraLineLoader(final @GerritServerConfig Config cfg) {
+    final int workers =
+        cfg.getInt("cache", PatchListCacheImpl.INTRA_NAME, "maxIdleWorkers",
+            Runtime.getRuntime().availableProcessors() * 3 / 2);
+    workerPool = new ArrayBlockingQueue<Worker>(workers, true /* fair */);
+
+    timeoutMillis =
+        ConfigUtil.getTimeUnit(cfg, "cache", PatchListCacheImpl.INTRA_NAME,
+            "timeout", TimeUnit.MILLISECONDS.convert(5, TimeUnit.SECONDS),
+            TimeUnit.MILLISECONDS);
+  }
+
   @Override
   public IntraLineDiff createEntry(IntraLineDiffKey key) throws Exception {
+    Worker w = workerPool.poll();
+    if (w == null) {
+      w = new Worker();
+    }
+
+    Worker.Result r = w.computeWithTimeout(key, timeoutMillis);
+
+    if (r == Worker.Result.TIMEOUT) {
+      // Don't keep this thread. We have to murder it unsafely, which
+      // means its unable to be reused in the future. Return back a
+      // null result, indicating the cache cannot load this key.
+      //
+      return new IntraLineDiff(IntraLineDiff.Status.TIMEOUT);
+    }
+
+    if (!workerPool.offer(w)) {
+      // If the idle worker pool is full, terminate this thread.
+      //
+      w.end();
+    }
+
+    if (r.error != null) {
+      // If there was an error computing the result, carry it
+      // up to the caller so the cache knows this key is invalid.
+      //
+      throw r.error;
+    }
+
+    return r.diff;
+  }
+
+  private static class Worker {
+    private static final AtomicInteger count = new AtomicInteger(1);
+
+    private final ArrayBlockingQueue<Input> input;
+    private final ArrayBlockingQueue<Result> result;
+    private final Thread thread;
+
+    Worker() {
+      input = new ArrayBlockingQueue<Input>(1);
+      result = new ArrayBlockingQueue<Result>(1);
+
+      thread = new Thread(new Runnable() {
+        public void run() {
+          workerLoop();
+        }
+      });
+      thread.setName("IntraLineDiff-" + count.getAndIncrement());
+      thread.setDaemon(true);
+      thread.start();
+    }
+
+    Result computeWithTimeout(IntraLineDiffKey key, long timeoutMillis)
+        throws Exception {
+      if (!input.offer(new Input(key))) {
+        log.error("Cannot enqueue task to thread " + thread.getName());
+        return null;
+      }
+
+      Result r = result.poll(timeoutMillis, TimeUnit.MILLISECONDS);
+      if (r != null) {
+        return r;
+      } else {
+        log.warn(timeoutMillis + " ms timeout reached for IntraLineDiff"
+            + " in project " + key.getProject().get() //
+            + " on commit " + key.getCommit().name() //
+            + " for path " + key.getPath() //
+            + " comparing " + key.getBlobA().name() //
+            + ".." + key.getBlobB().name() //
+            + ".  Killing " + thread.getName());
+        try {
+          thread.stop();
+        } catch (Throwable error) {
+          // Ignore any reason the thread won't stop.
+          log.error("Cannot stop runaway thread " + thread.getName(), error);
+        }
+        return Result.TIMEOUT;
+      }
+    }
+
+    void end() {
+      if (!input.offer(Input.END_THREAD)) {
+        log.error("Cannot gracefully stop thread " + thread.getName());
+      }
+    }
+
+    private void workerLoop() {
+      try {
+        for (;;) {
+          Input in;
+          try {
+            in = input.take();
+          } catch (InterruptedException e) {
+            log.error("Unexpected interrupt on " + thread.getName());
+            continue;
+          }
+
+          if (in == Input.END_THREAD) {
+            return;
+          }
+
+          Result r;
+          try {
+            r = new Result(IntraLineLoader.compute(in.key));
+          } catch (Exception error) {
+            r = new Result(error);
+          }
+
+          if (!result.offer(r)) {
+            log.error("Cannot return result from " + thread.getName());
+          }
+        }
+      } catch (ThreadDeath iHaveBeenShot) {
+        // Handle thread death by gracefully returning to the caller,
+        // allowing the thread to be destroyed.
+      }
+    }
+
+    private static class Input {
+      static final Input END_THREAD = new Input(null);
+
+      final IntraLineDiffKey key;
+
+      Input(IntraLineDiffKey key) {
+        this.key = key;
+      }
+    }
+
+    static class Result {
+      static final Result TIMEOUT = new Result((IntraLineDiff) null);
+
+      final IntraLineDiff diff;
+      final Exception error;
+
+      Result(IntraLineDiff diff) {
+        this.diff = diff;
+        this.error = null;
+      }
+
+      Result(Exception error) {
+        this.diff = null;
+        this.error = error;
+      }
+    }
+  }
+
+  private static IntraLineDiff compute(IntraLineDiffKey key) throws Exception {
     List<Edit> edits = new ArrayList<Edit>(key.getEdits());
     Text aContent = key.getTextA();
     Text bContent = key.getTextB();
diff --git a/gerrit-server/src/main/java/com/google/gerrit/server/patch/PatchListCache.java b/gerrit-server/src/main/java/com/google/gerrit/server/patch/PatchListCache.java
index c7bb08b..a7cf10a 100644
--- a/gerrit-server/src/main/java/com/google/gerrit/server/patch/PatchListCache.java
+++ b/gerrit-server/src/main/java/com/google/gerrit/server/patch/PatchListCache.java
@@ -17,17 +17,11 @@
 import com.google.gerrit.reviewdb.Change;
 import com.google.gerrit.reviewdb.PatchSet;
 
-import org.eclipse.jgit.diff.Edit;
-import org.eclipse.jgit.lib.ObjectId;
-
-import java.util.List;
-
 /** Provides a cached list of {@link PatchListEntry}. */
 public interface PatchListCache {
   public PatchList get(PatchListKey key);
 
   public PatchList get(Change change, PatchSet patchSet);
 
-  public IntraLineDiff getIntraLineDiff(ObjectId aId, Text aText, ObjectId bId,
-      Text bText, List<Edit> edits);
+  public IntraLineDiff getIntraLineDiff(IntraLineDiffKey key);
 }
diff --git a/gerrit-server/src/main/java/com/google/gerrit/server/patch/PatchListCacheImpl.java b/gerrit-server/src/main/java/com/google/gerrit/server/patch/PatchListCacheImpl.java
index 285420c..e1a0a40 100644
--- a/gerrit-server/src/main/java/com/google/gerrit/server/patch/PatchListCacheImpl.java
+++ b/gerrit-server/src/main/java/com/google/gerrit/server/patch/PatchListCacheImpl.java
@@ -30,17 +30,14 @@
 import com.google.inject.TypeLiteral;
 import com.google.inject.name.Named;
 
-import org.eclipse.jgit.diff.Edit;
 import org.eclipse.jgit.lib.Config;
 import org.eclipse.jgit.lib.ObjectId;
 
-import java.util.List;
-
 /** Provides a cached list of {@link PatchListEntry}. */
 @Singleton
 public class PatchListCacheImpl implements PatchListCache {
   private static final String FILE_NAME = "diff";
-  private static final String INTRA_NAME = "diff_intraline";
+  static final String INTRA_NAME = "diff_intraline";
 
   public static Module module() {
     return new CacheModule() {
@@ -98,14 +95,15 @@
   }
 
   @Override
-  public IntraLineDiff getIntraLineDiff(ObjectId aId, Text aText, ObjectId bId,
-      Text bText, List<Edit> edits) {
+  public IntraLineDiff getIntraLineDiff(IntraLineDiffKey key) {
     if (computeIntraline) {
-      IntraLineDiffKey key =
-          new IntraLineDiffKey(aId, aText, bId, bText, edits);
-      return intraCache.get(key);
+      IntraLineDiff d = intraCache.get(key);
+      if (d == null) {
+        d = new IntraLineDiff(IntraLineDiff.Status.ERROR);
+      }
+      return d;
     } else {
-      return null;
+      return new IntraLineDiff(IntraLineDiff.Status.DISABLED);
     }
   }
 }