You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jp...@apache.org on 2018/02/20 16:14:19 UTC

lucene-solr:master: LUCENE-8153: Make impacts checks lighter by default.

Repository: lucene-solr
Updated Branches:
  refs/heads/master 291248c75 -> 317a2e0c3


LUCENE-8153: Make impacts checks lighter by default.

The new `-slow` switch makes checks more complete but also more heavy. This
option also cross-checks term vectors.


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/317a2e0c
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/317a2e0c
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/317a2e0c

Branch: refs/heads/master
Commit: 317a2e0c3d16b9f8ea6ed1b1e4697c5cec51d05c
Parents: 291248c
Author: Adrien Grand <jp...@gmail.com>
Authored: Tue Feb 20 15:55:58 2018 +0100
Committer: Adrien Grand <jp...@gmail.com>
Committed: Tue Feb 20 17:14:11 2018 +0100

----------------------------------------------------------------------
 .../org/apache/lucene/index/CheckIndex.java     | 244 ++++++++++---------
 .../apache/lucene/index/TestIndexWriter.java    |   2 +-
 .../lucene/store/BaseDirectoryWrapper.java      |   8 +-
 .../java/org/apache/lucene/util/TestUtil.java   |  14 +-
 4 files changed, 139 insertions(+), 129 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/317a2e0c/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
index 7dd1aa9..54a227c 100644
--- a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
+++ b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
@@ -429,18 +429,17 @@ public final class CheckIndex implements Closeable {
     IOUtils.close(writeLock);
   }
 
-  private boolean crossCheckTermVectors;
+  private boolean doSlowChecks;
 
-  /** If true, term vectors are compared against postings to
-   *  make sure they are the same.  This will likely
+  /** If true, additional slow checks are performed.  This will likely
    *  drastically increase time it takes to run CheckIndex! */
-  public void setCrossCheckTermVectors(boolean v) {
-    crossCheckTermVectors = v;
+  public void setDoSlowChecks(boolean v) {
+    doSlowChecks = v;
   }
 
-  /** See {@link #setCrossCheckTermVectors}. */
-  public boolean getCrossCheckTermVectors() {
-    return crossCheckTermVectors;
+  /** See {@link #setDoSlowChecks}. */
+  public boolean doSlowChecks() {
+    return doSlowChecks;
   }
 
   private boolean failFast;
@@ -745,13 +744,13 @@ public final class CheckIndex implements Closeable {
           segInfoStat.fieldNormStatus = testFieldNorms(reader, infoStream, failFast);
 
           // Test the Term Index
-          segInfoStat.termIndexStatus = testPostings(reader, infoStream, verbose, failFast);
+          segInfoStat.termIndexStatus = testPostings(reader, infoStream, verbose, doSlowChecks, failFast);
 
           // Test Stored Fields
           segInfoStat.storedFieldStatus = testStoredFields(reader, infoStream, failFast);
 
           // Test Term Vectors
-          segInfoStat.termVectorStatus = testTermVectors(reader, infoStream, verbose, crossCheckTermVectors, failFast);
+          segInfoStat.termVectorStatus = testTermVectors(reader, infoStream, verbose, doSlowChecks, failFast);
 
           // Test Docvalues
           segInfoStat.docValuesStatus = testDocValues(reader, infoStream, failFast);
@@ -1210,7 +1209,7 @@ public final class CheckIndex implements Closeable {
    * checks Fields api is consistent with itself.
    * searcher is optional, to verify with queries. Can be null.
    */
-  private static Status.TermIndexStatus checkFields(Fields fields, Bits liveDocs, int maxDoc, FieldInfos fieldInfos, boolean doPrint, boolean isVectors, PrintStream infoStream, boolean verbose) throws IOException {
+  private static Status.TermIndexStatus checkFields(Fields fields, Bits liveDocs, int maxDoc, FieldInfos fieldInfos, boolean doPrint, boolean isVectors, PrintStream infoStream, boolean verbose, boolean doSlowChecks) throws IOException {
     // TODO: we should probably return our own stats thing...?!
     long startNS;
     if (doPrint) {
@@ -1600,104 +1599,112 @@ public final class CheckIndex implements Closeable {
           }
         }
 
-        // Test score blocks
-        // We only score on freq to keep things simple and not pull norms
-        SimScorer scorer = new SimScorer(field) {
-          @Override
-          public float score(float freq, long norm) {
-            return freq;
-          }
-        };
-
-        // First check max scores and block uptos
-        int max = -1;
-        float maxScore = 0;
-        ImpactsEnum impacts = termsEnum.impacts(scorer, PostingsEnum.FREQS);
-        postings = termsEnum.postings(postings, PostingsEnum.FREQS);
-        for (int doc = impacts.nextDoc(); ; doc = impacts.nextDoc()) {
-          if (postings.nextDoc() != doc) {
-            throw new RuntimeException("Wrong next doc: " + doc + ", expected " + postings.docID());
-          }
-          if (doc == DocIdSetIterator.NO_MORE_DOCS) {
-            break;
-          }
-          if (postings.freq() != impacts.freq()) {
-            throw new RuntimeException("Wrong freq, expected " + postings.freq() + ", but got " + impacts.freq());
-          }
-          if (doc > max) {
-            max = impacts.advanceShallow(doc);
-            if (max < doc) {
-              throw new RuntimeException("max block doc id " + max + " must be greater than the target: " + doc);
+        // Checking score blocks is heavy, we only do it on long postings lists, on every 1024th term
+        // or if slow checks are enabled.
+        if (doSlowChecks || docFreq > 1024 || (status.termCount + status.delTermCount) % 1024 == 0) {
+          // Test score blocks
+          // We only score on freq to keep things simple and not pull norms
+          SimScorer scorer = new SimScorer(field) {
+            @Override
+            public float score(float freq, long norm) {
+              return freq;
+            }
+          };
+
+          // First check max scores and block uptos
+          // But only if slok checks are enabled since we visit all docs
+          if (doSlowChecks) {
+            int max = -1;
+            float maxScore = 0;
+            ImpactsEnum impacts = termsEnum.impacts(scorer, PostingsEnum.FREQS);
+            postings = termsEnum.postings(postings, PostingsEnum.FREQS);
+            for (int doc = impacts.nextDoc(); ; doc = impacts.nextDoc()) {
+              if (postings.nextDoc() != doc) {
+                throw new RuntimeException("Wrong next doc: " + doc + ", expected " + postings.docID());
+              }
+              if (doc == DocIdSetIterator.NO_MORE_DOCS) {
+                break;
+              }
+              if (postings.freq() != impacts.freq()) {
+                throw new RuntimeException("Wrong freq, expected " + postings.freq() + ", but got " + impacts.freq());
+              }
+              if (doc > max) {
+                max = impacts.advanceShallow(doc);
+                if (max < doc) {
+                  throw new RuntimeException("max block doc id " + max + " must be greater than the target: " + doc);
+                }
+                maxScore = impacts.getMaxScore(max);
+              }
+              int max2 = impacts.advanceShallow(doc);
+              if (max != max2) {
+                throw new RuntimeException("max is not stable, initially had " + max + " but now " + max2);
+              }
+              float score = scorer.score(impacts.freq(), 1);
+              if (score > maxScore) {
+                throw new RuntimeException("score " + score + " is greater than the max score " + maxScore);
+              }
             }
-            maxScore = impacts.getMaxScore(max);
-          }
-          int max2 = impacts.advanceShallow(doc);
-          if (max != max2) {
-            throw new RuntimeException("max is not stable, initially had " + max + " but now " + max2);
-          }
-          float score = scorer.score(impacts.freq(), 1);
-          if (score > maxScore) {
-            throw new RuntimeException("score " + score + " is greater than the max score " + maxScore);
-          }
-        }
-
-        // Now check advancing
-        impacts = termsEnum.impacts(scorer, PostingsEnum.FREQS);
-        postings = termsEnum.postings(postings, PostingsEnum.FREQS);
-
-        max = -1;
-        while (true) {
-          int doc = impacts.docID();
-          boolean advance;
-          int target;
-          if (((field.hashCode() + doc) & 1) == 1) {
-            advance = false;
-            target = doc + 1;
-          } else {
-            advance = true;
-            int delta = Math.min(1 + ((31 * field.hashCode() + doc) & 0x1ff), DocIdSetIterator.NO_MORE_DOCS - doc);
-            target = impacts.docID() + delta;
           }
 
-          if (target > max && target % 2 == 1) {
-            int delta = Math.min((31 * field.hashCode() + target) & 0x1ff, DocIdSetIterator.NO_MORE_DOCS - target);
-            max = target + delta;
-            int m = impacts.advanceShallow(target);
-            if (m < target) {
-              throw new RuntimeException("Block max doc: " + m + " is less than the target " + target);
+          // Now check advancing
+          ImpactsEnum impacts = termsEnum.impacts(scorer, PostingsEnum.FREQS);
+          postings = termsEnum.postings(postings, PostingsEnum.FREQS);
+
+          int max = -1;
+          float maxScore = 0;
+          while (true) {
+            int doc = impacts.docID();
+            boolean advance;
+            int target;
+            if (((field.hashCode() + doc) & 1) == 1) {
+              advance = false;
+              target = doc + 1;
+            } else {
+              advance = true;
+              int delta = Math.min(1 + ((31 * field.hashCode() + doc) & 0x1ff), DocIdSetIterator.NO_MORE_DOCS - doc);
+              target = impacts.docID() + delta;
             }
-            maxScore = impacts.getMaxScore(max);
-          }
 
-          if (advance) {
-            doc = impacts.advance(target);
-          } else {
-            doc = impacts.nextDoc();
-          }
+            if (target > max && target % 2 == 1) {
+              int delta = Math.min((31 * field.hashCode() + target) & 0x1ff, DocIdSetIterator.NO_MORE_DOCS - target);
+              max = target + delta;
+              int m = impacts.advanceShallow(target);
+              if (m < target) {
+                throw new RuntimeException("Block max doc: " + m + " is less than the target " + target);
+              }
+              maxScore = impacts.getMaxScore(max);
+            }
 
-          if (postings.advance(target) != doc) {
-            throw new RuntimeException("Impacts do not advance to the same document as postings for target " + target + ", postings: " + postings.docID() + ", impacts: " + doc);
-          }
-          if (doc == DocIdSetIterator.NO_MORE_DOCS) {
-            break;
-          }
-          if (postings.freq() != impacts.freq()) {
-            throw new RuntimeException("Wrong freq, expected " + postings.freq() + ", but got " + impacts.freq());
-          }
+            if (advance) {
+              doc = impacts.advance(target);
+            } else {
+              doc = impacts.nextDoc();
+            }
 
-          if (doc >= max) {
-            int delta = Math.min((31 * field.hashCode() + target & 0x1ff), DocIdSetIterator.NO_MORE_DOCS - doc);
-            max = doc + delta;
-            int m = impacts.advanceShallow(doc);
-            if (m < doc) {
-              throw new RuntimeException("Block max doc: " + m + " is less than the target " + doc);
+            if (postings.advance(target) != doc) {
+              throw new RuntimeException("Impacts do not advance to the same document as postings for target " + target + ", postings: " + postings.docID() + ", impacts: " + doc);
+            }
+            if (doc == DocIdSetIterator.NO_MORE_DOCS) {
+              break;
+            }
+            if (postings.freq() != impacts.freq()) {
+              throw new RuntimeException("Wrong freq, expected " + postings.freq() + ", but got " + impacts.freq());
+            }
+  
+            if (doc >= max) {
+              int delta = Math.min((31 * field.hashCode() + target & 0x1ff), DocIdSetIterator.NO_MORE_DOCS - doc);
+              max = doc + delta;
+              int m = impacts.advanceShallow(doc);
+              if (m < doc) {
+                throw new RuntimeException("Block max doc: " + m + " is less than the target " + doc);
+              }
+              maxScore = impacts.getMaxScore(max);
             }
-            maxScore = impacts.getMaxScore(max);
-          }
 
-          float score = scorer.score(impacts.freq(), 1);
-          if (score > maxScore) {
-            throw new RuntimeException("score " + score + " is greater than the max score " + maxScore);
+            float score = scorer.score(impacts.freq(), 1);
+            if (score > maxScore) {
+              throw new RuntimeException("score " + score + " is greater than the max score " + maxScore);
+            }
           }
         }
       }
@@ -1848,17 +1855,17 @@ public final class CheckIndex implements Closeable {
    * @lucene.experimental
    */
   public static Status.TermIndexStatus testPostings(CodecReader reader, PrintStream infoStream) throws IOException {
-    return testPostings(reader, infoStream, false, false);
+    return testPostings(reader, infoStream, false, true, false);
   }
   
   /**
    * Test the term index.
    * @lucene.experimental
    */
-  public static Status.TermIndexStatus testPostings(CodecReader reader, PrintStream infoStream, boolean verbose, boolean failFast) throws IOException {
+  public static Status.TermIndexStatus testPostings(CodecReader reader, PrintStream infoStream, boolean verbose, boolean doSlowChecks, boolean failFast) throws IOException {
 
     // TODO: we should go and verify term vectors match, if
-    // crossCheckTermVectors is on...
+    // doSlowChecks is on...
 
     Status.TermIndexStatus status;
     final int maxDoc = reader.maxDoc();
@@ -1870,7 +1877,7 @@ public final class CheckIndex implements Closeable {
 
       final Fields fields = reader.getPostingsReader().getMergeInstance();
       final FieldInfos fieldInfos = reader.getFieldInfos();
-      status = checkFields(fields, reader.getLiveDocs(), maxDoc, fieldInfos, true, false, infoStream, verbose);
+      status = checkFields(fields, reader.getLiveDocs(), maxDoc, fieldInfos, true, false, infoStream, verbose, doSlowChecks);
     } catch (Throwable e) {
       if (failFast) {
         throw IOUtils.rethrowAlways(e);
@@ -2487,7 +2494,7 @@ public final class CheckIndex implements Closeable {
    * Test term vectors.
    * @lucene.experimental
    */
-  public static Status.TermVectorStatus testTermVectors(CodecReader reader, PrintStream infoStream, boolean verbose, boolean crossCheckTermVectors, boolean failFast) throws IOException {
+  public static Status.TermVectorStatus testTermVectors(CodecReader reader, PrintStream infoStream, boolean verbose, boolean doSlowChecks, boolean failFast) throws IOException {
     long startNS = System.nanoTime();
     final Status.TermVectorStatus status = new Status.TermVectorStatus();
     final FieldInfos fieldInfos = reader.getFieldInfos();
@@ -2499,14 +2506,14 @@ public final class CheckIndex implements Closeable {
 
       PostingsEnum postings = null;
 
-      // Only used if crossCheckTermVectors is true:
+      // Only used if doSlowChecks is true:
       PostingsEnum postingsDocs = null;
 
       final Bits liveDocs = reader.getLiveDocs();
 
       final Fields postingsFields;
       // TODO: testTermsIndex
-      if (crossCheckTermVectors) {
+      if (doSlowChecks) {
         postingsFields = reader.getPostingsReader().getMergeInstance();
       } else {
         postingsFields = null;
@@ -2527,7 +2534,7 @@ public final class CheckIndex implements Closeable {
           
           if (tfv != null) {
             // First run with no deletions:
-            checkFields(tfv, null, 1, fieldInfos, false, true, infoStream, verbose);
+            checkFields(tfv, null, 1, fieldInfos, false, true, infoStream, verbose, doSlowChecks);
             
             // Only agg stats if the doc is live:
             final boolean doStats = liveDocs == null || liveDocs.get(j);
@@ -2547,7 +2554,7 @@ public final class CheckIndex implements Closeable {
                 throw new RuntimeException("docID=" + j + " has term vectors for field=" + field + " but FieldInfo has storeTermVector=false");
               }
               
-              if (crossCheckTermVectors) {
+              if (doSlowChecks) {
                 Terms terms = tfv.terms(field);
                 TermsEnum termsEnum = terms.iterator();
                 final boolean postingsHasFreq = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0;
@@ -2761,7 +2768,7 @@ public final class CheckIndex implements Closeable {
    */
   public static class Options {
     boolean doExorcise = false;
-    boolean doCrossCheckTermVectors = false;
+    boolean doSlowChecks = false;
     boolean verbose = false;
     boolean doChecksumsOnly = false;
     List<String> onlySegments = new ArrayList<>();
@@ -2846,8 +2853,11 @@ public final class CheckIndex implements Closeable {
         opts.doChecksumsOnly = true;
       } else if ("-exorcise".equals(arg)) {
         opts.doExorcise = true;
-      } else if ("-crossCheckTermVectors".equals(arg)) {
-        opts.doCrossCheckTermVectors = true;
+      } else if ("-doSlowChecks".equals(arg)) {
+        System.err.println("-doSlowChecks is deprecated, use -slow instead");
+        opts.doSlowChecks = true;
+      } else if ("-slow".equals(arg)) {
+        opts.doSlowChecks = true;
       } else if (arg.equals("-verbose")) {
         opts.verbose = true;
       } else if (arg.equals("-segment")) {
@@ -2873,11 +2883,11 @@ public final class CheckIndex implements Closeable {
 
     if (opts.indexPath == null) {
       throw new IllegalArgumentException("\nERROR: index path not specified" +
-                         "\nUsage: java org.apache.lucene.index.CheckIndex pathToIndex [-exorcise] [-crossCheckTermVectors] [-segment X] [-segment Y] [-dir-impl X]\n" +
+                         "\nUsage: java org.apache.lucene.index.CheckIndex pathToIndex [-exorcise] [-doSlowChecks] [-segment X] [-segment Y] [-dir-impl X]\n" +
                          "\n" +
                          "  -exorcise: actually write a new segments_N file, removing any problematic segments\n" +
                          "  -fast: just verify file checksums, omitting logical integrity checks\n" + 
-                         "  -crossCheckTermVectors: verifies that term vectors match postings; THIS IS VERY SLOW!\n" +
+                         "  -slow: do additional slow checks; THIS IS VERY SLOW!\n" +
                          "  -codec X: when exorcising, codec to write the new segments_N file with\n" +
                          "  -verbose: print additional details\n" +
                          "  -segment X: only check the specified segments.  This can be specified multiple\n" + 
@@ -2907,8 +2917,8 @@ public final class CheckIndex implements Closeable {
       throw new IllegalArgumentException("ERROR: cannot specify both -exorcise and -segment");
     }
     
-    if (opts.doChecksumsOnly && opts.doCrossCheckTermVectors) {
-      throw new IllegalArgumentException("ERROR: cannot specify both -fast and -crossCheckTermVectors");
+    if (opts.doChecksumsOnly && opts.doSlowChecks) {
+      throw new IllegalArgumentException("ERROR: cannot specify both -fast and -slow");
     }
 
     return opts;
@@ -2920,7 +2930,7 @@ public final class CheckIndex implements Closeable {
    * @return 0 iff the index is clean, 1 otherwise
    */
   public int doCheck(Options opts) throws IOException, InterruptedException {
-    setCrossCheckTermVectors(opts.doCrossCheckTermVectors);
+    setDoSlowChecks(opts.doSlowChecks);
     setChecksumsOnly(opts.doChecksumsOnly);
     setInfoStream(opts.out, opts.verbose);
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/317a2e0c/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
index 9f59ae6..983581a 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
@@ -2445,7 +2445,7 @@ public class TestIndexWriter extends LuceneTestCase {
     // Make sure CheckIndex includes id output:
     ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
     CheckIndex checker = new CheckIndex(d);
-    checker.setCrossCheckTermVectors(false);
+    checker.setDoSlowChecks(false);
     checker.setInfoStream(new PrintStream(bos, false, IOUtils.UTF_8), false);
     CheckIndex.Status indexStatus = checker.checkIndex(null);
     String s = bos.toString(IOUtils.UTF_8);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/317a2e0c/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryWrapper.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryWrapper.java b/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryWrapper.java
index c928088..e73a57c 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryWrapper.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryWrapper.java
@@ -30,7 +30,7 @@ import org.apache.lucene.util.TestUtil;
 public abstract class BaseDirectoryWrapper extends FilterDirectory {
   
   private boolean checkIndexOnClose = true;
-  private boolean crossCheckTermVectorsOnClose = true;
+  private boolean doSlowChecksOnClose = true;
   protected volatile boolean isOpen = true;
 
   protected BaseDirectoryWrapper(Directory delegate) {
@@ -42,7 +42,7 @@ public abstract class BaseDirectoryWrapper extends FilterDirectory {
     if (isOpen) {
       isOpen = false;
       if (checkIndexOnClose && DirectoryReader.indexExists(this)) {
-        TestUtil.checkIndex(this, crossCheckTermVectorsOnClose);
+        TestUtil.checkIndex(this, doSlowChecksOnClose);
       }
     }
     super.close();
@@ -65,10 +65,10 @@ public abstract class BaseDirectoryWrapper extends FilterDirectory {
   }
 
   public void setCrossCheckTermVectorsOnClose(boolean value) {
-    this.crossCheckTermVectorsOnClose = value;
+    this.doSlowChecksOnClose = value;
   }
 
   public boolean getCrossCheckTermVectorsOnClose() {
-    return crossCheckTermVectorsOnClose;
+    return doSlowChecksOnClose;
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/317a2e0c/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java b/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
index 7f53017..2969788 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
@@ -281,20 +281,20 @@ public final class TestUtil {
     return checkIndex(dir, true);
   }
 
-  public static CheckIndex.Status checkIndex(Directory dir, boolean crossCheckTermVectors) throws IOException {
-    return checkIndex(dir, crossCheckTermVectors, false, null);
+  public static CheckIndex.Status checkIndex(Directory dir, boolean doSlowChecks) throws IOException {
+    return checkIndex(dir, doSlowChecks, false, null);
   }
 
   /** If failFast is true, then throw the first exception when index corruption is hit, instead of moving on to other fields/segments to
    *  look for any other corruption.  */
-  public static CheckIndex.Status checkIndex(Directory dir, boolean crossCheckTermVectors, boolean failFast, ByteArrayOutputStream output) throws IOException {
+  public static CheckIndex.Status checkIndex(Directory dir, boolean doSlowChecks, boolean failFast, ByteArrayOutputStream output) throws IOException {
     if (output == null) {
       output = new ByteArrayOutputStream(1024);
     }
     // TODO: actually use the dir's locking, unless test uses a special method?
     // some tests e.g. exception tests become much more complicated if they have to close the writer
     try (CheckIndex checker = new CheckIndex(dir, NoLockFactory.INSTANCE.obtainLock(dir, "bogus"))) {
-      checker.setCrossCheckTermVectors(crossCheckTermVectors);
+      checker.setDoSlowChecks(doSlowChecks);
       checker.setFailFast(failFast);
       checker.setInfoStream(new PrintStream(output, false, IOUtils.UTF_8), false);
       CheckIndex.Status indexStatus = checker.checkIndex(null);
@@ -320,7 +320,7 @@ public final class TestUtil {
     }
   }
   
-  public static void checkReader(LeafReader reader, boolean crossCheckTermVectors) throws IOException {
+  public static void checkReader(LeafReader reader, boolean doSlowChecks) throws IOException {
     ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
     PrintStream infoStream = new PrintStream(bos, false, IOUtils.UTF_8);
 
@@ -334,9 +334,9 @@ public final class TestUtil {
     CheckIndex.testLiveDocs(codecReader, infoStream, true);
     CheckIndex.testFieldInfos(codecReader, infoStream, true);
     CheckIndex.testFieldNorms(codecReader, infoStream, true);
-    CheckIndex.testPostings(codecReader, infoStream, false, true);
+    CheckIndex.testPostings(codecReader, infoStream, false, doSlowChecks, true);
     CheckIndex.testStoredFields(codecReader, infoStream, true);
-    CheckIndex.testTermVectors(codecReader, infoStream, false, crossCheckTermVectors, true);
+    CheckIndex.testTermVectors(codecReader, infoStream, false, doSlowChecks, true);
     CheckIndex.testDocValues(codecReader, infoStream, true);
     CheckIndex.testPoints(codecReader, infoStream, true);