You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2014/07/29 18:21:47 UTC

svn commit: r1614402 - in /lucene/dev/branches/branch_4x: ./ lucene/ lucene/core/ lucene/core/src/java/org/apache/lucene/index/ lucene/core/src/test/org/apache/lucene/index/ lucene/test-framework/ lucene/test-framework/src/java/org/apache/lucene/util/

Author: mikemccand
Date: Tue Jul 29 16:21:47 2014
New Revision: 1614402

URL: http://svn.apache.org/r1614402
Log:
LUCENE-5843: IndexWriter now enforces max docs in one index

Added:
    lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java
      - copied, changed from r1614388, lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java
Removed:
    lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/Test2BDocs.java
Modified:
    lucene/dev/branches/branch_4x/   (props changed)
    lucene/dev/branches/branch_4x/lucene/   (props changed)
    lucene/dev/branches/branch_4x/lucene/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_4x/lucene/core/   (props changed)
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/BaseCompositeReader.java
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java
    lucene/dev/branches/branch_4x/lucene/test-framework/   (props changed)
    lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java

Modified: lucene/dev/branches/branch_4x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/CHANGES.txt?rev=1614402&r1=1614401&r2=1614402&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_4x/lucene/CHANGES.txt Tue Jul 29 16:21:47 2014
@@ -46,6 +46,12 @@ New Features
   checksum against all the bytes), retrieve checksum to validate structure
   of footer, this can detect some forms of corruption such as truncation.
   (Robert Muir)
+
+* LUCENE-5843: Added IndexWriter.MAX_DOCS which is the maximum number
+  of documents allowed in a single index, and any operations that add
+  documents will now throw IllegalStateException if the max count
+  would be exceeded, instead of silently creating an unusable
+  index.  (Mike McCandless)
   
 API Changes
 

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/BaseCompositeReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/BaseCompositeReader.java?rev=1614402&r1=1614401&r2=1614402&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/BaseCompositeReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/BaseCompositeReader.java Tue Jul 29 16:21:47 2014
@@ -73,8 +73,8 @@ public abstract class BaseCompositeReade
       starts[i] = maxDoc;
       final IndexReader r = subReaders[i];
       maxDoc += r.maxDoc();      // compute maxDocs
-      if (maxDoc < 0 /* overflow */) {
-        throw new IllegalArgumentException("Too many documents, composite IndexReaders cannot exceed " + Integer.MAX_VALUE);
+      if (maxDoc < 0 /* overflow */ || maxDoc > IndexWriter.getActualMaxDocs()) {
+        throw new IllegalArgumentException("Too many documents, composite IndexReaders cannot exceed " + IndexWriter.getActualMaxDocs());
       }
       numDocs += r.numDocs();    // compute numDocs
       r.registerParentReader(this);

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java?rev=1614402&r1=1614401&r2=1614402&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java Tue Jul 29 16:21:47 2014
@@ -388,7 +388,8 @@ final class DocumentsWriter implements C
       final FieldInfos.Builder infos = new FieldInfos.Builder(
           writer.globalFieldNumberMap);
       state.dwpt = new DocumentsWriterPerThread(writer.newSegmentName(),
-          directory, config, infoStream, deleteQueue, infos);
+                                                directory, config, infoStream, deleteQueue, infos,
+                                                writer.pendingNumDocs);
     }
   }
 

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java?rev=1614402&r1=1614401&r2=1614402&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java Tue Jul 29 16:21:47 2014
@@ -17,14 +17,12 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
-import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_MASK;
-import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
-
 import java.io.IOException;
 import java.text.NumberFormat;
 import java.util.HashSet;
 import java.util.Locale;
 import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.codecs.Codec;
@@ -43,6 +41,9 @@ import org.apache.lucene.util.IntBlockPo
 import org.apache.lucene.util.MutableBits;
 import org.apache.lucene.util.RamUsageEstimator;
 
+import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_MASK;
+import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
+
 class DocumentsWriterPerThread {
 
   /**
@@ -154,11 +155,11 @@ class DocumentsWriterPerThread {
   private final NumberFormat nf = NumberFormat.getInstance(Locale.ROOT);
   final Allocator byteBlockAllocator;
   final IntBlockPool.Allocator intBlockAllocator;
+  private final AtomicLong pendingNumDocs;
   private final LiveIndexWriterConfig indexWriterConfig;
-
   
   public DocumentsWriterPerThread(String segmentName, Directory directory, LiveIndexWriterConfig indexWriterConfig, InfoStream infoStream, DocumentsWriterDeleteQueue deleteQueue,
-      FieldInfos.Builder fieldInfos) throws IOException {
+                                  FieldInfos.Builder fieldInfos, AtomicLong pendingNumDocs) throws IOException {
     this.directoryOrig = directory;
     this.directory = new TrackingDirectoryWrapper(directory);
     this.fieldInfos = fieldInfos;
@@ -167,6 +168,7 @@ class DocumentsWriterPerThread {
     this.codec = indexWriterConfig.getCodec();
     this.docState = new DocState(this, infoStream);
     this.docState.similarity = indexWriterConfig.getSimilarity();
+    this.pendingNumDocs = pendingNumDocs;
     bytesUsed = Counter.newCounter();
     byteBlockAllocator = new DirectTrackingAllocator(bytesUsed);
     pendingUpdates = new BufferedUpdates();
@@ -207,6 +209,16 @@ class DocumentsWriterPerThread {
     return true;
   }
 
+  /** Anything that will add N docs to the index should reserve first to
+   *  make sure it's allowed. */
+  private void reserveDoc() {
+    if (pendingNumDocs.incrementAndGet() > IndexWriter.getActualMaxDocs()) {
+      // Reserve failed
+      pendingNumDocs.decrementAndGet();
+      throw new IllegalStateException("number of documents in the index cannot exceed " + IndexWriter.getActualMaxDocs());
+    }
+  }
+
   public void updateDocument(Iterable<? extends IndexableField> doc, Analyzer analyzer, Term delTerm) throws IOException {
     assert testPoint("DocumentsWriterPerThread addDocument start");
     assert deleteQueue != null;
@@ -216,6 +228,13 @@ class DocumentsWriterPerThread {
     if (INFO_VERBOSE && infoStream.isEnabled("DWPT")) {
       infoStream.message("DWPT", Thread.currentThread().getName() + " update delTerm=" + delTerm + " docID=" + docState.docID + " seg=" + segmentInfo.name);
     }
+    // Even on exception, the document is still added (but marked
+    // deleted), so we don't need to un-reserve at that point.
+    // Aborting exceptions will actually "lose" more than one
+    // document, so the counter will be "wrong" in that case, but
+    // it's very hard to fix (we can't easily distinguish aborting
+    // vs non-aborting exceptions):
+    reserveDoc();
     boolean success = false;
     try {
       try {
@@ -249,6 +268,13 @@ class DocumentsWriterPerThread {
     boolean allDocsIndexed = false;
     try {
       for(Iterable<? extends IndexableField> doc : docs) {
+        // Even on exception, the document is still added (but marked
+        // deleted), so we don't need to un-reserve at that point.
+        // Aborting exceptions will actually "lose" more than one
+        // document, so the counter will be "wrong" in that case, but
+        // it's very hard to fix (we can't easily distinguish aborting
+        // vs non-aborting exceptions):
+        reserveDoc();
         docState.doc = doc;
         docState.docID = numDocsInRAM;
         docCount++;

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java?rev=1614402&r1=1614401&r2=1614402&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java Tue Jul 29 16:21:47 2014
@@ -32,11 +32,12 @@ import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Locale;
-import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Map;
 import java.util.Queue;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.codecs.Codec;
@@ -195,7 +196,31 @@ import org.apache.lucene.util.ThreadInte
  * keeps track of the last non commit checkpoint.
  */
 public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
-  
+
+  /** Hard limit on maximum number of documents that may be added to the
+   *  index.  If you try to add more than this you'll hit {@code IllegalStateException}. */
+  // We defensively subtract 128 to be well below the lowest
+  // ArrayUtil.MAX_ARRAY_LENGTH on "typical" JVMs.  We don't just use
+  // ArrayUtil.MAX_ARRAY_LENGTH here because this can vary across JVMs:
+  public static final int MAX_DOCS = Integer.MAX_VALUE - 128;
+
+  // Use package-private instance var to enforce the limit so testing
+  // can use less electricity:
+  private static int actualMaxDocs = MAX_DOCS;
+
+  /** Used only for testing. */
+  static void setMaxDocs(int maxDocs) {
+    if (maxDocs > MAX_DOCS) {
+      // Cannot go higher than the hard max:
+      throw new IllegalArgumentException("maxDocs must be <= IndexWriter.MAX_DOCS=" + MAX_DOCS + "; got: " + maxDocs);
+    }
+    IndexWriter.actualMaxDocs = maxDocs;
+  }
+
+  static int getActualMaxDocs() {
+    return IndexWriter.actualMaxDocs;
+  }
+
   private static final int UNBOUNDED_MAX_MERGE_SEGMENTS = -1;
   
   /**
@@ -290,6 +315,13 @@ public class IndexWriter implements Clos
    *  an infoStream message about how long commit took. */
   private long startCommitTime;
 
+  /** How many documents are in the index, or are in the process of being
+   *  added (reserved).  E.g., operations like addIndexes will first reserve
+   *  the right to add N docs, before they actually change the index,
+   *  much like how hotels place an "authorization hold" on your credit
+   *  card to make sure they can later charge you when you check out. */
+  final AtomicLong pendingNumDocs = new AtomicLong();
+
   DirectoryReader getReader() throws IOException {
     return getReader(true);
   }
@@ -2538,6 +2570,9 @@ public class IndexWriter implements Clos
       flush(false, true);
 
       List<SegmentCommitInfo> infos = new ArrayList<>();
+
+      int totalDocCount = 0;
+
       boolean success = false;
       try {
         for (Directory dir : dirs) {
@@ -2546,9 +2581,13 @@ public class IndexWriter implements Clos
           }
           SegmentInfos sis = new SegmentInfos(); // read infos from dir
           sis.read(dir);
+
           final Set<String> dsFilesCopied = new HashSet<>();
           final Map<String, String> dsNames = new HashMap<>();
           final Set<String> copiedFiles = new HashSet<>();
+
+          totalDocCount += sis.totalDocCount();
+
           for (SegmentCommitInfo info : sis) {
             assert !infos.contains(info): "dup info dir=" + info.info.dir + " name=" + info.info.name;
 
@@ -2584,6 +2623,9 @@ public class IndexWriter implements Clos
         success = false;
         try {
           ensureOpen();
+          // Make sure adding the new documents to this index won't
+          // exceed the limit:
+          reserveDocs(totalDocCount);
           success = true;
         } finally {
           if (!success) {
@@ -2671,6 +2713,10 @@ public class IndexWriter implements Clos
           mergeReaders.add(ctx.reader());
         }
       }
+
+      // Make sure adding the new documents to this index won't
+      // exceed the limit:
+      reserveDocs(numDocs);
       
       final IOContext context = new IOContext(new MergeInfo(numDocs, -1, true, -1));
 
@@ -3280,6 +3326,7 @@ public class IndexWriter implements Clos
         // it once it's done:
         if (!mergingSegments.contains(info)) {
           segmentInfos.remove(info);
+          pendingNumDocs.addAndGet(-info.info.getDocCount());
           readerPool.drop(info);
         }
       }
@@ -3661,6 +3708,12 @@ public class IndexWriter implements Clos
     // merge:
     segmentInfos.applyMergeChanges(merge, dropSegment);
 
+    // Now deduct the deleted docs that we just reclaimed from this
+    // merge:
+    int delDocCount = merge.totalDocCount - merge.info.info.getDocCount();
+    assert delDocCount >= 0;
+    pendingNumDocs.addAndGet(-delDocCount);
+
     if (dropSegment) {
       assert !segmentInfos.contains(merge.info);
       readerPool.drop(merge.info);
@@ -3944,6 +3997,7 @@ public class IndexWriter implements Clos
       }
       for(SegmentCommitInfo info : result.allDeleted) {
         segmentInfos.remove(info);
+        pendingNumDocs.addAndGet(-info.info.getDocCount());
         if (merge.segments.contains(info)) {
           mergingSegments.remove(info);
           merge.segments.remove(info);
@@ -4815,4 +4869,15 @@ public class IndexWriter implements Clos
       return false;
     }
   }
+
+  /** Anything that will add N docs to the index should reserve first to
+   *  make sure it's allowed.  This will throw {@code
+   *  IllegalStateException} if it's not allowed. */ 
+  private void reserveDocs(int numDocs) {
+    if (pendingNumDocs.addAndGet(numDocs) > actualMaxDocs) {
+      // Reserve failed
+      pendingNumDocs.addAndGet(-numDocs);
+      throw new IllegalStateException("number of documents in the index cannot exceed " + actualMaxDocs);
+    }
+  }
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java?rev=1614402&r1=1614401&r2=1614402&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java Tue Jul 29 16:21:47 2014
@@ -20,8 +20,6 @@ package org.apache.lucene.index;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.MergeInfo;
 import org.apache.lucene.util.FixedBitSet;
-import org.apache.lucene.util.SetOnce;
-import org.apache.lucene.util.SetOnce.AlreadySetException;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -92,7 +90,7 @@ public abstract class MergePolicy implem
 
   public static class OneMerge {
 
-    SegmentCommitInfo info;      // used by IndexWriter
+    SegmentCommitInfo info;         // used by IndexWriter
     boolean registerDone;           // used by IndexWriter
     long mergeGen;                  // used by IndexWriter
     boolean isExternal;             // used by IndexWriter
@@ -109,7 +107,7 @@ public abstract class MergePolicy implem
     /** Segments to be merged. */
     public final List<SegmentCommitInfo> segments;
 
-    /** Number of documents in the merged segment. */
+    /** Total number of documents in segments to be merged, not accounting for deletions. */
     public final int totalDocCount;
     boolean aborted;
     Throwable error;

Copied: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java (from r1614388, lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java?p2=lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java&p1=lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java&r1=1614388&r2=1614402&rev=1614402&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java Tue Jul 29 16:21:47 2014
@@ -73,7 +73,7 @@ public class TestIndexWriterMaxDocs exte
       iw.forceMerge(1);
     }
 
-    iw.shutdown();
+    iw.close();
     dir.close();
   }
 
@@ -262,7 +262,7 @@ public class TestIndexWriterMaxDocs exte
       for(int i=0;i<10;i++) {
         w.addDocument(new Document());
       }
-      w.shutdown();
+      w.close();
 
       Directory dir2 = newDirectory();
       IndexWriter w2 = new IndexWriter(dir2, new IndexWriterConfig(TEST_VERSION_CURRENT, null));
@@ -298,7 +298,7 @@ public class TestIndexWriterMaxDocs exte
     for (int i = 0; i < 100000; i++) {
       w.addDocument(doc);
     }
-    w.shutdown();
+    w.close();
 
     int remainder = IndexWriter.MAX_DOCS % 100000;
     Directory dir2 = newDirectory();
@@ -306,7 +306,7 @@ public class TestIndexWriterMaxDocs exte
     for (int i = 0; i < remainder; i++) {
       w.addDocument(doc);
     }
-    w.shutdown();
+    w.close();
 
     int copies = IndexWriter.MAX_DOCS / 100000;
 
@@ -333,7 +333,7 @@ public class TestIndexWriterMaxDocs exte
     for (int i = 0; i < 100000; i++) {
       w.addDocument(doc);
     }
-    w.shutdown();
+    w.close();
 
     int remainder = IndexWriter.MAX_DOCS % 100000;
 
@@ -345,7 +345,7 @@ public class TestIndexWriterMaxDocs exte
     for (int i = 0; i < remainder; i++) {
       w.addDocument(doc);
     }
-    w.shutdown();
+    w.close();
 
     int copies = IndexWriter.MAX_DOCS / 100000;
 

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java?rev=1614402&r1=1614401&r2=1614402&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java Tue Jul 29 16:21:47 2014
@@ -29,6 +29,7 @@ import java.lang.annotation.Retention;
 import java.lang.annotation.RetentionPolicy;
 import java.lang.annotation.Target;
 import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.nio.file.NoSuchFileException;
 import java.util.ArrayList;
@@ -77,6 +78,7 @@ import org.apache.lucene.index.FieldInfo
 import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.IndexReader.ReaderClosedListener;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.LiveIndexWriterConfig;
@@ -669,8 +671,34 @@ public abstract class LuceneTestCase ext
   public void tearDown() throws Exception {
     parentChainCallRule.teardownCalled = true;
     fieldToType.clear();
+
+    // Test is supposed to call this itself, but we do this defensively in case it forgot:
+    restoreIndexWriterMaxDocs();
+  }
+
+  /** Tells {@link IndexWriter} to enforce the specified limit as the maximum number of documents in one index; call
+   *  {@link #restoreIndexWriterMaxDocs} once your test is done. */
+  public void setIndexWriterMaxDocs(int limit) {
+    Method m;
+    try {
+      m = IndexWriter.class.getDeclaredMethod("setMaxDocs", int.class);
+    } catch (NoSuchMethodException nsme) {
+      throw new RuntimeException(nsme);
+    }
+    m.setAccessible(true);
+    try {
+      m.invoke(IndexWriter.class, limit);
+    } catch (IllegalAccessException iae) {
+      throw new RuntimeException(iae);
+    } catch (InvocationTargetException ite) {
+      throw new RuntimeException(ite);
+    }
   }
 
+  /** Returns the default {@link IndexWriter#MAX_DOCS} limit. */
+  public void restoreIndexWriterMaxDocs() {
+    setIndexWriterMaxDocs(IndexWriter.MAX_DOCS);
+  }
 
   // -----------------------------------------------------------------
   // Test facilities and facades for subclasses. 
@@ -2325,8 +2353,6 @@ public abstract class LuceneTestCase ext
           // numOrds
           assertEquals(info, leftValues.getValueCount(), rightValues.getValueCount());
           // ords
-          BytesRef scratchLeft = new BytesRef();
-          BytesRef scratchRight = new BytesRef();
           for (int i = 0; i < leftValues.getValueCount(); i++) {
             final BytesRef left = BytesRef.deepCopyOf(leftValues.lookupOrd(i));
             final BytesRef right = rightValues.lookupOrd(i);