You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2015/03/03 02:36:12 UTC

svn commit: r1663510 - in /lucene/dev/branches/branch_5x: ./ lucene/ lucene/core/ lucene/core/src/java/org/apache/lucene/index/ lucene/test-framework/ lucene/test-framework/src/java/org/apache/lucene/index/ lucene/test-framework/src/java/org/apache/luc...

Author: rmuir
Date: Tue Mar  3 01:36:12 2015
New Revision: 1663510

URL: http://svn.apache.org/r1663510
Log:
LUCENE-6320: speed up checkindex

Modified:
    lucene/dev/branches/branch_5x/   (props changed)
    lucene/dev/branches/branch_5x/lucene/   (props changed)
    lucene/dev/branches/branch_5x/lucene/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_5x/lucene/core/   (props changed)
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
    lucene/dev/branches/branch_5x/lucene/test-framework/   (props changed)
    lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java
    lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java

Modified: lucene/dev/branches/branch_5x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/CHANGES.txt?rev=1663510&r1=1663509&r2=1663510&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_5x/lucene/CHANGES.txt Tue Mar  3 01:36:12 2015
@@ -99,6 +99,8 @@ Optimizations
 * LUCENE-6318: Reduce RAM usage of FieldInfos when there are many fields.
   (Mike McCandless, Robert Muir)
 
+* LUCENE-6320: Speed up CheckIndex. (Robert Muir)
+
 API Changes
 
 * LUCENE-6204, LUCENE-6208: Simplify CompoundFormat: remove files()

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java?rev=1663510&r1=1663509&r2=1663510&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java Tue Mar  3 01:36:12 2015
@@ -31,8 +31,13 @@ import java.util.Locale;
 import java.util.Map;
 
 import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.DocValuesProducer;
+import org.apache.lucene.codecs.NormsProducer;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.document.Document;
+import org.apache.lucene.codecs.StoredFieldsReader;
+import org.apache.lucene.codecs.TermVectorsReader;
+import org.apache.lucene.document.DocumentStoredFieldVisitor;
 import org.apache.lucene.index.CheckIndex.Status.DocValuesStatus;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.store.AlreadyClosedException;
@@ -175,7 +180,7 @@ public class CheckIndex implements Close
       /** Current deletions generation. */
       public long deletionsGen;
 
-      /** True if we were able to open an LeafReader on this
+      /** True if we were able to open a CodecReader on this
        *  segment. */
       public boolean openReaderPassed;
 
@@ -756,7 +761,7 @@ public class CheckIndex implements Close
    * Test live docs.
    * @lucene.experimental
    */
-  public static Status.LiveDocStatus testLiveDocs(LeafReader reader, PrintStream infoStream, boolean failFast) throws IOException {
+  public static Status.LiveDocStatus testLiveDocs(CodecReader reader, PrintStream infoStream, boolean failFast) throws IOException {
     long startNS = System.nanoTime();
     final Status.LiveDocStatus status = new Status.LiveDocStatus();
     
@@ -813,7 +818,7 @@ public class CheckIndex implements Close
    * Test field infos.
    * @lucene.experimental
    */
-  public static Status.FieldInfoStatus testFieldInfos(LeafReader reader, PrintStream infoStream, boolean failFast) throws IOException {
+  public static Status.FieldInfoStatus testFieldInfos(CodecReader reader, PrintStream infoStream, boolean failFast) throws IOException {
     long startNS = System.nanoTime();
     final Status.FieldInfoStatus status = new Status.FieldInfoStatus();
     
@@ -846,7 +851,7 @@ public class CheckIndex implements Close
    * Test field norms.
    * @lucene.experimental
    */
-  public static Status.FieldNormStatus testFieldNorms(LeafReader reader, PrintStream infoStream, boolean failFast) throws IOException {
+  public static Status.FieldNormStatus testFieldNorms(CodecReader reader, PrintStream infoStream, boolean failFast) throws IOException {
     long startNS = System.nanoTime();
     final Status.FieldNormStatus status = new Status.FieldNormStatus();
 
@@ -855,14 +860,14 @@ public class CheckIndex implements Close
       if (infoStream != null) {
         infoStream.print("    test: field norms.........");
       }
+      NormsProducer normsReader = reader.getNormsReader();
+      if (normsReader != null) {
+        normsReader = normsReader.getMergeInstance();
+      }
       for (FieldInfo info : reader.getFieldInfos()) {
         if (info.hasNorms()) {
-          checkNorms(info, reader, infoStream);
+          checkNumericDocValues(info.name, reader.maxDoc(), normsReader.getNorms(info), new Bits.MatchAllBits(reader.maxDoc()));
           ++status.totFields;
-        } else {
-          if (reader.getNormValues(info.name) != null) {
-            throw new RuntimeException("field: " + info.name + " should omit norms but has them!");
-          }
         }
       }
 
@@ -898,7 +903,6 @@ public class CheckIndex implements Close
     int computedFieldCount = 0;
     
     PostingsEnum docs = null;
-    PostingsEnum docsAndFreqs = null;
     PostingsEnum postings = null;
     
     String lastField = null;
@@ -1175,20 +1179,20 @@ public class CheckIndex implements Close
         // Re-count if there are deleted docs:
         if (liveDocs != null) {
           if (hasFreqs) {
-            final PostingsEnum docsNoDel = termsEnum.postings(null, docsAndFreqs);
+            docs = termsEnum.postings(null, docs);
             docCount = 0;
             totalTermFreq = 0;
-            while(docsNoDel.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
-              visitedDocs.set(docsNoDel.docID());
+            while(docs.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
+              visitedDocs.set(docs.docID());
               docCount++;
-              totalTermFreq += docsNoDel.freq();
+              totalTermFreq += docs.freq();
             }
           } else {
-            final PostingsEnum docsNoDel = termsEnum.postings(null, docs, PostingsEnum.NONE);
+            docs = termsEnum.postings(null, docs, PostingsEnum.NONE);
             docCount = 0;
             totalTermFreq = -1;
-            while(docsNoDel.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
-              visitedDocs.set(docsNoDel.docID());
+            while(docs.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
+              visitedDocs.set(docs.docID());
               docCount++;
             }
           }
@@ -1458,7 +1462,7 @@ public class CheckIndex implements Close
    * Test the term index.
    * @lucene.experimental
    */
-  public static Status.TermIndexStatus testPostings(LeafReader reader, PrintStream infoStream) throws IOException {
+  public static Status.TermIndexStatus testPostings(CodecReader reader, PrintStream infoStream) throws IOException {
     return testPostings(reader, infoStream, false, false);
   }
   
@@ -1466,7 +1470,7 @@ public class CheckIndex implements Close
    * Test the term index.
    * @lucene.experimental
    */
-  public static Status.TermIndexStatus testPostings(LeafReader reader, PrintStream infoStream, boolean verbose, boolean failFast) throws IOException {
+  public static Status.TermIndexStatus testPostings(CodecReader reader, PrintStream infoStream, boolean verbose, boolean failFast) throws IOException {
 
     // TODO: we should go and verify term vectors match, if
     // crossCheckTermVectors is on...
@@ -1480,7 +1484,7 @@ public class CheckIndex implements Close
         infoStream.print("    test: terms, freq, prox...");
       }
 
-      final Fields fields = reader.fields();
+      final Fields fields = reader.getPostingsReader().getMergeInstance();
       final FieldInfos fieldInfos = reader.getFieldInfos();
       status = checkFields(fields, liveDocs, maxDoc, fieldInfos, true, false, infoStream, verbose);
       if (liveDocs != null) {
@@ -1508,7 +1512,7 @@ public class CheckIndex implements Close
    * Test stored fields.
    * @lucene.experimental
    */
-  public static Status.StoredFieldStatus testStoredFields(LeafReader reader, PrintStream infoStream, boolean failFast) throws IOException {
+  public static Status.StoredFieldStatus testStoredFields(CodecReader reader, PrintStream infoStream, boolean failFast) throws IOException {
     long startNS = System.nanoTime();
     final Status.StoredFieldStatus status = new Status.StoredFieldStatus();
 
@@ -1519,10 +1523,13 @@ public class CheckIndex implements Close
 
       // Scan stored fields for all documents
       final Bits liveDocs = reader.getLiveDocs();
+      StoredFieldsReader storedFields = reader.getFieldsReader().getMergeInstance();
       for (int j = 0; j < reader.maxDoc(); ++j) {
         // Intentionally pull even deleted documents to
         // make sure they too are not corrupt:
-        Document doc = reader.document(j);
+        DocumentStoredFieldVisitor visitor = new DocumentStoredFieldVisitor();
+        storedFields.visitDocument(j, visitor);
+        Document doc = visitor.getDocument();
         if (liveDocs == null || liveDocs.get(j)) {
           status.docCount++;
           status.totFields += doc.getFields().size();
@@ -1556,7 +1563,7 @@ public class CheckIndex implements Close
    * Test docvalues.
    * @lucene.experimental
    */
-  public static Status.DocValuesStatus testDocValues(LeafReader reader,
+  public static Status.DocValuesStatus testDocValues(CodecReader reader,
                                                      PrintStream infoStream,
                                                      boolean failFast) throws IOException {
     long startNS = System.nanoTime();
@@ -1565,18 +1572,14 @@ public class CheckIndex implements Close
       if (infoStream != null) {
         infoStream.print("    test: docvalues...........");
       }
+      DocValuesProducer dvReader = reader.getDocValuesReader();
+      if (dvReader != null) {
+        dvReader = dvReader.getMergeInstance();
+      }
       for (FieldInfo fieldInfo : reader.getFieldInfos()) {
         if (fieldInfo.getDocValuesType() != DocValuesType.NONE) {
           status.totalValueFields++;
-          checkDocValues(fieldInfo, reader, infoStream, status);
-        } else {
-          if (reader.getBinaryDocValues(fieldInfo.name) != null ||
-              reader.getNumericDocValues(fieldInfo.name) != null ||
-              reader.getSortedDocValues(fieldInfo.name) != null || 
-              reader.getSortedSetDocValues(fieldInfo.name) != null || 
-              reader.getDocsWithField(fieldInfo.name) != null) {
-            throw new RuntimeException("field: " + fieldInfo.name + " has docvalues but should omit them!");
-          }
+          checkDocValues(fieldInfo, dvReader, reader.maxDoc(), infoStream, status);
         }
       }
 
@@ -1602,8 +1605,8 @@ public class CheckIndex implements Close
     return status;
   }
   
-  private static void checkBinaryDocValues(String fieldName, LeafReader reader, BinaryDocValues dv, Bits docsWithField) {
-    for (int i = 0; i < reader.maxDoc(); i++) {
+  private static void checkBinaryDocValues(String fieldName, int maxDoc, BinaryDocValues dv, Bits docsWithField) {
+    for (int i = 0; i < maxDoc; i++) {
       final BytesRef term = dv.get(i);
       assert term.isValid();
       if (docsWithField.get(i) == false && term.length > 0) {
@@ -1612,12 +1615,12 @@ public class CheckIndex implements Close
     }
   }
   
-  private static void checkSortedDocValues(String fieldName, LeafReader reader, SortedDocValues dv, Bits docsWithField) {
-    checkBinaryDocValues(fieldName, reader, dv, docsWithField);
+  private static void checkSortedDocValues(String fieldName, int maxDoc, SortedDocValues dv, Bits docsWithField) {
+    checkBinaryDocValues(fieldName, maxDoc, dv, docsWithField);
     final int maxOrd = dv.getValueCount()-1;
     FixedBitSet seenOrds = new FixedBitSet(dv.getValueCount());
     int maxOrd2 = -1;
-    for (int i = 0; i < reader.maxDoc(); i++) {
+    for (int i = 0; i < maxDoc; i++) {
       int ord = dv.getOrd(i);
       if (ord == -1) {
         if (docsWithField.get(i)) {
@@ -1652,11 +1655,11 @@ public class CheckIndex implements Close
     }
   }
   
-  private static void checkSortedSetDocValues(String fieldName, LeafReader reader, SortedSetDocValues dv, Bits docsWithField) {
+  private static void checkSortedSetDocValues(String fieldName, int maxDoc, SortedSetDocValues dv, Bits docsWithField) {
     final long maxOrd = dv.getValueCount()-1;
     LongBitSet seenOrds = new LongBitSet(dv.getValueCount());
     long maxOrd2 = -1;
-    for (int i = 0; i < reader.maxDoc(); i++) {
+    for (int i = 0; i < maxDoc; i++) {
       dv.setDocument(i);
       long lastOrd = -1;
       long ord;
@@ -1722,8 +1725,8 @@ public class CheckIndex implements Close
     }
   }
   
-  private static void checkSortedNumericDocValues(String fieldName, LeafReader reader, SortedNumericDocValues ndv, Bits docsWithField) {
-    for (int i = 0; i < reader.maxDoc(); i++) {
+  private static void checkSortedNumericDocValues(String fieldName, int maxDoc, SortedNumericDocValues ndv, Bits docsWithField) {
+    for (int i = 0; i < maxDoc; i++) {
       ndv.setDocument(i);
       int count = ndv.count();
       if (docsWithField.get(i)) {
@@ -1746,8 +1749,8 @@ public class CheckIndex implements Close
     }
   }
 
-  private static void checkNumericDocValues(String fieldName, LeafReader reader, NumericDocValues ndv, Bits docsWithField) {
-    for (int i = 0; i < reader.maxDoc(); i++) {
+  private static void checkNumericDocValues(String fieldName, int maxDoc, NumericDocValues ndv, Bits docsWithField) {
+    for (int i = 0; i < maxDoc; i++) {
       long value = ndv.get(i);
       if (docsWithField.get(i) == false && value != 0) {
         throw new RuntimeException("dv for field: " + fieldName + " is marked missing but has value=" + value + " for doc: " + i);
@@ -1755,80 +1758,44 @@ public class CheckIndex implements Close
     }
   }
   
-  private static void checkDocValues(FieldInfo fi, LeafReader reader, PrintStream infoStream, DocValuesStatus status) throws Exception {
-    Bits docsWithField = reader.getDocsWithField(fi.name);
+  private static void checkDocValues(FieldInfo fi, DocValuesProducer dvReader, int maxDoc, PrintStream infoStream, DocValuesStatus status) throws Exception {
+    Bits docsWithField = dvReader.getDocsWithField(fi);
     if (docsWithField == null) {
       throw new RuntimeException(fi.name + " docsWithField does not exist");
-    } else if (docsWithField.length() != reader.maxDoc()) {
-      throw new RuntimeException(fi.name + " docsWithField has incorrect length: " + docsWithField.length() + ",expected: " + reader.maxDoc());
+    } else if (docsWithField.length() != maxDoc) {
+      throw new RuntimeException(fi.name + " docsWithField has incorrect length: " + docsWithField.length() + ",expected: " + maxDoc);
     }
     switch(fi.getDocValuesType()) {
       case SORTED:
         status.totalSortedFields++;
-        checkSortedDocValues(fi.name, reader, reader.getSortedDocValues(fi.name), docsWithField);
-        if (reader.getBinaryDocValues(fi.name) != null ||
-            reader.getNumericDocValues(fi.name) != null ||
-            reader.getSortedNumericDocValues(fi.name) != null ||
-            reader.getSortedSetDocValues(fi.name) != null) {
-          throw new RuntimeException(fi.name + " returns multiple docvalues types!");
-        }
+        checkSortedDocValues(fi.name, maxDoc, dvReader.getSorted(fi), docsWithField);
         break;
       case SORTED_NUMERIC:
         status.totalSortedNumericFields++;
-        checkSortedNumericDocValues(fi.name, reader, reader.getSortedNumericDocValues(fi.name), docsWithField);
-        if (reader.getBinaryDocValues(fi.name) != null ||
-            reader.getNumericDocValues(fi.name) != null ||
-            reader.getSortedSetDocValues(fi.name) != null ||
-            reader.getSortedDocValues(fi.name) != null) {
-          throw new RuntimeException(fi.name + " returns multiple docvalues types!");
-        }
+        checkSortedNumericDocValues(fi.name, maxDoc, dvReader.getSortedNumeric(fi), docsWithField);
         break;
       case SORTED_SET:
         status.totalSortedSetFields++;
-        checkSortedSetDocValues(fi.name, reader, reader.getSortedSetDocValues(fi.name), docsWithField);
-        if (reader.getBinaryDocValues(fi.name) != null ||
-            reader.getNumericDocValues(fi.name) != null ||
-            reader.getSortedNumericDocValues(fi.name) != null ||
-            reader.getSortedDocValues(fi.name) != null) {
-          throw new RuntimeException(fi.name + " returns multiple docvalues types!");
-        }
+        checkSortedSetDocValues(fi.name, maxDoc, dvReader.getSortedSet(fi), docsWithField);
         break;
       case BINARY:
         status.totalBinaryFields++;
-        checkBinaryDocValues(fi.name, reader, reader.getBinaryDocValues(fi.name), docsWithField);
-        if (reader.getNumericDocValues(fi.name) != null ||
-            reader.getSortedDocValues(fi.name) != null ||
-            reader.getSortedNumericDocValues(fi.name) != null ||
-            reader.getSortedSetDocValues(fi.name) != null) {
-          throw new RuntimeException(fi.name + " returns multiple docvalues types!");
-        }
+        checkBinaryDocValues(fi.name, maxDoc, dvReader.getBinary(fi), docsWithField);
         break;
       case NUMERIC:
         status.totalNumericFields++;
-        checkNumericDocValues(fi.name, reader, reader.getNumericDocValues(fi.name), docsWithField);
-        if (reader.getBinaryDocValues(fi.name) != null ||
-            reader.getSortedDocValues(fi.name) != null ||
-            reader.getSortedNumericDocValues(fi.name) != null ||
-            reader.getSortedSetDocValues(fi.name) != null) {
-          throw new RuntimeException(fi.name + " returns multiple docvalues types!");
-        }
+        checkNumericDocValues(fi.name, maxDoc, dvReader.getNumeric(fi), docsWithField);
         break;
       default:
         throw new AssertionError();
     }
   }
-  
-  private static void checkNorms(FieldInfo fi, LeafReader reader, PrintStream infoStream) throws IOException {
-    if (fi.hasNorms()) {
-      checkNumericDocValues(fi.name, reader, reader.getNormValues(fi.name), new Bits.MatchAllBits(reader.maxDoc()));
-    }
-  }
 
   /**
    * Test term vectors.
    * @lucene.experimental
    */
-  public static Status.TermVectorStatus testTermVectors(LeafReader reader, PrintStream infoStream) throws IOException {
+  public static Status.TermVectorStatus testTermVectors(CodecReader reader, PrintStream infoStream) throws IOException {
     return testTermVectors(reader, infoStream, false, false, false);
   }
 
@@ -1836,7 +1803,7 @@ public class CheckIndex implements Close
    * Test term vectors.
    * @lucene.experimental
    */
-  public static Status.TermVectorStatus testTermVectors(LeafReader reader, PrintStream infoStream, boolean verbose, boolean crossCheckTermVectors, boolean failFast) throws IOException {
+  public static Status.TermVectorStatus testTermVectors(CodecReader reader, PrintStream infoStream, boolean verbose, boolean crossCheckTermVectors, boolean failFast) throws IOException {
     long startNS = System.nanoTime();
     final Status.TermVectorStatus status = new Status.TermVectorStatus();
     final FieldInfos fieldInfos = reader.getFieldInfos();
@@ -1859,139 +1826,143 @@ public class CheckIndex implements Close
       final Fields postingsFields;
       // TODO: testTermsIndex
       if (crossCheckTermVectors) {
-        postingsFields = reader.fields();
+        postingsFields = reader.getPostingsReader().getMergeInstance();
       } else {
         postingsFields = null;
       }
 
       TermsEnum termsEnum = null;
       TermsEnum postingsTermsEnum = null;
+      
+      TermVectorsReader vectorsReader = reader.getTermVectorsReader();
 
-      for (int j = 0; j < reader.maxDoc(); ++j) {
-        // Intentionally pull/visit (but don't count in
-        // stats) deleted documents to make sure they too
-        // are not corrupt:
-        Fields tfv = reader.getTermVectors(j);
-
-        // TODO: can we make a IS(FIR) that searches just
-        // this term vector... to pass for searcher?
-
-        if (tfv != null) {
-          // First run with no deletions:
-          checkFields(tfv, null, 1, fieldInfos, false, true, infoStream, verbose);
-
-          if (j == 0) {
-            // Also test with the 1 doc deleted; we only do this for first doc because this really is just looking for a [slightly] buggy
-            // TermVectors impl that fails to respect the incoming live docs:
-            checkFields(tfv, onlyDocIsDeleted, 1, fieldInfos, false, true, infoStream, verbose);
-          }
-
-          // Only agg stats if the doc is live:
-          final boolean doStats = liveDocs == null || liveDocs.get(j);
-
-          if (doStats) {
-            status.docCount++;
-          }
-
-          for(String field : tfv) {
-            if (doStats) {
-              status.totVectors++;
+      if (vectorsReader != null) {
+        vectorsReader = vectorsReader.getMergeInstance();
+        for (int j = 0; j < reader.maxDoc(); ++j) {
+          // Intentionally pull/visit (but don't count in
+          // stats) deleted documents to make sure they too
+          // are not corrupt:
+          Fields tfv = vectorsReader.get(j);
+          
+          // TODO: can we make a IS(FIR) that searches just
+          // this term vector... to pass for searcher?
+          
+          if (tfv != null) {
+            // First run with no deletions:
+            checkFields(tfv, null, 1, fieldInfos, false, true, infoStream, verbose);
+            
+            if (j == 0) {
+              // Also test with the 1 doc deleted; we only do this for first doc because this really is just looking for a [slightly] buggy
+              // TermVectors impl that fails to respect the incoming live docs:
+              checkFields(tfv, onlyDocIsDeleted, 1, fieldInfos, false, true, infoStream, verbose);
             }
-
-            // Make sure FieldInfo thinks this field is vector'd:
-            final FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
-            if (!fieldInfo.hasVectors()) {
-              throw new RuntimeException("docID=" + j + " has term vectors for field=" + field + " but FieldInfo has storeTermVector=false");
+            
+            // Only agg stats if the doc is live:
+            final boolean doStats = liveDocs == null || liveDocs.get(j);
+            
+            if (doStats) {
+              status.docCount++;
             }
-
-            if (crossCheckTermVectors) {
-              Terms terms = tfv.terms(field);
-              termsEnum = terms.iterator(termsEnum);
-              final boolean postingsHasFreq = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0;
-              final boolean postingsHasPayload = fieldInfo.hasPayloads();
-              final boolean vectorsHasPayload = terms.hasPayloads();
-
-              Terms postingsTerms = postingsFields.terms(field);
-              if (postingsTerms == null) {
-                throw new RuntimeException("vector field=" + field + " does not exist in postings; doc=" + j);
+            
+            for(String field : tfv) {
+              if (doStats) {
+                status.totVectors++;
               }
-              postingsTermsEnum = postingsTerms.iterator(postingsTermsEnum);
               
-              final boolean hasProx = terms.hasOffsets() || terms.hasPositions();
-              BytesRef term = null;
-              while ((term = termsEnum.next()) != null) {
-
-                if (hasProx) {
-                  postings = termsEnum.postings(null, postings, PostingsEnum.ALL);
-                  assert postings != null;
-                  docs = null;
-                } else {
-                  docs = termsEnum.postings(null, docs);
-                  assert docs != null;
-                  postings = null;
-                }
-
-                final PostingsEnum docs2;
-                if (hasProx) {
-                  assert postings != null;
-                  docs2 = postings;
-                } else {
-                  assert docs != null;
-                  docs2 = docs;
-                }
-
-                final PostingsEnum postingsDocs2;
-                if (!postingsTermsEnum.seekExact(term)) {
-                  throw new RuntimeException("vector term=" + term + " field=" + field + " does not exist in postings; doc=" + j);
+              // Make sure FieldInfo thinks this field is vector'd:
+              final FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
+              if (!fieldInfo.hasVectors()) {
+                throw new RuntimeException("docID=" + j + " has term vectors for field=" + field + " but FieldInfo has storeTermVector=false");
+              }
+              
+              if (crossCheckTermVectors) {
+                Terms terms = tfv.terms(field);
+                termsEnum = terms.iterator(termsEnum);
+                final boolean postingsHasFreq = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0;
+                final boolean postingsHasPayload = fieldInfo.hasPayloads();
+                final boolean vectorsHasPayload = terms.hasPayloads();
+                
+                Terms postingsTerms = postingsFields.terms(field);
+                if (postingsTerms == null) {
+                  throw new RuntimeException("vector field=" + field + " does not exist in postings; doc=" + j);
                 }
-                postingsPostings = postingsTermsEnum.postings(null, postingsPostings, PostingsEnum.ALL);
-                if (postingsPostings == null) {
-                  // Term vectors were indexed w/ pos but postings were not
-                  postingsDocs = postingsTermsEnum.postings(null, postingsDocs);
-                  if (postingsDocs == null) {
+                postingsTermsEnum = postingsTerms.iterator(postingsTermsEnum);
+                
+                final boolean hasProx = terms.hasOffsets() || terms.hasPositions();
+                BytesRef term = null;
+                while ((term = termsEnum.next()) != null) {
+                  
+                  if (hasProx) {
+                    postings = termsEnum.postings(null, postings, PostingsEnum.ALL);
+                    assert postings != null;
+                    docs = null;
+                  } else {
+                    docs = termsEnum.postings(null, docs);
+                    assert docs != null;
+                    postings = null;
+                  }
+                  
+                  final PostingsEnum docs2;
+                  if (hasProx) {
+                    assert postings != null;
+                    docs2 = postings;
+                  } else {
+                    assert docs != null;
+                    docs2 = docs;
+                  }
+                  
+                  final PostingsEnum postingsDocs2;
+                  if (!postingsTermsEnum.seekExact(term)) {
                     throw new RuntimeException("vector term=" + term + " field=" + field + " does not exist in postings; doc=" + j);
                   }
-                }
-
-                if (postingsPostings != null) {
-                  postingsDocs2 = postingsPostings;
-                } else {
-                  postingsDocs2 = postingsDocs;
-                }
+                  postingsPostings = postingsTermsEnum.postings(null, postingsPostings, PostingsEnum.ALL);
+                  if (postingsPostings == null) {
+                    // Term vectors were indexed w/ pos but postings were not
+                    postingsDocs = postingsTermsEnum.postings(null, postingsDocs);
+                    if (postingsDocs == null) {
+                      throw new RuntimeException("vector term=" + term + " field=" + field + " does not exist in postings; doc=" + j);
+                    }
+                  }
                   
-                final int advanceDoc = postingsDocs2.advance(j);
-                if (advanceDoc != j) {
-                  throw new RuntimeException("vector term=" + term + " field=" + field + ": doc=" + j + " was not found in postings (got: " + advanceDoc + ")");
-                }
-
-                final int doc = docs2.nextDoc();
+                  if (postingsPostings != null) {
+                    postingsDocs2 = postingsPostings;
+                  } else {
+                    postingsDocs2 = postingsDocs;
+                  }
                   
-                if (doc != 0) {
-                  throw new RuntimeException("vector for doc " + j + " didn't return docID=0: got docID=" + doc);
-                }
-
-                if (postingsHasFreq) {
-                  final int tf = docs2.freq();
-                  if (postingsHasFreq && postingsDocs2.freq() != tf) {
-                    throw new RuntimeException("vector term=" + term + " field=" + field + " doc=" + j + ": freq=" + tf + " differs from postings freq=" + postingsDocs2.freq());
+                  final int advanceDoc = postingsDocs2.advance(j);
+                  if (advanceDoc != j) {
+                    throw new RuntimeException("vector term=" + term + " field=" + field + ": doc=" + j + " was not found in postings (got: " + advanceDoc + ")");
                   }
-                
-                  if (hasProx) {
-                    for (int i = 0; i < tf; i++) {
-                      int pos = postings.nextPosition();
-                      if (postingsPostings != null) {
-                        int postingsPos = postingsPostings.nextPosition();
-                        if (terms.hasPositions() && pos != postingsPos) {
-                          throw new RuntimeException("vector term=" + term + " field=" + field + " doc=" + j + ": pos=" + pos + " differs from postings pos=" + postingsPos);
+                  
+                  final int doc = docs2.nextDoc();
+                  
+                  if (doc != 0) {
+                    throw new RuntimeException("vector for doc " + j + " didn't return docID=0: got docID=" + doc);
+                  }
+                  
+                  if (postingsHasFreq) {
+                    final int tf = docs2.freq();
+                    if (postingsHasFreq && postingsDocs2.freq() != tf) {
+                      throw new RuntimeException("vector term=" + term + " field=" + field + " doc=" + j + ": freq=" + tf + " differs from postings freq=" + postingsDocs2.freq());
+                    }
+                    
+                    if (hasProx) {
+                      for (int i = 0; i < tf; i++) {
+                        int pos = postings.nextPosition();
+                        if (postingsPostings != null) {
+                          int postingsPos = postingsPostings.nextPosition();
+                          if (terms.hasPositions() && pos != postingsPos) {
+                            throw new RuntimeException("vector term=" + term + " field=" + field + " doc=" + j + ": pos=" + pos + " differs from postings pos=" + postingsPos);
+                          }
                         }
-                      }
-
-                      // Call the methods to at least make
-                      // sure they don't throw exc:
-                      final int startOffset = postings.startOffset();
-                      final int endOffset = postings.endOffset();
-                      // TODO: these are too anal...?
-                      /*
+                        
+                        // Call the methods to at least make
+                        // sure they don't throw exc:
+                        final int startOffset = postings.startOffset();
+                        final int endOffset = postings.endOffset();
+                        // TODO: these are too anal...?
+                        /*
                         if (endOffset < startOffset) {
                         throw new RuntimeException("vector startOffset=" + startOffset + " is > endOffset=" + endOffset);
                         }
@@ -1999,44 +1970,45 @@ public class CheckIndex implements Close
                         throw new RuntimeException("vector startOffset=" + startOffset + " is < prior startOffset=" + lastStartOffset);
                         }
                         lastStartOffset = startOffset;
-                      */
-
-                      if (postingsPostings != null) {
-                        final int postingsStartOffset = postingsPostings.startOffset();
-
-                        final int postingsEndOffset = postingsPostings.endOffset();
-                        if (startOffset != -1 && postingsStartOffset != -1 && startOffset != postingsStartOffset) {
-                          throw new RuntimeException("vector term=" + term + " field=" + field + " doc=" + j + ": startOffset=" + startOffset + " differs from postings startOffset=" + postingsStartOffset);
-                        }
-                        if (endOffset != -1 && postingsEndOffset != -1 && endOffset != postingsEndOffset) {
-                          throw new RuntimeException("vector term=" + term + " field=" + field + " doc=" + j + ": endOffset=" + endOffset + " differs from postings endOffset=" + postingsEndOffset);
-                        }
-                      }
-                      
-                      BytesRef payload = postings.getPayload();
-           
-                      if (payload != null) {
-                        assert vectorsHasPayload;
-                      }
-                      
-                      if (postingsHasPayload && vectorsHasPayload) {
-                        assert postingsPostings != null;
+                         */
                         
-                        if (payload == null) {
-                          // we have payloads, but not at this position. 
-                          // postings has payloads too, it should not have one at this position
-                          if (postingsPostings.getPayload() != null) {
-                            throw new RuntimeException("vector term=" + term + " field=" + field + " doc=" + j + " has no payload but postings does: " + postingsPostings.getPayload());
+                        if (postingsPostings != null) {
+                          final int postingsStartOffset = postingsPostings.startOffset();
+                          
+                          final int postingsEndOffset = postingsPostings.endOffset();
+                          if (startOffset != -1 && postingsStartOffset != -1 && startOffset != postingsStartOffset) {
+                            throw new RuntimeException("vector term=" + term + " field=" + field + " doc=" + j + ": startOffset=" + startOffset + " differs from postings startOffset=" + postingsStartOffset);
                           }
-                        } else {
-                          // we have payloads, and one at this position
-                          // postings should also have one at this position, with the same bytes.
-                          if (postingsPostings.getPayload() == null) {
-                            throw new RuntimeException("vector term=" + term + " field=" + field + " doc=" + j + " has payload=" + payload + " but postings does not.");
+                          if (endOffset != -1 && postingsEndOffset != -1 && endOffset != postingsEndOffset) {
+                            throw new RuntimeException("vector term=" + term + " field=" + field + " doc=" + j + ": endOffset=" + endOffset + " differs from postings endOffset=" + postingsEndOffset);
                           }
-                          BytesRef postingsPayload = postingsPostings.getPayload();
-                          if (!payload.equals(postingsPayload)) {
-                            throw new RuntimeException("vector term=" + term + " field=" + field + " doc=" + j + " has payload=" + payload + " but differs from postings payload=" + postingsPayload);
+                        }
+                        
+                        BytesRef payload = postings.getPayload();
+                        
+                        if (payload != null) {
+                          assert vectorsHasPayload;
+                        }
+                        
+                        if (postingsHasPayload && vectorsHasPayload) {
+                          assert postingsPostings != null;
+                          
+                          if (payload == null) {
+                            // we have payloads, but not at this position. 
+                            // postings has payloads too, it should not have one at this position
+                            if (postingsPostings.getPayload() != null) {
+                              throw new RuntimeException("vector term=" + term + " field=" + field + " doc=" + j + " has no payload but postings does: " + postingsPostings.getPayload());
+                            }
+                          } else {
+                            // we have payloads, and one at this position
+                            // postings should also have one at this position, with the same bytes.
+                            if (postingsPostings.getPayload() == null) {
+                              throw new RuntimeException("vector term=" + term + " field=" + field + " doc=" + j + " has payload=" + payload + " but postings does not.");
+                            }
+                            BytesRef postingsPayload = postingsPostings.getPayload();
+                            if (!payload.equals(postingsPayload)) {
+                              throw new RuntimeException("vector term=" + term + " field=" + field + " doc=" + j + " has payload=" + payload + " but differs from postings payload=" + postingsPayload);
+                            }
                           }
                         }
                       }

Modified: lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java?rev=1663510&r1=1663509&r2=1663510&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java (original)
+++ lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java Tue Mar  3 01:36:12 2015
@@ -45,6 +45,7 @@ import org.apache.lucene.document.Sorted
 import org.apache.lucene.document.StoredField;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.CheckIndex.Status.DocValuesStatus;
 import org.apache.lucene.index.TermsEnum.SeekStatus;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
@@ -2674,9 +2675,12 @@ public abstract class BaseDocValuesForma
               PrintStream infoStream = new PrintStream(bos, false, IOUtils.UTF_8);
               startingGun.await();
               for (LeafReaderContext leaf : r.leaves()) {
-                CheckIndex.testDocValues(leaf.reader(), infoStream, true);
+                DocValuesStatus status = CheckIndex.testDocValues((SegmentReader)leaf.reader(), infoStream, true);
+                if (status.error != null) {
+                  throw status.error;
+                }
               }
-            } catch (Exception e) {
+            } catch (Throwable e) {
               throw new RuntimeException();
             }
           }

Modified: lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java?rev=1663510&r1=1663509&r2=1663510&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java (original)
+++ lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java Tue Mar  3 01:36:12 2015
@@ -73,6 +73,7 @@ import org.apache.lucene.index.CheckInde
 import org.apache.lucene.index.ConcurrentMergeScheduler;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.DocValuesType;
+import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.FilterLeafReader;
@@ -301,14 +302,23 @@ public final class TestUtil {
     ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
     PrintStream infoStream = new PrintStream(bos, false, IOUtils.UTF_8);
 
-    reader.checkIntegrity();
-    CheckIndex.testLiveDocs(reader, infoStream, true);
-    CheckIndex.testFieldInfos(reader, infoStream, true);
-    CheckIndex.testFieldNorms(reader, infoStream, true);
-    CheckIndex.testPostings(reader, infoStream, false, true);
-    CheckIndex.testStoredFields(reader, infoStream, true);
-    CheckIndex.testTermVectors(reader, infoStream, false, crossCheckTermVectors, true);
-    CheckIndex.testDocValues(reader, infoStream, true);
+    final CodecReader codecReader;
+    if (reader instanceof CodecReader) {
+      codecReader = (CodecReader) reader;
+      reader.checkIntegrity();
+    } else {
+      codecReader = SlowCodecReaderWrapper.wrap(reader);
+    }
+    CheckIndex.testLiveDocs(codecReader, infoStream, true);
+    CheckIndex.testFieldInfos(codecReader, infoStream, true);
+    CheckIndex.testFieldNorms(codecReader, infoStream, true);
+    CheckIndex.testPostings(codecReader, infoStream, false, true);
+    CheckIndex.testStoredFields(codecReader, infoStream, true);
+    CheckIndex.testTermVectors(codecReader, infoStream, false, crossCheckTermVectors, true);
+    CheckIndex.testDocValues(codecReader, infoStream, true);
+    
+    // some checks really against the reader API
+    checkReaderSanity(reader);
     
     if (LuceneTestCase.INFOSTREAM) {
       System.out.println(bos.toString(IOUtils.UTF_8));
@@ -324,6 +334,76 @@ public final class TestUtil {
       assert Accountables.toString(sr) != null;
     }
   }
+  
+  // used by TestUtil.checkReader to check some things really unrelated to the index,
+  // just looking for bugs in indexreader implementations.
+  private static void checkReaderSanity(LeafReader reader) throws IOException {
+    for (FieldInfo info : reader.getFieldInfos()) {
+      
+      // reader shouldn't return normValues if the field does not have them
+      if (!info.hasNorms()) {
+        if (reader.getNormValues(info.name) != null) {
+          throw new RuntimeException("field: " + info.name + " should omit norms but has them!");
+        }
+      }
+      
+      // reader shouldn't return docValues if the field does not have them
+      // reader shouldn't return multiple docvalues types for the same field.
+      switch(info.getDocValuesType()) {
+        case NONE:
+          if (reader.getBinaryDocValues(info.name) != null ||
+              reader.getNumericDocValues(info.name) != null ||
+              reader.getSortedDocValues(info.name) != null || 
+              reader.getSortedSetDocValues(info.name) != null || 
+              reader.getDocsWithField(info.name) != null) {
+            throw new RuntimeException("field: " + info.name + " has docvalues but should omit them!");
+          }
+          break;
+        case SORTED:
+          if (reader.getBinaryDocValues(info.name) != null ||
+              reader.getNumericDocValues(info.name) != null ||
+              reader.getSortedNumericDocValues(info.name) != null ||
+              reader.getSortedSetDocValues(info.name) != null) {
+            throw new RuntimeException(info.name + " returns multiple docvalues types!");
+          }
+          break;
+        case SORTED_NUMERIC:
+          if (reader.getBinaryDocValues(info.name) != null ||
+              reader.getNumericDocValues(info.name) != null ||
+              reader.getSortedSetDocValues(info.name) != null ||
+              reader.getSortedDocValues(info.name) != null) {
+            throw new RuntimeException(info.name + " returns multiple docvalues types!");
+          }
+          break;
+        case SORTED_SET:
+          if (reader.getBinaryDocValues(info.name) != null ||
+              reader.getNumericDocValues(info.name) != null ||
+              reader.getSortedNumericDocValues(info.name) != null ||
+              reader.getSortedDocValues(info.name) != null) {
+            throw new RuntimeException(info.name + " returns multiple docvalues types!");
+          }
+          break;
+        case BINARY:
+          if (reader.getNumericDocValues(info.name) != null ||
+              reader.getSortedDocValues(info.name) != null ||
+              reader.getSortedNumericDocValues(info.name) != null ||
+              reader.getSortedSetDocValues(info.name) != null) {
+            throw new RuntimeException(info.name + " returns multiple docvalues types!");
+          }
+          break;
+        case NUMERIC:
+          if (reader.getBinaryDocValues(info.name) != null ||
+              reader.getSortedDocValues(info.name) != null ||
+              reader.getSortedNumericDocValues(info.name) != null ||
+              reader.getSortedSetDocValues(info.name) != null) {
+            throw new RuntimeException(info.name + " returns multiple docvalues types!");
+          }
+          break;
+        default:
+          throw new AssertionError();
+      }
+    }
+  }
 
   /** start and end are BOTH inclusive */
   public static int nextInt(Random r, int start, int end) {