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 2012/10/02 12:43:16 UTC

svn commit: r1392829 - in /lucene/dev/branches/branch_4x: ./ lucene/ lucene/codecs/ lucene/codecs/src/test/org/apache/lucene/codecs/memory/ lucene/core/ lucene/core/src/test/org/apache/lucene/codecs/perfield/ lucene/core/src/test/org/apache/lucene/inde...

Author: mikemccand
Date: Tue Oct  2 10:43:15 2012
New Revision: 1392829

URL: http://svn.apache.org/viewvc?rev=1392829&view=rev
Log:
rework BasePostingsFormatTestCase to not use so much CPU/RAM

Modified:
    lucene/dev/branches/branch_4x/   (props changed)
    lucene/dev/branches/branch_4x/lucene/   (props changed)
    lucene/dev/branches/branch_4x/lucene/codecs/   (props changed)
    lucene/dev/branches/branch_4x/lucene/codecs/src/test/org/apache/lucene/codecs/memory/TestDirectPostingsFormat.java
    lucene/dev/branches/branch_4x/lucene/core/   (props changed)
    lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat.java
    lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestPostingsFormat.java
    lucene/dev/branches/branch_4x/lucene/test-framework/   (props changed)
    lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java

Modified: lucene/dev/branches/branch_4x/lucene/codecs/src/test/org/apache/lucene/codecs/memory/TestDirectPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/codecs/src/test/org/apache/lucene/codecs/memory/TestDirectPostingsFormat.java?rev=1392829&r1=1392828&r2=1392829&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/codecs/src/test/org/apache/lucene/codecs/memory/TestDirectPostingsFormat.java (original)
+++ lucene/dev/branches/branch_4x/lucene/codecs/src/test/org/apache/lucene/codecs/memory/TestDirectPostingsFormat.java Tue Oct  2 10:43:15 2012
@@ -21,15 +21,14 @@ import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.lucene40.Lucene40Codec;
 import org.apache.lucene.index.BasePostingsFormatTestCase;
-import org.junit.Ignore;
 
 /**
  * Tests DirectPostingsFormat
  */
-@Ignore("Put this test back once we fix OOMEs")
 public class TestDirectPostingsFormat extends BasePostingsFormatTestCase {
   // TODO: randomize parameters
   private final PostingsFormat postings = new DirectPostingsFormat();
+
   private final Codec codec = new Lucene40Codec() {
     @Override
     public PostingsFormat getPostingsFormatForField(String field) {

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat.java?rev=1392829&r1=1392828&r2=1392829&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat.java Tue Oct  2 10:43:15 2012
@@ -33,9 +33,7 @@ public class TestPerFieldPostingsFormat 
   @Override
   public void setUp() throws Exception {
     super.setUp();
-    // TODO: re-enable once we fix OOMEs in DirectPF
-    //codec = new RandomCodec(new Random(random().nextLong()), Collections.EMPTY_SET);
-    codec = new RandomCodec(new Random(random().nextLong()), Collections.singleton("Direct"));
+    codec = new RandomCodec(new Random(random().nextLong()), Collections.EMPTY_SET);
   }
   
   @Override

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestPostingsFormat.java?rev=1392829&r1=1392828&r2=1392829&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestPostingsFormat.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestPostingsFormat.java Tue Oct  2 10:43:15 2012
@@ -17,43 +17,11 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.EnumSet;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.NoSuchElementException;
-import java.util.Set;
-import java.util.TreeMap;
-
 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.index.FieldInfo.IndexOptions;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.FlushInfo;
-import org.apache.lucene.store.IOContext;
-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.LuceneTestCase.SuppressCodecs;
-import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util._TestUtil;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
 
 /** Tests the codec configuration defined by LuceneTestCase randomly
  *  (typically a mix across different fields).
  */
-@SuppressCodecs({"Direct"}) // Put back once we fix DirectPF's OOMEs
 public class TestPostingsFormat extends BasePostingsFormatTestCase {
 
   @Override

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java?rev=1392829&r1=1392828&r2=1392829&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java Tue Oct  2 10:43:15 2012
@@ -28,6 +28,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.NoSuchElementException;
+import java.util.Random;
 import java.util.Set;
 import java.util.TreeMap;
 
@@ -107,8 +108,179 @@ public abstract class BasePostingsFormat
     PAYLOADS,
 
     // Test w/ multiple threads
-    THREADS};
+    THREADS
+  };
 
+  /** Given the same random seed this always enumerates the
+   *  same random postings */
+  private static class SeedPostings extends DocsAndPositionsEnum {
+    // Used only to generate docIDs; this way if you pull w/
+    // or w/o positions you get the same docID sequence:
+    private final Random docRandom;
+    private final Random random;
+    public int docFreq;
+    private final int maxDocSpacing;
+    private final int payloadSize;
+    private final boolean fixedPayloads;
+    private final Bits liveDocs;
+    private final BytesRef payload;
+    private final IndexOptions options;
+    private final boolean doPositions;
+
+    private int docID;
+    private int freq;
+    public int upto;
+
+    private int pos;
+    private int offset;
+    private int startOffset;
+    private int endOffset;
+    private int posSpacing;
+    private int posUpto;
+
+    public SeedPostings(long seed, int minDocFreq, int maxDocFreq, Bits liveDocs, IndexOptions options) {
+      random = new Random(seed);
+      docRandom = new Random(random.nextLong());
+      docFreq = _TestUtil.nextInt(random, minDocFreq, maxDocFreq);
+      this.liveDocs = liveDocs;
+
+      // TODO: more realistic to inversely tie this to numDocs:
+      maxDocSpacing = _TestUtil.nextInt(random, 1, 100);
+
+      if (random.nextInt(10) == 7) {
+        // 10% of the time create big payloads:
+        payloadSize = 1 + random.nextInt(3);
+      } else {
+        payloadSize = 1 + random.nextInt(1);
+      }
+
+      fixedPayloads = random.nextBoolean();
+      byte[] payloadBytes = new byte[payloadSize];
+      payload = new BytesRef(payloadBytes);
+      this.options = options;
+      doPositions = IndexOptions.DOCS_AND_FREQS_AND_POSITIONS.compareTo(options) <= 0;
+    }
+
+    @Override
+    public int nextDoc() {
+      while(true) {
+        _nextDoc();
+        if (liveDocs == null || docID == NO_MORE_DOCS || liveDocs.get(docID)) {
+          return docID;
+        }
+      }
+    }
+
+    private int _nextDoc() {
+      // Must consume random:
+      while(posUpto < freq) {
+        nextPosition();
+      }
+
+      if (upto < docFreq) {
+        if (upto == 0 && docRandom.nextBoolean()) {
+          // Sometimes index docID = 0
+        } else if (maxDocSpacing == 1) {
+          docID++;
+        } else {
+          // TODO: sometimes have a biggish gap here!
+          docID += _TestUtil.nextInt(docRandom, 1, maxDocSpacing);
+        }
+
+        if (random.nextInt(200) == 17) {
+          freq = _TestUtil.nextInt(random, 1, 1000);
+        } else if (random.nextInt(10) == 17) {
+          freq = _TestUtil.nextInt(random, 1, 20);
+        } else {
+          freq = _TestUtil.nextInt(random, 1, 4);
+        }
+
+        pos = 0;
+        offset = 0;
+        posUpto = 0;
+        posSpacing = _TestUtil.nextInt(random, 1, 100);
+
+        upto++;
+        return docID;
+      } else {
+        return docID = NO_MORE_DOCS;
+      }
+    }
+
+    @Override
+    public int docID() {
+      return docID;
+    }
+
+    @Override
+    public int freq() {
+      return freq;
+    }
+
+    @Override
+    public int nextPosition() {
+      if (!doPositions) {
+        posUpto = freq;
+        return 0;
+      }
+      assert posUpto < freq;
+
+      if (posUpto == 0 && random.nextBoolean()) {
+        // Sometimes index pos = 0
+      } else if (posSpacing == 1) {
+        pos++;
+      } else {
+        pos += _TestUtil.nextInt(random, 1, posSpacing);
+      }
+
+      if (payloadSize != 0) {
+        if (fixedPayloads) {
+          payload.length = payloadSize;
+          random.nextBytes(payload.bytes); 
+        } else {
+          int thisPayloadSize = random.nextInt(payloadSize);
+          if (thisPayloadSize != 0) {
+            payload.length = payloadSize;
+            random.nextBytes(payload.bytes); 
+          } else {
+            payload.length = 0;
+          }
+        } 
+      } else {
+        payload.length = 0;
+      }
+
+      startOffset = offset + random.nextInt(5);
+      endOffset = startOffset + random.nextInt(10);
+      offset = endOffset;
+
+      posUpto++;
+      return pos;
+    }
+  
+    @Override
+    public int startOffset() {
+      return startOffset;
+    }
+
+    @Override
+    public int endOffset() {
+      return endOffset;
+    }
+
+    @Override
+    public BytesRef getPayload() {
+      return payload.length == 0 ? null : payload;
+    }
+
+    @Override
+    public int advance(int target) {
+      while(nextDoc() < target) {
+      }
+      return docID;
+    }
+  }
+  
   private static class FieldAndTerm {
     String field;
     BytesRef term;
@@ -119,57 +291,52 @@ public abstract class BasePostingsFormat
     }
   }
 
-  private static class Position {
-    int position;
-    byte[] payload;
-    int startOffset;
-    int endOffset;
-  }
-
-  private static class Posting implements Comparable<Posting> {
-    int docID;
-    List<Position> positions;
-
-    public int compareTo(Posting other) {
-      return docID - other.docID;
-    }
-  }
-
   // Holds all postings:
-  private static Map<String,Map<BytesRef,List<Posting>>> fields;
-
-  // Holds only live doc postings:
-  private static Map<String,Map<BytesRef,List<Posting>>> fieldsLive;
+  private static Map<String,Map<BytesRef,Long>> fields;
 
   private static FieldInfos fieldInfos;
 
-  private static int maxDocID;
-
   private static FixedBitSet globalLiveDocs;
 
   private static List<FieldAndTerm> allTerms;
+  private static int maxDoc;
 
   private static long totalPostings;
   private static long totalPayloadBytes;
 
+  private static SeedPostings getSeedPostings(String term, long seed, boolean withLiveDocs, IndexOptions options) {
+    int minDocFreq, maxDocFreq;
+    if (term.startsWith("big_")) {
+      minDocFreq = RANDOM_MULTIPLIER * 50000;
+      maxDocFreq = RANDOM_MULTIPLIER * 70000;
+    } else if (term.startsWith("medium_")) {
+      minDocFreq = RANDOM_MULTIPLIER * 3000;
+      maxDocFreq = RANDOM_MULTIPLIER * 6000;
+    } else if (term.startsWith("low_")) {
+      minDocFreq = RANDOM_MULTIPLIER;
+      maxDocFreq = RANDOM_MULTIPLIER * 40;
+    } else {
+      minDocFreq = 1;
+      maxDocFreq = 3;
+    }
+
+    return new SeedPostings(seed, minDocFreq, maxDocFreq, withLiveDocs ? globalLiveDocs : null, options);
+  }
+
   @BeforeClass
   public static void createPostings() throws IOException {
-    maxDocID = 0;
     totalPostings = 0;
     totalPayloadBytes = 0;
-    fields = new TreeMap<String,Map<BytesRef,List<Posting>>>();
-    fieldsLive = new TreeMap<String,Map<BytesRef,List<Posting>>>();
+    fields = new TreeMap<String,Map<BytesRef,Long>>();
 
     final int numFields = _TestUtil.nextInt(random(), 1, 5);
     if (VERBOSE) {
       System.out.println("TEST: " + numFields + " fields");
     }
+    maxDoc = 0;
 
     FieldInfo[] fieldInfoArray = new FieldInfo[numFields];
     int fieldUpto = 0;
-    int numMediumTerms = 0;
-    int numBigTerms = 0;
-    int numManyPositions = 0;
     while (fieldUpto < numFields) {
       String field = _TestUtil.randomSimpleString(random());
       if (fields.containsKey(field)) {
@@ -181,7 +348,7 @@ public abstract class BasePostingsFormat
                                                 null, DocValues.Type.FIXED_INTS_8, null);
       fieldUpto++;
 
-      Map<BytesRef,List<Posting>> postings = new TreeMap<BytesRef,List<Posting>>();
+      Map<BytesRef,Long> postings = new TreeMap<BytesRef,Long>();
       fields.put(field, postings);
       Set<String> seenTerms = new HashSet<String>();
 
@@ -196,145 +363,58 @@ public abstract class BasePostingsFormat
         }
         seenTerms.add(term);
 
-        int numDocs;
-        if (numBigTerms == 0 || (random().nextInt(10) == 3 && numBigTerms < 2)) {
-          // Make at least 1 big term, then maybe (~10%
-          // chance) make another:
-          numDocs = RANDOM_MULTIPLIER * _TestUtil.nextInt(random(), 50000, 70000);
-          numBigTerms++;
+        if (TEST_NIGHTLY && termUpto == 0 && fieldUpto == 1) {
+          // Make 1 big term:
           term = "big_" + term;
-        } else if (numMediumTerms == 0 || (random().nextInt(10) == 3 && numMediumTerms < 5)) {
-          // Make at least 1 medium term, then maybe (~10%
-          // chance) make up to 4 more:
-          numDocs = RANDOM_MULTIPLIER * _TestUtil.nextInt(random(), 3000, 6000);
-          numMediumTerms++;
+        } else if (termUpto == 1 && fieldUpto == 1) {
+          // Make 1 medium term:
           term = "medium_" + term;
         } else if (random().nextBoolean()) {
           // Low freq term:
-          numDocs = RANDOM_MULTIPLIER * _TestUtil.nextInt(random(), 1, 40);
           term = "low_" + term;
         } else {
           // Very low freq term (don't multiply by RANDOM_MULTIPLIER):
-          numDocs = _TestUtil.nextInt(random(), 1, 3);
           term = "verylow_" + term;
         }
 
-        List<Posting> termPostings = new ArrayList<Posting>();
-        postings.put(new BytesRef(term), termPostings);
+        long termSeed = random().nextLong();
+        postings.put(new BytesRef(term), termSeed);
 
-        int docID = 0;
-
-        // TODO: more realistic to inversely tie this to numDocs:
-        int maxDocSpacing = _TestUtil.nextInt(random(), 1, 100);
-
-        int payloadSize;
-        if (random().nextInt(10) == 7) {
-          // 10% of the time create big payloads:
-          payloadSize = 1 + random().nextInt(3);
-        } else {
-          payloadSize = 1 + random().nextInt(1);
-        }
-
-        boolean fixedPayloads = random().nextBoolean();
-
-        for(int docUpto=0;docUpto<numDocs;docUpto++) {
-          if (docUpto == 0 && random().nextBoolean()) {
-            // Sometimes index docID = 0
-          } else if (maxDocSpacing == 1) {
-            docID++;
-          } else {
-            // TODO: sometimes have a biggish gap here!
-            docID += _TestUtil.nextInt(random(), 1, maxDocSpacing);
-          }
-
-          Posting posting = new Posting();
-          posting.docID = docID;
-          maxDocID = Math.max(docID, maxDocID);
-          posting.positions = new ArrayList<Position>();
-          termPostings.add(posting);
-
-          int freq;
-          if (random().nextInt(30) == 17 && numManyPositions < 5) {
-            freq = _TestUtil.nextInt(random(), 1, 1000);
-            numManyPositions++;
-          } else {
-            freq = _TestUtil.nextInt(random(), 1, 20);
-          }
-          int pos = 0;
-          int offset = 0;
-          int posSpacing = _TestUtil.nextInt(random(), 1, 100);
-          totalPostings += freq;
-          for(int posUpto=0;posUpto<freq;posUpto++) {
-            if (posUpto == 0 && random().nextBoolean()) {
-              // Sometimes index pos = 0
-            } else if (posSpacing == 1) {
-              pos++;
-            } else {
-              pos += _TestUtil.nextInt(random(), 1, posSpacing);
-            }
-
-            Position position = new Position();
-            posting.positions.add(position);
-            position.position = pos;
-            if (payloadSize != 0) {
-              if (fixedPayloads) {
-                position.payload = new byte[payloadSize];
-              } else {
-                int thisPayloadSize = random().nextInt(payloadSize);
-                if (thisPayloadSize != 0) {
-                  position.payload = new byte[thisPayloadSize];
-                }
-              }
-            }
-
-            if (position.payload != null) {
-              random().nextBytes(position.payload); 
-              totalPayloadBytes += position.payload.length;
-            }
-
-            position.startOffset = offset + random().nextInt(5);
-            position.endOffset = position.startOffset + random().nextInt(10);
-            offset = position.endOffset;
-          }
+        // NOTE: sort of silly: we enum all the docs just to
+        // get the maxDoc
+        DocsEnum docsEnum = getSeedPostings(term, termSeed, false, IndexOptions.DOCS_ONLY);
+        int doc;
+        int lastDoc = 0;
+        while((doc = docsEnum.nextDoc()) != DocsEnum.NO_MORE_DOCS) {
+          lastDoc = doc;
         }
+        maxDoc = Math.max(lastDoc, maxDoc);
       }
     }
 
     fieldInfos = new FieldInfos(fieldInfoArray);
 
-    globalLiveDocs = new FixedBitSet(1+maxDocID);
+    // It's the count, not the last docID:
+    maxDoc++;
+
+    globalLiveDocs = new FixedBitSet(maxDoc);
     double liveRatio = random().nextDouble();
-    for(int i=0;i<1+maxDocID;i++) {
+    for(int i=0;i<maxDoc;i++) {
       if (random().nextDouble() <= liveRatio) {
         globalLiveDocs.set(i);
       }
     }
 
-    // Pre-filter postings by globalLiveDocs:
-    for(Map.Entry<String,Map<BytesRef,List<Posting>>> fieldEnt : fields.entrySet()) {
-      Map<BytesRef,List<Posting>> postingsLive = new TreeMap<BytesRef,List<Posting>>();
-      fieldsLive.put(fieldEnt.getKey(), postingsLive);
-      for(Map.Entry<BytesRef,List<Posting>> termEnt : fieldEnt.getValue().entrySet()) {
-        List<Posting> termPostingsLive = new ArrayList<Posting>();
-        postingsLive.put(termEnt.getKey(), termPostingsLive);
-        for(Posting posting : termEnt.getValue()) {
-          if (globalLiveDocs.get(posting.docID)) {
-            termPostingsLive.add(posting);
-          }
-        }
-      }
-    }
-
     allTerms = new ArrayList<FieldAndTerm>();
-    for(Map.Entry<String,Map<BytesRef,List<Posting>>> fieldEnt : fields.entrySet()) {
+    for(Map.Entry<String,Map<BytesRef,Long>> fieldEnt : fields.entrySet()) {
       String field = fieldEnt.getKey();
-      for(Map.Entry<BytesRef,List<Posting>> termEnt : fieldEnt.getValue().entrySet()) {
+      for(Map.Entry<BytesRef,Long> termEnt : fieldEnt.getValue().entrySet()) {
         allTerms.add(new FieldAndTerm(field, termEnt.getKey()));
       }
     }
 
     if (VERBOSE) {
-      System.out.println("TEST: done init postings; maxDocID=" + maxDocID + "; " + allTerms.size() + " total terms, across " + fieldInfos.size() + " fields");
+      System.out.println("TEST: done init postings; " + allTerms.size() + " total terms, across " + fieldInfos.size() + " fields");
     }
   }
   
@@ -343,7 +423,6 @@ public abstract class BasePostingsFormat
     allTerms = null;
     fieldInfos = null;
     fields = null;
-    fieldsLive = null;
     globalLiveDocs = null;
   }
 
@@ -355,7 +434,7 @@ public abstract class BasePostingsFormat
   // randomly index at lower IndexOption
   private FieldsProducer buildIndex(Directory dir, IndexOptions maxAllowed, boolean allowPayloads, boolean alwaysTestMax) throws IOException {
     Codec codec = getCodec();
-    SegmentInfo segmentInfo = new SegmentInfo(dir, Constants.LUCENE_MAIN_VERSION, "_0", 1+maxDocID, false, codec, null, null);
+    SegmentInfo segmentInfo = new SegmentInfo(dir, Constants.LUCENE_MAIN_VERSION, "_0", maxDoc, false, codec, null, null);
 
     int maxIndexOption = Arrays.asList(IndexOptions.values()).indexOf(maxAllowed);
     if (VERBOSE) {
@@ -403,12 +482,12 @@ public abstract class BasePostingsFormat
 
     SegmentWriteState writeState = new SegmentWriteState(null, dir,
                                                          segmentInfo, newFieldInfos,
-                                                         32, null, new IOContext(new FlushInfo(1+maxDocID, bytes)));
+                                                         32, null, new IOContext(new FlushInfo(maxDoc, bytes)));
     FieldsConsumer fieldsConsumer = codec.postingsFormat().fieldsConsumer(writeState);
 
-    for(Map.Entry<String,Map<BytesRef,List<Posting>>> fieldEnt : fields.entrySet()) {
+    for(Map.Entry<String,Map<BytesRef,Long>> fieldEnt : fields.entrySet()) {
       String field = fieldEnt.getKey();
-      Map<BytesRef,List<Posting>> terms = fieldEnt.getValue();
+      Map<BytesRef,Long> terms = fieldEnt.getValue();
 
       FieldInfo fieldInfo = newFieldInfos.fieldInfo(field);
 
@@ -426,46 +505,51 @@ public abstract class BasePostingsFormat
       TermsConsumer termsConsumer = fieldsConsumer.addField(fieldInfo);
       long sumTotalTF = 0;
       long sumDF = 0;
-      FixedBitSet seenDocs = new FixedBitSet(maxDocID+1);
-      for(Map.Entry<BytesRef,List<Posting>> termEnt : terms.entrySet()) {
+      FixedBitSet seenDocs = new FixedBitSet(maxDoc);
+      for(Map.Entry<BytesRef,Long> termEnt : terms.entrySet()) {
         BytesRef term = termEnt.getKey();
-        List<Posting> postings = termEnt.getValue();
+        SeedPostings postings = getSeedPostings(term.utf8ToString(), termEnt.getValue(), false, maxAllowed);
         if (VERBOSE) {
-          System.out.println("  term=" + field + ":" + term.utf8ToString() + " docFreq=" + postings.size());
+          System.out.println("  term=" + field + ":" + term.utf8ToString() + " docFreq=" + postings.docFreq + " seed=" + termEnt.getValue());
         }
         
         PostingsConsumer postingsConsumer = termsConsumer.startTerm(term);
         long totalTF = 0;
-        int docCount = 0;
-        for(Posting posting : postings) {
+        int docID = 0;
+        while((docID = postings.nextDoc()) != DocsEnum.NO_MORE_DOCS) {
+          final int freq = postings.freq();
           if (VERBOSE) {
-            System.out.println("    " + docCount + ": docID=" + posting.docID + " freq=" + posting.positions.size());
+            System.out.println("    " + postings.upto + ": docID=" + docID + " freq=" + postings.freq);
           }
-          postingsConsumer.startDoc(posting.docID, doFreq ? posting.positions.size() : -1);
-          seenDocs.set(posting.docID);
+          postingsConsumer.startDoc(docID, doFreq ? postings.freq : -1);
+          seenDocs.set(docID);
           if (doPos) {
-            totalTF += posting.positions.size();
-            for(Position pos : posting.positions) {
+            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.position + " payload=" + (pos.payload == null ? "null" : pos.payload.length + " bytes"));
+                  System.out.println("      pos=" + pos + " payload=" + (payload == null ? "null" : payload.length + " bytes"));
                 } else {
-                  System.out.println("      pos=" + pos.position);
+                  System.out.println("      pos=" + pos);
                 }
               }
-              postingsConsumer.addPosition(pos.position, (doPayloads && pos.payload != null) ? new BytesRef(pos.payload) : null, doOffsets ? pos.startOffset : -1, doOffsets ? pos.endOffset : -1);
+              postingsConsumer.addPosition(pos, doPayloads ? payload : null,
+                                           doOffsets ? postings.startOffset() : -1,
+                                           doOffsets ? postings.endOffset() : -1);
             }
           } else if (doFreq) {
-            totalTF += posting.positions.size();
+            totalTF += freq;
           } else {
             totalTF++;
           }
           postingsConsumer.finishDoc();
-          docCount++;
         }
-        termsConsumer.finishTerm(term, new TermStats(postings.size(), doFreq ? totalTF : -1));
+        termsConsumer.finishTerm(term, new TermStats(postings.docFreq, doFreq ? totalTF : -1));
         sumTotalTF += totalTF;
-        sumDF += postings.size();
+        sumDF += postings.docFreq;
       }
 
       termsConsumer.finish(doFreq ? sumTotalTF : -1, sumDF, seenDocs.cardinality());
@@ -499,27 +583,27 @@ public abstract class BasePostingsFormat
                           TermsEnum termsEnum,
 
                           // Maximum options (docs/freqs/positions/offsets) to test:
+                          IndexOptions maxTestOptions,
+
                           IndexOptions maxIndexOptions,
 
                           EnumSet<Option> options,
                           boolean alwaysTestMax) throws IOException {
         
     if (VERBOSE) {
-      System.out.println("  verifyEnum: options=" + options + " maxIndexOptions=" + maxIndexOptions);
+      System.out.println("  verifyEnum: options=" + options + " maxTestOptions=" + maxTestOptions);
     }
 
     // 50% of the time time pass liveDocs:
+    boolean useLiveDocs = options.contains(Option.LIVE_DOCS) && random().nextBoolean();
     Bits liveDocs;
-    Map<String,Map<BytesRef,List<Posting>>> fieldsToUse;
-    if (options.contains(Option.LIVE_DOCS) && random().nextBoolean()) {
+    if (useLiveDocs) {
       liveDocs = globalLiveDocs;
-      fieldsToUse = fieldsLive;
       if (VERBOSE) {
         System.out.println("  use liveDocs");
       }
     } else {
       liveDocs = null;
-      fieldsToUse = fields;
       if (VERBOSE) {
         System.out.println("  no liveDocs");
       }
@@ -527,21 +611,23 @@ public abstract class BasePostingsFormat
 
     FieldInfo fieldInfo = currentFieldInfos.fieldInfo(field);
 
-    assertEquals(fields.get(field).get(term).size(), termsEnum.docFreq());
-
     // NOTE: can be empty list if we are using liveDocs:
-    List<Posting> expected = fieldsToUse.get(field).get(term);
-    
+    SeedPostings expected = getSeedPostings(term.utf8ToString(), 
+                                            fields.get(field).get(term),
+                                            useLiveDocs,
+                                            maxIndexOptions);
+    assertEquals(expected.docFreq, termsEnum.docFreq());
+
     boolean allowFreqs = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0 &&
-      maxIndexOptions.compareTo(IndexOptions.DOCS_AND_FREQS) >= 0;
+      maxTestOptions.compareTo(IndexOptions.DOCS_AND_FREQS) >= 0;
     boolean doCheckFreqs = allowFreqs && (alwaysTestMax || random().nextInt(3) <= 2);
 
     boolean allowPositions = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0 &&
-      maxIndexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
+      maxTestOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
     boolean doCheckPositions = allowPositions && (alwaysTestMax || random().nextInt(3) <= 2);
 
     boolean allowOffsets = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0 &&
-      maxIndexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
+      maxTestOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
     boolean doCheckOffsets = allowOffsets && (alwaysTestMax || random().nextInt(3) <= 2);
 
     boolean doCheckPayloads = options.contains(Option.PAYLOADS) && allowPositions && fieldInfo.hasPayloads() && (alwaysTestMax || random().nextInt(3) <= 2);
@@ -623,22 +709,22 @@ public abstract class BasePostingsFormat
 
     // 10% of the time don't consume all docs:
     int stopAt;
-    if (!alwaysTestMax && options.contains(Option.PARTIAL_DOC_CONSUME) && expected.size() > 1 && random().nextInt(10) == 7) {
-      stopAt = random().nextInt(expected.size()-1);
+    if (!alwaysTestMax && options.contains(Option.PARTIAL_DOC_CONSUME) && expected.docFreq > 1 && random().nextInt(10) == 7) {
+      stopAt = random().nextInt(expected.docFreq-1);
       if (VERBOSE) {
-        System.out.println("  will not consume all docs (" + stopAt + " vs " + expected.size() + ")");
+        System.out.println("  will not consume all docs (" + stopAt + " vs " + expected.docFreq + ")");
       }
     } else {
-      stopAt = expected.size();
+      stopAt = expected.docFreq;
       if (VERBOSE) {
         System.out.println("  consume all docs");
       }
     }
 
     double skipChance = alwaysTestMax ? 0.5 : random().nextDouble();
-    int numSkips = expected.size() < 3 ? 1 : _TestUtil.nextInt(random(), 1, Math.min(20, expected.size()/3));
-    int skipInc = expected.size()/numSkips;
-    int skipDocInc = (1+maxDocID)/numSkips;
+    int numSkips = expected.docFreq < 3 ? 1 : _TestUtil.nextInt(random(), 1, Math.min(20, expected.docFreq/3));
+    int skipInc = expected.docFreq/numSkips;
+    int skipDocInc = maxDoc/numSkips;
 
     // Sometimes do 100% skipping:
     boolean doAllSkipping = options.contains(Option.SKIPPING) && random().nextInt(7) == 1;
@@ -664,10 +750,9 @@ public abstract class BasePostingsFormat
       }
     }
 
-    int nextPosting = 0;
-    while (nextPosting <= stopAt) {
-      if (nextPosting == stopAt) {
-        if (stopAt == expected.size()) {
+    while (expected.upto <= stopAt) {
+      if (expected.upto == stopAt) {
+        if (stopAt == expected.docFreq) {
           assertEquals("DocsEnum should have ended but didn't", DocsEnum.NO_MORE_DOCS, docsEnum.nextDoc());
 
           // Common bug is to forget to set this.doc=NO_MORE_DOCS in the enum!:
@@ -676,58 +761,61 @@ public abstract class BasePostingsFormat
         break;
       }
 
-      Posting posting;
       if (options.contains(Option.SKIPPING) && (doAllSkipping || random().nextDouble() <= skipChance)) {
         int targetDocID = -1;
-        if (nextPosting < stopAt && random().nextBoolean()) {
+        if (expected.upto < stopAt && random().nextBoolean()) {
           // Pick target we know exists:
-          nextPosting = _TestUtil.nextInt(random(), nextPosting, nextPosting+skipInc);
+          final int skipCount = _TestUtil.nextInt(random(), 1, skipInc);
+          for(int skip=0;skip<skipCount;skip++) {
+            if (expected.nextDoc() == DocsEnum.NO_MORE_DOCS) {
+              break;
+            }
+          }
         } else {
           // Pick random target (might not exist):
-          Posting target = new Posting();
-          target.docID = _TestUtil.nextInt(random(), expected.get(nextPosting).docID, expected.get(nextPosting).docID+skipDocInc);
-          targetDocID = target.docID;
-          int loc = Collections.binarySearch(expected.subList(nextPosting, expected.size()), target);
-          if (loc < 0) {
-            loc = -loc-1;
+          final int skipDocIDs = _TestUtil.nextInt(random(), 1, skipDocInc);
+          if (skipDocIDs > 0) {
+            targetDocID = expected.docID() + skipDocIDs;
+            expected.advance(targetDocID);
           }
-          nextPosting = nextPosting + loc;
         }
 
-        if (nextPosting >= stopAt) {
-          int target = random().nextBoolean() ? (maxDocID+1) : DocsEnum.NO_MORE_DOCS;
+        if (expected.upto >= stopAt) {
+          int target = random().nextBoolean() ? maxDoc : DocsEnum.NO_MORE_DOCS;
           if (VERBOSE) {
             System.out.println("  now advance to end (target=" + target + ")");
           }
           assertEquals("DocsEnum should have ended but didn't", DocsEnum.NO_MORE_DOCS, docsEnum.advance(target));
           break;
         } else {
-          posting = expected.get(nextPosting++);
           if (VERBOSE) {
             if (targetDocID != -1) {
-              System.out.println("  now advance to random target=" + targetDocID + " (" + nextPosting + " of " + stopAt + ")");
+              System.out.println("  now advance to random target=" + targetDocID + " (" + expected.upto + " of " + stopAt + ") current=" + docsEnum.docID());
             } else {
-              System.out.println("  now advance to known-exists target=" + posting.docID + " (" + nextPosting + " of " + stopAt + ")");
+              System.out.println("  now advance to known-exists target=" + expected.docID() + " (" + expected.upto + " of " + stopAt + ") current=" + docsEnum.docID());
             }
           }
-          int docID = docsEnum.advance(targetDocID != -1 ? targetDocID : posting.docID);
-          assertEquals("docID is wrong", posting.docID, docID);
+          int docID = docsEnum.advance(targetDocID != -1 ? targetDocID : expected.docID());
+          assertEquals("docID is wrong", expected.docID(), docID);
         }
       } else {
-        posting = expected.get(nextPosting++);
+        expected.nextDoc();
         if (VERBOSE) {
-          System.out.println("  now nextDoc to " + posting.docID + " (" + nextPosting + " of " + stopAt + ")");
+          System.out.println("  now nextDoc to " + expected.docID() + " (" + expected.upto + " of " + stopAt + ")");
         }
         int docID = docsEnum.nextDoc();
-        assertEquals("docID is wrong", posting.docID, docID);
+        assertEquals("docID is wrong", expected.docID(), docID);
+        if (docID == DocsEnum.NO_MORE_DOCS) {
+          break;
+        }
       }
 
       if (doCheckFreqs && random().nextDouble() <= freqAskChance) {
         if (VERBOSE) {
-          System.out.println("    now freq()=" + posting.positions.size());
+          System.out.println("    now freq()=" + expected.freq());
         }
         int freq = docsEnum.freq();
-        assertEquals("freq is wrong", posting.positions.size(), freq);
+        assertEquals("freq is wrong", expected.freq(), freq);
       }
 
       if (doCheckPositions) {
@@ -740,30 +828,28 @@ public abstract class BasePostingsFormat
         }
 
         for(int i=0;i<numPosToConsume;i++) {
-          Position position = posting.positions.get(i);
+          int pos = expected.nextPosition();
           if (VERBOSE) {
-            System.out.println("    now nextPosition to " + position.position);
+            System.out.println("    now nextPosition to " + pos);
           }
-          assertEquals("position is wrong", position.position, docsAndPositionsEnum.nextPosition());
-
-          // TODO sometimes don't pull the payload even
-          // though we pulled the position
+          assertEquals("position is wrong", pos, docsAndPositionsEnum.nextPosition());
 
           if (doCheckPayloads) {
+            BytesRef expectedPayload = expected.getPayload();
             if (random().nextDouble() <= payloadCheckChance) {
               if (VERBOSE) {
-                System.out.println("      now check payload length=" + (position.payload == null ? 0 : position.payload.length));
+                System.out.println("      now check expectedPayload length=" + (expectedPayload == null ? 0 : expectedPayload.length));
               }
-              if (position.payload == null || position.payload.length == 0) {
+              if (expectedPayload == null || expectedPayload.length == 0) {
                 assertNull("should not have payload", docsAndPositionsEnum.getPayload());
               } else {
                 BytesRef payload = docsAndPositionsEnum.getPayload();
                 assertNotNull("should have payload but doesn't", payload);
 
-                assertEquals("payload length is wrong", position.payload.length, payload.length);
-                for(int byteUpto=0;byteUpto<position.payload.length;byteUpto++) {
+                assertEquals("payload length is wrong", expectedPayload.length, payload.length);
+                for(int byteUpto=0;byteUpto<expectedPayload.length;byteUpto++) {
                   assertEquals("payload bytes are wrong",
-                               position.payload[byteUpto],
+                               expectedPayload.bytes[expectedPayload.offset + byteUpto],
                                payload.bytes[payload.offset+byteUpto]);
                 }
                 
@@ -773,7 +859,7 @@ public abstract class BasePostingsFormat
               }
             } else {
               if (VERBOSE) {
-                System.out.println("      skip check payload length=" + (position.payload == null ? 0 : position.payload.length));
+                System.out.println("      skip check payload length=" + (expectedPayload == null ? 0 : expectedPayload.length));
               }
             }
           }
@@ -781,10 +867,10 @@ public abstract class BasePostingsFormat
           if (doCheckOffsets) {
             if (random().nextDouble() <= offsetCheckChance) {
               if (VERBOSE) {
-                System.out.println("      now check offsets: startOff=" + position.startOffset + " endOffset=" + position.endOffset);
+                System.out.println("      now check offsets: startOff=" + expected.startOffset() + " endOffset=" + expected.endOffset());
               }
-              assertEquals("startOffset is wrong", position.startOffset, docsAndPositionsEnum.startOffset());
-              assertEquals("endOffset is wrong", position.endOffset, docsAndPositionsEnum.endOffset());
+              assertEquals("startOffset is wrong", expected.startOffset(), docsAndPositionsEnum.startOffset());
+              assertEquals("endOffset is wrong", expected.endOffset(), docsAndPositionsEnum.endOffset());
             } else {
               if (VERBOSE) {
                 System.out.println("      skip check offsets");
@@ -806,12 +892,15 @@ public abstract class BasePostingsFormat
     private Fields fieldsSource;
     private EnumSet<Option> options;
     private IndexOptions maxIndexOptions;
+    private IndexOptions maxTestOptions;
     private boolean alwaysTestMax;
     private BasePostingsFormatTestCase testCase;
 
-    public TestThread(BasePostingsFormatTestCase testCase, Fields fieldsSource, EnumSet<Option> options, IndexOptions maxIndexOptions, boolean alwaysTestMax) {
+    public TestThread(BasePostingsFormatTestCase testCase, Fields fieldsSource, EnumSet<Option> options, IndexOptions maxTestOptions,
+                      IndexOptions maxIndexOptions, boolean alwaysTestMax) {
       this.fieldsSource = fieldsSource;
       this.options = options;
+      this.maxTestOptions = maxTestOptions;
       this.maxIndexOptions = maxIndexOptions;
       this.alwaysTestMax = alwaysTestMax;
       this.testCase = testCase;
@@ -821,7 +910,7 @@ public abstract class BasePostingsFormat
     public void run() {
       try {
         try {
-          testCase.testTermsOneThread(fieldsSource, options, maxIndexOptions, alwaysTestMax);
+          testCase.testTermsOneThread(fieldsSource, options, maxTestOptions, maxIndexOptions, alwaysTestMax);
         } catch (Throwable t) {
           throw new RuntimeException(t);
         }
@@ -833,6 +922,7 @@ public abstract class BasePostingsFormat
   }
 
   private void testTerms(final Fields fieldsSource, final EnumSet<Option> options,
+                         final IndexOptions maxTestOptions,
                          final IndexOptions maxIndexOptions,
                          final boolean alwaysTestMax) throws Exception {
 
@@ -840,18 +930,20 @@ public abstract class BasePostingsFormat
       int numThreads = _TestUtil.nextInt(random(), 2, 5);
       Thread[] threads = new Thread[numThreads];
       for(int threadUpto=0;threadUpto<numThreads;threadUpto++) {
-        threads[threadUpto] = new TestThread(this, fieldsSource, options, maxIndexOptions, alwaysTestMax);
+        threads[threadUpto] = new TestThread(this, fieldsSource, options, maxTestOptions, maxIndexOptions, alwaysTestMax);
         threads[threadUpto].start();
       }
       for(int threadUpto=0;threadUpto<numThreads;threadUpto++) {
         threads[threadUpto].join();
       }
     } else {
-      testTermsOneThread(fieldsSource, options, maxIndexOptions, alwaysTestMax);
+      testTermsOneThread(fieldsSource, options, maxTestOptions, maxIndexOptions, alwaysTestMax);
     }
   }
 
-  private void testTermsOneThread(Fields fieldsSource, EnumSet<Option> options, IndexOptions maxIndexOptions, boolean alwaysTestMax) throws IOException {
+  private void testTermsOneThread(Fields fieldsSource, EnumSet<Option> options,
+                                  IndexOptions maxTestOptions,
+                                  IndexOptions maxIndexOptions, boolean alwaysTestMax) throws IOException {
 
     ThreadState threadState = new ThreadState();
 
@@ -908,6 +1000,7 @@ public abstract class BasePostingsFormat
                  fieldAndTerm.field,
                  fieldAndTerm.term,
                  termsEnum,
+                 maxTestOptions,
                  maxIndexOptions,
                  options,
                  alwaysTestMax);
@@ -932,6 +1025,7 @@ public abstract class BasePostingsFormat
                    fieldAndTerm.field,
                    fieldAndTerm.term,
                    termsEnum,
+                   maxTestOptions,
                    maxIndexOptions,
                    options,
                    alwaysTestMax);
@@ -942,7 +1036,7 @@ public abstract class BasePostingsFormat
   private void testFields(Fields fields) throws Exception {
     Iterator<String> iterator = fields.iterator();
     while (iterator.hasNext()) {
-      String field = iterator.next();
+      iterator.next();
       try {
         iterator.remove();
         fail("Fields.iterator() allows for removal");
@@ -974,10 +1068,10 @@ public abstract class BasePostingsFormat
     int maxIndexOption = Arrays.asList(allOptions).indexOf(options);
 
     for(int i=0;i<=maxIndexOption;i++) {
-      testTerms(fieldsProducer, EnumSet.allOf(Option.class), allOptions[i], true);
+      testTerms(fieldsProducer, EnumSet.allOf(Option.class), allOptions[i], options, true);
       if (withPayloads) {
         // If we indexed w/ payloads, also test enums w/o accessing payloads:
-        testTerms(fieldsProducer, EnumSet.complementOf(EnumSet.of(Option.PAYLOADS)), allOptions[i], true);
+        testTerms(fieldsProducer, EnumSet.complementOf(EnumSet.of(Option.PAYLOADS)), allOptions[i], options, true);
       }
     }
 
@@ -1012,7 +1106,7 @@ public abstract class BasePostingsFormat
 
   public void testRandom() throws Exception {
 
-    int iters = atLeast(10);
+    int iters = 5;
 
     for(int iter=0;iter<iters;iter++) {
       File path = _TestUtil.getTempDir("testPostingsFormat");
@@ -1026,7 +1120,7 @@ public abstract class BasePostingsFormat
 
       // NOTE: you can also test "weaker" index options than
       // you indexed with:
-      testTerms(fieldsProducer, EnumSet.allOf(Option.class), IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS, false);
+      testTerms(fieldsProducer, EnumSet.allOf(Option.class), IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS, false);
 
       fieldsProducer.close();
       fieldsProducer = null;