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 2013/09/19 22:57:11 UTC

svn commit: r1524840 [4/4] - in /lucene/dev/trunk/lucene: ./ codecs/src/java/org/apache/lucene/codecs/blockterms/ codecs/src/java/org/apache/lucene/codecs/bloom/ codecs/src/java/org/apache/lucene/codecs/memory/ codecs/src/java/org/apache/lucene/codecs/...

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java?rev=1524840&r1=1524839&r2=1524840&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java Thu Sep 19 20:57:09 2013
@@ -23,6 +23,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.EnumSet;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
@@ -30,14 +31,17 @@ import java.util.Map;
 import java.util.NoSuchElementException;
 import java.util.Random;
 import java.util.Set;
+import java.util.SortedMap;
 import java.util.TreeMap;
+import java.util.concurrent.atomic.AtomicLong;
 
+import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.FieldsConsumer;
 import org.apache.lucene.codecs.FieldsProducer;
-import org.apache.lucene.codecs.PostingsConsumer;
-import org.apache.lucene.codecs.TermStats;
-import org.apache.lucene.codecs.TermsConsumer;
+import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.codecs.lucene45.Lucene45Codec;
+import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.index.FieldInfo.DocValuesType;
@@ -49,7 +53,9 @@ import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.Constants;
 import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.LineFileDocs;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util._TestUtil;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -126,6 +132,7 @@ public abstract class BasePostingsFormat
     private final BytesRef payload;
     private final IndexOptions options;
     private final boolean doPositions;
+    private final boolean allowPayloads;
 
     private int docID;
     private int freq;
@@ -138,11 +145,12 @@ public abstract class BasePostingsFormat
     private int posSpacing;
     private int posUpto;
 
-    public SeedPostings(long seed, int minDocFreq, int maxDocFreq, Bits liveDocs, IndexOptions options) {
+    public SeedPostings(long seed, int minDocFreq, int maxDocFreq, Bits liveDocs, IndexOptions options, boolean allowPayloads) {
       random = new Random(seed);
       docRandom = new Random(random.nextLong());
       docFreq = _TestUtil.nextInt(random, minDocFreq, maxDocFreq);
       this.liveDocs = liveDocs;
+      this.allowPayloads = allowPayloads;
 
       // TODO: more realistic to inversely tie this to numDocs:
       maxDocSpacing = _TestUtil.nextInt(random, 1, 100);
@@ -249,6 +257,9 @@ public abstract class BasePostingsFormat
       } else {
         payload.length = 0;
       }
+      if (!allowPayloads) {
+        payload.length = 0;
+      }
 
       startOffset = offset + random.nextInt(5);
       endOffset = startOffset + random.nextInt(10);
@@ -295,7 +306,7 @@ public abstract class BasePostingsFormat
   }
 
   // Holds all postings:
-  private static Map<String,Map<BytesRef,Long>> fields;
+  private static Map<String,SortedMap<BytesRef,Long>> fields;
 
   private static FieldInfos fieldInfos;
 
@@ -307,7 +318,7 @@ public abstract class BasePostingsFormat
   private static long totalPostings;
   private static long totalPayloadBytes;
 
-  private static SeedPostings getSeedPostings(String term, long seed, boolean withLiveDocs, IndexOptions options) {
+  private static SeedPostings getSeedPostings(String term, long seed, boolean withLiveDocs, IndexOptions options, boolean allowPayloads) {
     int minDocFreq, maxDocFreq;
     if (term.startsWith("big_")) {
       minDocFreq = RANDOM_MULTIPLIER * 50000;
@@ -323,14 +334,14 @@ public abstract class BasePostingsFormat
       maxDocFreq = 3;
     }
 
-    return new SeedPostings(seed, minDocFreq, maxDocFreq, withLiveDocs ? globalLiveDocs : null, options);
+    return new SeedPostings(seed, minDocFreq, maxDocFreq, withLiveDocs ? globalLiveDocs : null, options, allowPayloads);
   }
 
   @BeforeClass
   public static void createPostings() throws IOException {
     totalPostings = 0;
     totalPayloadBytes = 0;
-    fields = new TreeMap<String,Map<BytesRef,Long>>();
+    fields = new TreeMap<String,SortedMap<BytesRef,Long>>();
 
     final int numFields = _TestUtil.nextInt(random(), 1, 5);
     if (VERBOSE) {
@@ -351,7 +362,7 @@ public abstract class BasePostingsFormat
                                                 null, DocValuesType.NUMERIC, null);
       fieldUpto++;
 
-      Map<BytesRef,Long> postings = new TreeMap<BytesRef,Long>();
+      SortedMap<BytesRef,Long> postings = new TreeMap<BytesRef,Long>();
       fields.put(field, postings);
       Set<String> seenTerms = new HashSet<String>();
 
@@ -388,7 +399,7 @@ public abstract class BasePostingsFormat
 
         // NOTE: sort of silly: we enum all the docs just to
         // get the maxDoc
-        DocsEnum docsEnum = getSeedPostings(term, termSeed, false, IndexOptions.DOCS_ONLY);
+        DocsEnum docsEnum = getSeedPostings(term, termSeed, false, IndexOptions.DOCS_ONLY, true);
         int doc;
         int lastDoc = 0;
         while((doc = docsEnum.nextDoc()) != DocsEnum.NO_MORE_DOCS) {
@@ -412,7 +423,7 @@ public abstract class BasePostingsFormat
     }
 
     allTerms = new ArrayList<FieldAndTerm>();
-    for(Map.Entry<String,Map<BytesRef,Long>> fieldEnt : fields.entrySet()) {
+    for(Map.Entry<String,SortedMap<BytesRef,Long>> fieldEnt : fields.entrySet()) {
       String field = fieldEnt.getKey();
       for(Map.Entry<BytesRef,Long> termEnt : fieldEnt.getValue().entrySet()) {
         allTerms.add(new FieldAndTerm(field, termEnt.getKey()));
@@ -432,6 +443,206 @@ public abstract class BasePostingsFormat
     globalLiveDocs = null;
   }
 
+  private static class SeedFields extends Fields {
+    final Map<String,SortedMap<BytesRef,Long>> fields;
+    final FieldInfos fieldInfos;
+    final IndexOptions maxAllowed;
+    final boolean allowPayloads;
+
+    public SeedFields(Map<String,SortedMap<BytesRef,Long>> fields, FieldInfos fieldInfos, IndexOptions maxAllowed, boolean allowPayloads) {
+      this.fields = fields;
+      this.fieldInfos = fieldInfos;
+      this.maxAllowed = maxAllowed;
+      this.allowPayloads = allowPayloads;
+    }
+
+    @Override
+    public Iterator<String> iterator() {
+      return fields.keySet().iterator();
+    }
+
+    @Override
+    public Terms terms(String field) {
+      SortedMap<BytesRef,Long> terms = fields.get(field);
+      if (terms == null) {
+        return null;
+      } else {
+        return new SeedTerms(terms, fieldInfos.fieldInfo(field), maxAllowed, allowPayloads);
+      }
+    }
+
+    @Override
+    public int size() {
+      return fields.size();
+    }
+  }
+
+  private static class SeedTerms extends Terms {
+    final SortedMap<BytesRef,Long> terms;
+    final FieldInfo fieldInfo;
+    final IndexOptions maxAllowed;
+    final boolean allowPayloads;
+
+    public SeedTerms(SortedMap<BytesRef,Long> terms, FieldInfo fieldInfo, IndexOptions maxAllowed, boolean allowPayloads) {
+      this.terms = terms;
+      this.fieldInfo = fieldInfo;
+      this.maxAllowed = maxAllowed;
+      this.allowPayloads = allowPayloads;
+    }
+
+    @Override
+    public TermsEnum iterator(TermsEnum reuse) {
+      SeedTermsEnum termsEnum;
+      if (reuse != null && reuse instanceof SeedTermsEnum) {
+        termsEnum = (SeedTermsEnum) reuse;
+        if (termsEnum.terms != terms) {
+          termsEnum = new SeedTermsEnum(terms, maxAllowed, allowPayloads);
+        }
+      } else {
+        termsEnum = new SeedTermsEnum(terms, maxAllowed, allowPayloads);
+      }
+      termsEnum.reset();
+
+      return termsEnum;
+    }
+
+    @Override
+    public long size() {
+      return terms.size();
+    }
+
+    @Override
+    public long getSumTotalTermFreq() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public long getSumDocFreq() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public int getDocCount() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public boolean hasOffsets() {
+      return fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
+    }
+  
+    @Override
+    public boolean hasPositions() {
+      return fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
+    }
+  
+    @Override
+    public boolean hasPayloads() {
+      return fieldInfo.hasPayloads();
+    }
+  }
+
+  private static class SeedTermsEnum extends TermsEnum {
+    final SortedMap<BytesRef,Long> terms;
+    final IndexOptions maxAllowed;
+    final boolean allowPayloads;
+
+    private Iterator<Map.Entry<BytesRef,Long>> iterator;
+
+    private Map.Entry<BytesRef,Long> current;
+
+    public SeedTermsEnum(SortedMap<BytesRef,Long> terms, IndexOptions maxAllowed, boolean allowPayloads) {
+      this.terms = terms;
+      this.maxAllowed = maxAllowed;
+      this.allowPayloads = allowPayloads;
+    }
+
+    void reset() {
+      iterator = terms.entrySet().iterator();
+    }
+
+    @Override
+    public SeekStatus seekCeil(BytesRef text) {
+      SortedMap<BytesRef,Long> tailMap = terms.tailMap(text);
+      if (tailMap.isEmpty()) {
+        return SeekStatus.END;
+      } else {
+        iterator = tailMap.entrySet().iterator();
+        if (tailMap.firstKey().equals(text)) {
+          return SeekStatus.FOUND;
+        } else {
+          return SeekStatus.NOT_FOUND;
+        }
+      }
+    }
+
+    @Override
+    public BytesRef next() {
+      if (iterator.hasNext()) {
+        current = iterator.next();
+        return term();
+      } else {
+        return null;
+      }
+    }
+
+    @Override
+    public void seekExact(long ord) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public BytesRef term() {
+      return current.getKey();
+    }
+
+    @Override
+    public long ord() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public int docFreq() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public long totalTermFreq() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public final DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
+      if (liveDocs != null) {
+        throw new IllegalArgumentException("liveDocs must be null");
+      }
+      if ((flags & DocsEnum.FLAG_FREQS) != 0 && maxAllowed.compareTo(IndexOptions.DOCS_AND_FREQS) < 0) {
+        return null;
+      }
+      return getSeedPostings(current.getKey().utf8ToString(), current.getValue(), false, maxAllowed, allowPayloads);
+    }
+
+    @Override
+    public final DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
+      if (liveDocs != null) {
+        throw new IllegalArgumentException("liveDocs must be null");
+      }
+      if (maxAllowed.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) {
+        System.out.println("no: max");
+        return null;
+      }
+      if ((flags & DocsAndPositionsEnum.FLAG_OFFSETS) != 0 && maxAllowed.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) < 0) {
+        System.out.println("no: offsets");
+        return null;
+      }
+      if ((flags & DocsAndPositionsEnum.FLAG_PAYLOADS) != 0 && allowPayloads == false) {
+        System.out.println("no: payloads");
+        return null;
+      }
+      return getSeedPostings(current.getKey().utf8ToString(), current.getValue(), false, maxAllowed, allowPayloads);
+    }
+  }
+
   // TODO maybe instead of @BeforeClass just make a single test run: build postings & index & test it?
 
   private FieldInfos currentFieldInfos;
@@ -489,79 +700,10 @@ public abstract class BasePostingsFormat
     SegmentWriteState writeState = new SegmentWriteState(null, dir,
                                                          segmentInfo, newFieldInfos,
                                                          null, new IOContext(new FlushInfo(maxDoc, bytes)));
-    FieldsConsumer fieldsConsumer = codec.postingsFormat().fieldsConsumer(writeState);
-
-    for(Map.Entry<String,Map<BytesRef,Long>> fieldEnt : fields.entrySet()) {
-      String field = fieldEnt.getKey();
-      Map<BytesRef,Long> terms = fieldEnt.getValue();
-
-      FieldInfo fieldInfo = newFieldInfos.fieldInfo(field);
-
-      IndexOptions indexOptions = fieldInfo.getIndexOptions();
-
-      if (VERBOSE) {
-        System.out.println("field=" + field + " indexOtions=" + indexOptions);
-      }
-
-      boolean doFreq = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS) >= 0;
-      boolean doPos = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
-      boolean doPayloads = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0 && allowPayloads;
-      boolean doOffsets = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
-      
-      TermsConsumer termsConsumer = fieldsConsumer.addField(fieldInfo);
-      long sumTotalTF = 0;
-      long sumDF = 0;
-      FixedBitSet seenDocs = new FixedBitSet(maxDoc);
-      for(Map.Entry<BytesRef,Long> termEnt : terms.entrySet()) {
-        BytesRef term = termEnt.getKey();
-        SeedPostings postings = getSeedPostings(term.utf8ToString(), termEnt.getValue(), false, maxAllowed);
-        if (VERBOSE) {
-          System.out.println("  term=" + field + ":" + term.utf8ToString() + " docFreq=" + postings.docFreq + " seed=" + termEnt.getValue());
-        }
-        
-        PostingsConsumer postingsConsumer = termsConsumer.startTerm(term);
-        long totalTF = 0;
-        int docID = 0;
-        while((docID = postings.nextDoc()) != DocsEnum.NO_MORE_DOCS) {
-          final int freq = postings.freq();
-          if (VERBOSE) {
-            System.out.println("    " + postings.upto + ": docID=" + docID + " freq=" + postings.freq);
-          }
-          postingsConsumer.startDoc(docID, doFreq ? postings.freq : -1);
-          seenDocs.set(docID);
-          if (doPos) {
-            totalTF += postings.freq;
-            for(int posUpto=0;posUpto<freq;posUpto++) {
-              int pos = postings.nextPosition();
-              BytesRef payload = postings.getPayload();
 
-              if (VERBOSE) {
-                if (doPayloads) {
-                  System.out.println("      pos=" + pos + " payload=" + (payload == null ? "null" : payload.length + " bytes"));
-                } else {
-                  System.out.println("      pos=" + pos);
-                }
-              }
-              postingsConsumer.addPosition(pos, doPayloads ? payload : null,
-                                           doOffsets ? postings.startOffset() : -1,
-                                           doOffsets ? postings.endOffset() : -1);
-            }
-          } else if (doFreq) {
-            totalTF += freq;
-          } else {
-            totalTF++;
-          }
-          postingsConsumer.finishDoc();
-        }
-        termsConsumer.finishTerm(term, new TermStats(postings.docFreq, doFreq ? totalTF : -1));
-        sumTotalTF += totalTF;
-        sumDF += postings.docFreq;
-      }
-
-      termsConsumer.finish(doFreq ? sumTotalTF : -1, sumDF, seenDocs.cardinality());
-    }
+    Fields seedFields = new SeedFields(fields, newFieldInfos, maxAllowed, allowPayloads);
 
-    fieldsConsumer.close();
+    codec.postingsFormat().fieldsConsumer(writeState).write(seedFields);
 
     if (VERBOSE) {
       System.out.println("TEST: after indexing: files=");
@@ -625,7 +767,8 @@ public abstract class BasePostingsFormat
     SeedPostings expected = getSeedPostings(term.utf8ToString(), 
                                             fields.get(field).get(term),
                                             useLiveDocs,
-                                            maxIndexOptions);
+                                            maxIndexOptions,
+                                            true);
     assertEquals(expected.docFreq, termsEnum.docFreq());
 
     boolean allowFreqs = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0 &&
@@ -1224,4 +1367,197 @@ public abstract class BasePostingsFormat
     iw.close();
     dir.close();
   }
+
+  private static class TermFreqs {
+    long totalTermFreq;
+    int docFreq;
+  };
+
+  // LUCENE-5123: make sure we can visit postings twice
+  // during flush/merge
+  public void testInvertedWrite() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
+
+    final Map<String,TermFreqs> termFreqs = new HashMap<String,TermFreqs>();
+    final AtomicLong sumDocFreq = new AtomicLong();
+    final AtomicLong sumTotalTermFreq = new AtomicLong();
+
+    // TODO: would be better to use / delegate to the current
+    // Codec returned by getCodec()
+
+    iwc.setCodec(new Lucene45Codec() {
+        @Override
+        public PostingsFormat getPostingsFormatForField(String field) {
+
+          PostingsFormat p = getCodec().postingsFormat();
+          if (p instanceof PerFieldPostingsFormat) {
+            p = ((PerFieldPostingsFormat) p).getPostingsFormatForField(field);
+          }
+          final PostingsFormat defaultPostingsFormat = p;
+
+          if (field.equals("body")) {
+
+            // A PF that counts up some stats and then in
+            // the end we verify the stats match what the
+            // final IndexReader says, just to exercise the
+            // new freedom of iterating the postings more
+            // than once at flush/merge:
+
+            return new PostingsFormat(defaultPostingsFormat.getName()) {
+
+              @Override
+              public FieldsConsumer fieldsConsumer(final SegmentWriteState state) throws IOException {
+
+                final FieldsConsumer fieldsConsumer = defaultPostingsFormat.fieldsConsumer(state);
+
+                return new FieldsConsumer() {
+                  @Override
+                  public void write(Fields fields) throws IOException {
+                    fieldsConsumer.write(fields);
+
+                    boolean isMerge = state.context.context == IOContext.Context.MERGE;
+
+                    boolean addOnSecondPass = random().nextBoolean();
+
+                    //System.out.println("write isMerge=" + isMerge + " 2ndPass=" + addOnSecondPass);
+
+                    // Gather our own stats:
+                    Terms terms = fields.terms("body");
+                    assert terms != null;
+
+                    TermsEnum termsEnum = terms.iterator(null);
+                    DocsEnum docs = null;
+                    while(termsEnum.next() != null) {
+                      BytesRef term = termsEnum.term();
+                      if (random().nextBoolean()) {
+                        docs = termsEnum.docs(null, docs, DocsEnum.FLAG_FREQS);
+                      } else if (docs instanceof DocsAndPositionsEnum) {
+                        docs = termsEnum.docsAndPositions(null, (DocsAndPositionsEnum) docs, 0);
+                      } else {
+                        docs = termsEnum.docsAndPositions(null, null, 0);
+                      }
+                      int docFreq = 0;
+                      long totalTermFreq = 0;
+                      while (docs.nextDoc() != DocsEnum.NO_MORE_DOCS) {
+                        docFreq++;
+                        totalTermFreq += docs.freq();
+                        if (docs instanceof DocsAndPositionsEnum) {
+                          DocsAndPositionsEnum posEnum = (DocsAndPositionsEnum) docs;
+                          int limit = _TestUtil.nextInt(random(), 1, docs.freq());
+                          for(int i=0;i<limit;i++) {
+                            posEnum.nextPosition();
+                          }
+                        }
+                      }
+
+                      String termString = term.utf8ToString();
+
+                      // During merge we should only see terms
+                      // we had already seen during flush:
+                      assertTrue(isMerge==false || termFreqs.containsKey(termString));
+
+                      if (isMerge == false && addOnSecondPass == false) {
+                        TermFreqs tf = termFreqs.get(termString);
+                        if (tf == null) {
+                          tf = new TermFreqs();
+                          termFreqs.put(termString, tf);
+                        }
+                        tf.docFreq += docFreq;
+                        tf.totalTermFreq += totalTermFreq;
+                        sumDocFreq.addAndGet(docFreq);
+                        sumTotalTermFreq.addAndGet(totalTermFreq);
+                      } else if (termFreqs.containsKey(termString) == false) {
+                        termFreqs.put(termString, new TermFreqs());
+                      }
+                    }
+
+                    // Also test seeking the TermsEnum:
+                    for(String term : termFreqs.keySet()) {
+                      if (termsEnum.seekExact(new BytesRef(term))) {
+                        if (random().nextBoolean()) {
+                          docs = termsEnum.docs(null, docs, DocsEnum.FLAG_FREQS);
+                        } else if (docs instanceof DocsAndPositionsEnum) {
+                          docs = termsEnum.docsAndPositions(null, (DocsAndPositionsEnum) docs, 0);
+                        } else {
+                          docs = termsEnum.docsAndPositions(null, null, 0);
+                        }
+
+                        int docFreq = 0;
+                        long totalTermFreq = 0;
+                        while (docs.nextDoc() != DocsEnum.NO_MORE_DOCS) {
+                          docFreq++;
+                          totalTermFreq += docs.freq();
+                          if (docs instanceof DocsAndPositionsEnum) {
+                            DocsAndPositionsEnum posEnum = (DocsAndPositionsEnum) docs;
+                            int limit = _TestUtil.nextInt(random(), 1, docs.freq());
+                            for(int i=0;i<limit;i++) {
+                              posEnum.nextPosition();
+                            }
+                          }
+                        }
+
+                        if (isMerge == false && addOnSecondPass) {
+                          TermFreqs tf = termFreqs.get(term);
+                          if (tf == null) {
+                            tf = new TermFreqs();
+                            termFreqs.put(term, tf);
+                          }
+                          tf.docFreq += docFreq;
+                          tf.totalTermFreq += totalTermFreq;
+                          sumDocFreq.addAndGet(docFreq);
+                          sumTotalTermFreq.addAndGet(totalTermFreq);
+                        }
+
+                        //System.out.println("  term=" + term + " docFreq=" + docFreq + " ttDF=" + termToDocFreq.get(term));
+                        assertTrue(docFreq <= termFreqs.get(term).docFreq);
+                        assertTrue(totalTermFreq <= termFreqs.get(term).totalTermFreq);
+                      }
+                    }
+                  }
+                };
+              }
+
+              @Override
+              public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
+                return defaultPostingsFormat.fieldsProducer(state);
+              }
+            };
+          } else {
+            return defaultPostingsFormat;
+          }
+        }
+      });
+
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
+
+    LineFileDocs docs = new LineFileDocs(random());
+    int bytesToIndex = atLeast(100) * 1024;
+    int bytesIndexed = 0;
+    while (bytesIndexed < bytesToIndex) {
+      Document doc = docs.nextDoc();
+      w.addDocument(doc);
+      bytesIndexed += RamUsageEstimator.sizeOf(doc);
+    }
+
+    IndexReader r = w.getReader();
+    w.close();
+
+    Terms terms = MultiFields.getTerms(r, "body");
+    assertEquals(sumDocFreq.get(), terms.getSumDocFreq());
+    assertEquals(sumTotalTermFreq.get(), terms.getSumTotalTermFreq());
+
+    TermsEnum termsEnum = terms.iterator(null);
+    long termCount = 0;
+    while(termsEnum.next() != null) {
+      BytesRef term = termsEnum.term();
+      termCount++;
+      assertEquals(termFreqs.get(term.utf8ToString()).docFreq, termsEnum.docFreq());
+      assertEquals(termFreqs.get(term.utf8ToString()).totalTermFreq, termsEnum.totalTermFreq());
+    }
+    assertEquals(termFreqs.size(), termCount);
+
+    r.close();
+    dir.close();
+  }
 }

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BaseTermVectorsFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BaseTermVectorsFormatTestCase.java?rev=1524840&r1=1524839&r2=1524840&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BaseTermVectorsFormatTestCase.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BaseTermVectorsFormatTestCase.java Thu Sep 19 20:57:09 2013
@@ -406,7 +406,7 @@ public abstract class BaseTermVectorsFor
       uniqueTerms.add(new BytesRef(term));
     }
     final BytesRef[] sortedTerms = uniqueTerms.toArray(new BytesRef[0]);
-    Arrays.sort(sortedTerms, terms.getComparator());
+    Arrays.sort(sortedTerms);
     final TermsEnum termsEnum = terms.iterator(random().nextBoolean() ? null : this.termsEnum.get());
     this.termsEnum.set(termsEnum);
     for (int i = 0; i < sortedTerms.length; ++i) {

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java?rev=1524840&r1=1524839&r2=1524840&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java Thu Sep 19 20:57:09 2013
@@ -1486,7 +1486,6 @@ public abstract class LuceneTestCase ext
    * checks collection-level statistics on Terms 
    */
   public void assertTermsStatisticsEquals(String info, Terms leftTerms, Terms rightTerms) throws IOException {
-    assert leftTerms.getComparator() == rightTerms.getComparator();
     if (leftTerms.getDocCount() != -1 && rightTerms.getDocCount() != -1) {
       assertEquals(info, leftTerms.getDocCount(), rightTerms.getDocCount());
     }