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 2013/10/21 20:58:44 UTC

svn commit: r1534320 [26/39] - in /lucene/dev/branches/lucene4956: ./ dev-tools/ dev-tools/idea/.idea/ dev-tools/idea/lucene/expressions/ dev-tools/idea/solr/contrib/velocity/ dev-tools/maven/ dev-tools/maven/lucene/ dev-tools/maven/lucene/expressions/...

Modified: lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java (original)
+++ lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java Mon Oct 21 18:58:24 2013
@@ -17,21 +17,21 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
-import static org.apache.lucene.index.SortedSetDocValues.NO_MORE_ORDS;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
+import java.util.concurrent.CountDownLatch;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.lucene42.Lucene42DocValuesFormat;
 import org.apache.lucene.document.BinaryDocValuesField;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
@@ -52,11 +52,14 @@ import org.apache.lucene.search.ScoreDoc
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefHash;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util._TestUtil;
 
+import static org.apache.lucene.index.SortedSetDocValues.NO_MORE_ORDS;
+
 /**
  * Abstract class to do basic tests for a docvalues format.
  * NOTE: This test focuses on the docvalues impl, nothing else.
@@ -648,6 +651,43 @@ public abstract class BaseDocValuesForma
     ireader.close();
     directory.close();
   }
+  
+  public void testSortedMergeAwayAllValues() throws IOException {
+    Directory directory = newDirectory();
+    Analyzer analyzer = new MockAnalyzer(random());
+    IndexWriterConfig iwconfig = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
+    iwconfig.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iwriter = new RandomIndexWriter(random(), directory, iwconfig);
+    
+    Document doc = new Document();
+    doc.add(new StringField("id", "0", Field.Store.NO));
+    iwriter.addDocument(doc);    
+    doc = new Document();
+    doc.add(new StringField("id", "1", Field.Store.NO));
+    doc.add(new SortedDocValuesField("field", new BytesRef("hello")));
+    iwriter.addDocument(doc);
+    iwriter.commit();
+    iwriter.deleteDocuments(new Term("id", "1"));
+    iwriter.forceMerge(1);
+    
+    DirectoryReader ireader = iwriter.getReader();
+    iwriter.close();
+    
+    SortedDocValues dv = getOnlySegmentReader(ireader).getSortedDocValues("field");
+    if (defaultCodecSupportsDocsWithField()) {
+      assertEquals(-1, dv.getOrd(0));
+      assertEquals(0, dv.getValueCount());
+    } else {
+      assertEquals(0, dv.getOrd(0));
+      assertEquals(1, dv.getValueCount());
+      BytesRef ref = new BytesRef();
+      dv.lookupOrd(0, ref);
+      assertEquals(new BytesRef(), ref);
+    }
+    
+    ireader.close();
+    directory.close();
+  }
 
   public void testBytesWithNewline() throws IOException {
     Analyzer analyzer = new MockAnalyzer(random());
@@ -694,7 +734,10 @@ public abstract class BaseDocValuesForma
     BytesRef scratch = new BytesRef();
     dv.lookupOrd(dv.getOrd(0), scratch);
     assertEquals(new BytesRef("hello world 2"), scratch);
-    dv.lookupOrd(dv.getOrd(1), scratch);
+    if (defaultCodecSupportsDocsWithField()) {
+      assertEquals(-1, dv.getOrd(1));
+    }
+    dv.get(1, scratch);
     assertEquals(new BytesRef(""), scratch);
     ireader.close();
     directory.close();
@@ -1049,6 +1092,11 @@ public abstract class BaseDocValuesForma
   public void testRandomSortedBytes() throws IOException {
     Directory dir = newDirectory();
     IndexWriterConfig cfg = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
+    if (!defaultCodecSupportsDocsWithField()) {
+      // if the codec doesnt support missing, we expect missing to be mapped to byte[]
+      // by the impersonator, but we have to give it a chance to merge them to this
+      cfg.setMergePolicy(newLogMergePolicy());
+    }
     RandomIndexWriter w = new RandomIndexWriter(random(), dir, cfg);
     int numDocs = atLeast(100);
     BytesRefHash hash = new BytesRefHash();
@@ -1073,11 +1121,18 @@ public abstract class BaseDocValuesForma
       doc.add(newTextField("id", "noValue", Field.Store.YES));
       w.addDocument(doc);
     }
-    BytesRef bytesRef = new BytesRef();
-    hash.add(bytesRef); // add empty value for the gaps
+    if (!defaultCodecSupportsDocsWithField()) {
+      BytesRef bytesRef = new BytesRef();
+      hash.add(bytesRef); // add empty value for the gaps
+    }
     if (rarely()) {
       w.commit();
     }
+    if (!defaultCodecSupportsDocsWithField()) {
+      // if the codec doesnt support missing, we expect missing to be mapped to byte[]
+      // by the impersonator, but we have to give it a chance to merge them to this
+      w.forceMerge(1);
+    }
     for (int i = 0; i < numDocs; i++) {
       Document doc = new Document();
       String id = "" + i + numDocs;
@@ -1188,6 +1243,73 @@ public abstract class BaseDocValuesForma
     dir.close();
   }
   
+  private void doTestMissingVsFieldCache(final long minValue, final long maxValue) throws Exception {
+    doTestMissingVsFieldCache(new LongProducer() {
+      @Override
+      long next() {
+        return _TestUtil.nextLong(random(), minValue, maxValue);
+      }
+    });
+  }
+  
+  private void doTestMissingVsFieldCache(LongProducer longs) throws Exception {
+    assumeTrue("Codec does not support getDocsWithField", defaultCodecSupportsDocsWithField());
+    Directory dir = newDirectory();
+    IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
+    Field idField = new StringField("id", "", Field.Store.NO);
+    Field indexedField = newStringField("indexed", "", Field.Store.NO);
+    Field dvField = new NumericDocValuesField("dv", 0);
+
+    
+    // index some docs
+    int numDocs = atLeast(300);
+    // numDocs should be always > 256 so that in case of a codec that optimizes
+    // for numbers of values <= 256, all storage layouts are tested
+    assert numDocs > 256;
+    for (int i = 0; i < numDocs; i++) {
+      idField.setStringValue(Integer.toString(i));
+      long value = longs.next();
+      indexedField.setStringValue(Long.toString(value));
+      dvField.setLongValue(value);
+      Document doc = new Document();
+      doc.add(idField);
+      // 1/4 of the time we neglect to add the fields
+      if (random().nextInt(4) > 0) {
+        doc.add(indexedField);
+        doc.add(dvField);
+      }
+      writer.addDocument(doc);
+      if (random().nextInt(31) == 0) {
+        writer.commit();
+      }
+    }
+    
+    // delete some docs
+    int numDeletions = random().nextInt(numDocs/10);
+    for (int i = 0; i < numDeletions; i++) {
+      int id = random().nextInt(numDocs);
+      writer.deleteDocuments(new Term("id", Integer.toString(id)));
+    }
+
+    // merge some segments and ensure that at least one of them has more than
+    // 256 values
+    writer.forceMerge(numDocs / 256);
+
+    writer.close();
+    
+    // compare
+    DirectoryReader ir = DirectoryReader.open(dir);
+    for (AtomicReaderContext context : ir.leaves()) {
+      AtomicReader r = context.reader();
+      Bits expected = FieldCache.DEFAULT.getDocsWithField(r, "indexed");
+      Bits actual = FieldCache.DEFAULT.getDocsWithField(r, "dv");
+      assertEquals(expected, actual);
+    }
+    ir.close();
+    dir.close();
+  }
+  
   public void testBooleanNumericsVsStoredFields() throws Exception {
     int numIterations = atLeast(1);
     for (int i = 0; i < numIterations; i++) {
@@ -1202,6 +1324,13 @@ public abstract class BaseDocValuesForma
     }
   }
   
+  public void testByteMissingVsFieldCache() throws Exception {
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestMissingVsFieldCache(Byte.MIN_VALUE, Byte.MAX_VALUE);
+    }
+  }
+  
   public void testShortNumericsVsStoredFields() throws Exception {
     int numIterations = atLeast(1);
     for (int i = 0; i < numIterations; i++) {
@@ -1209,6 +1338,13 @@ public abstract class BaseDocValuesForma
     }
   }
   
+  public void testShortMissingVsFieldCache() throws Exception {
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestMissingVsFieldCache(Short.MIN_VALUE, Short.MAX_VALUE);
+    }
+  }
+  
   public void testIntNumericsVsStoredFields() throws Exception {
     int numIterations = atLeast(1);
     for (int i = 0; i < numIterations; i++) {
@@ -1216,6 +1352,13 @@ public abstract class BaseDocValuesForma
     }
   }
   
+  public void testIntMissingVsFieldCache() throws Exception {
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestMissingVsFieldCache(Integer.MIN_VALUE, Integer.MAX_VALUE);
+    }
+  }
+  
   public void testLongNumericsVsStoredFields() throws Exception {
     int numIterations = atLeast(1);
     for (int i = 0; i < numIterations; i++) {
@@ -1223,6 +1366,13 @@ public abstract class BaseDocValuesForma
     }
   }
   
+  public void testLongMissingVsFieldCache() throws Exception {
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestMissingVsFieldCache(Long.MIN_VALUE, Long.MAX_VALUE);
+    }
+  }
+  
   private void doTestBinaryVsStoredFields(int minLength, int maxLength) throws Exception {
     Directory dir = newDirectory();
     IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
@@ -1282,7 +1432,7 @@ public abstract class BaseDocValuesForma
   public void testBinaryFixedLengthVsStoredFields() throws Exception {
     int numIterations = atLeast(1);
     for (int i = 0; i < numIterations; i++) {
-      int fixedLength = _TestUtil.nextInt(random(), 1, 10);
+      int fixedLength = _TestUtil.nextInt(random(), 0, 10);
       doTestBinaryVsStoredFields(fixedLength, fixedLength);
     }
   }
@@ -1290,7 +1440,7 @@ public abstract class BaseDocValuesForma
   public void testBinaryVariableLengthVsStoredFields() throws Exception {
     int numIterations = atLeast(1);
     for (int i = 0; i < numIterations; i++) {
-      doTestBinaryVsStoredFields(1, 10);
+      doTestBinaryVsStoredFields(0, 10);
     }
   }
   
@@ -1970,6 +2120,13 @@ public abstract class BaseDocValuesForma
       doTestSortedSetVsStoredFields(1, 10);
     }
   }
+
+  private void assertEquals(Bits expected, Bits actual) throws Exception {
+    assertEquals(expected.length(), actual.length());
+    for (int i = 0; i < expected.length(); i++) {
+      assertEquals(expected.get(i), actual.get(i));
+    }
+  }
   
   private void assertEquals(int maxDoc, SortedDocValues expected, SortedDocValues actual) throws Exception {
     assertEquals(maxDoc, new SingletonSortedSetDocValues(expected), new SingletonSortedSetDocValues(actual));
@@ -2197,5 +2354,648 @@ public abstract class BaseDocValuesForma
       doTestNumericsVsStoredFields(longs);
     }
   }
+  
+  public void testTwoNumbersOneMissing() throws IOException {
+    assumeTrue("Codec does not support getDocsWithField", defaultCodecSupportsDocsWithField());
+    Directory directory = newDirectory();
+    IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, null);
+    conf.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), directory, conf);
+    Document doc = new Document();
+    doc.add(new StringField("id", "0", Field.Store.YES));
+    doc.add(new NumericDocValuesField("dv1", 0));
+    iw.addDocument(doc);
+    doc = new Document();
+    doc.add(new StringField("id", "1", Field.Store.YES));
+    iw.addDocument(doc);
+    iw.forceMerge(1);
+    iw.close();
+    
+    IndexReader ir = DirectoryReader.open(directory);
+    assertEquals(1, ir.leaves().size());
+    AtomicReader ar = ir.leaves().get(0).reader();
+    NumericDocValues dv = ar.getNumericDocValues("dv1");
+    assertEquals(0, dv.get(0));
+    assertEquals(0, dv.get(1));
+    Bits docsWithField = ar.getDocsWithField("dv1");
+    assertTrue(docsWithField.get(0));
+    assertFalse(docsWithField.get(1));
+    ir.close();
+    directory.close();
+  }
+  
+  public void testTwoNumbersOneMissingWithMerging() throws IOException {
+    assumeTrue("Codec does not support getDocsWithField", defaultCodecSupportsDocsWithField());
+    Directory directory = newDirectory();
+    IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, null);
+    conf.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), directory, conf);
+    Document doc = new Document();
+    doc.add(new StringField("id", "0", Field.Store.YES));
+    doc.add(new NumericDocValuesField("dv1", 0));
+    iw.addDocument(doc);
+    iw.commit();
+    doc = new Document();
+    doc.add(new StringField("id", "1", Field.Store.YES));
+    iw.addDocument(doc);
+    iw.forceMerge(1);
+    iw.close();
+    
+    IndexReader ir = DirectoryReader.open(directory);
+    assertEquals(1, ir.leaves().size());
+    AtomicReader ar = ir.leaves().get(0).reader();
+    NumericDocValues dv = ar.getNumericDocValues("dv1");
+    assertEquals(0, dv.get(0));
+    assertEquals(0, dv.get(1));
+    Bits docsWithField = ar.getDocsWithField("dv1");
+    assertTrue(docsWithField.get(0));
+    assertFalse(docsWithField.get(1));
+    ir.close();
+    directory.close();
+  }
+  
+  public void testThreeNumbersOneMissingWithMerging() throws IOException {
+    assumeTrue("Codec does not support getDocsWithField", defaultCodecSupportsDocsWithField());
+    Directory directory = newDirectory();
+    IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, null);
+    conf.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), directory, conf);
+    Document doc = new Document();
+    doc.add(new StringField("id", "0", Field.Store.YES));
+    doc.add(new NumericDocValuesField("dv1", 0));
+    iw.addDocument(doc);
+    doc = new Document();
+    doc.add(new StringField("id", "1", Field.Store.YES));
+    iw.addDocument(doc);
+    iw.commit();
+    doc = new Document();
+    doc.add(new StringField("id", "2", Field.Store.YES));
+    doc.add(new NumericDocValuesField("dv1", 5));
+    iw.addDocument(doc);
+    iw.forceMerge(1);
+    iw.close();
+    
+    IndexReader ir = DirectoryReader.open(directory);
+    assertEquals(1, ir.leaves().size());
+    AtomicReader ar = ir.leaves().get(0).reader();
+    NumericDocValues dv = ar.getNumericDocValues("dv1");
+    assertEquals(0, dv.get(0));
+    assertEquals(0, dv.get(1));
+    assertEquals(5, dv.get(2));
+    Bits docsWithField = ar.getDocsWithField("dv1");
+    assertTrue(docsWithField.get(0));
+    assertFalse(docsWithField.get(1));
+    assertTrue(docsWithField.get(2));
+    ir.close();
+    directory.close();
+  }
+  
+  public void testTwoBytesOneMissing() throws IOException {
+    assumeTrue("Codec does not support getDocsWithField", defaultCodecSupportsDocsWithField());
+    Directory directory = newDirectory();
+    IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, null);
+    conf.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), directory, conf);
+    Document doc = new Document();
+    doc.add(new StringField("id", "0", Field.Store.YES));
+    doc.add(new BinaryDocValuesField("dv1", new BytesRef()));
+    iw.addDocument(doc);
+    doc = new Document();
+    doc.add(new StringField("id", "1", Field.Store.YES));
+    iw.addDocument(doc);
+    iw.forceMerge(1);
+    iw.close();
+    
+    IndexReader ir = DirectoryReader.open(directory);
+    assertEquals(1, ir.leaves().size());
+    AtomicReader ar = ir.leaves().get(0).reader();
+    BinaryDocValues dv = ar.getBinaryDocValues("dv1");
+    BytesRef ref = new BytesRef();
+    dv.get(0, ref);
+    assertEquals(new BytesRef(), ref);
+    dv.get(1, ref);
+    assertEquals(new BytesRef(), ref);
+    Bits docsWithField = ar.getDocsWithField("dv1");
+    assertTrue(docsWithField.get(0));
+    assertFalse(docsWithField.get(1));
+    ir.close();
+    directory.close();
+  }
+  
+  public void testTwoBytesOneMissingWithMerging() throws IOException {
+    assumeTrue("Codec does not support getDocsWithField", defaultCodecSupportsDocsWithField());
+    Directory directory = newDirectory();
+    IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, null);
+    conf.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), directory, conf);
+    Document doc = new Document();
+    doc.add(new StringField("id", "0", Field.Store.YES));
+    doc.add(new BinaryDocValuesField("dv1", new BytesRef()));
+    iw.addDocument(doc);
+    iw.commit();
+    doc = new Document();
+    doc.add(new StringField("id", "1", Field.Store.YES));
+    iw.addDocument(doc);
+    iw.forceMerge(1);
+    iw.close();
+    
+    IndexReader ir = DirectoryReader.open(directory);
+    assertEquals(1, ir.leaves().size());
+    AtomicReader ar = ir.leaves().get(0).reader();
+    BinaryDocValues dv = ar.getBinaryDocValues("dv1");
+    BytesRef ref = new BytesRef();
+    dv.get(0, ref);
+    assertEquals(new BytesRef(), ref);
+    dv.get(1, ref);
+    assertEquals(new BytesRef(), ref);
+    Bits docsWithField = ar.getDocsWithField("dv1");
+    assertTrue(docsWithField.get(0));
+    assertFalse(docsWithField.get(1));
+    ir.close();
+    directory.close();
+  }
+  
+  public void testThreeBytesOneMissingWithMerging() throws IOException {
+    assumeTrue("Codec does not support getDocsWithField", defaultCodecSupportsDocsWithField());
+    Directory directory = newDirectory();
+    IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, null);
+    conf.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), directory, conf);
+    Document doc = new Document();
+    doc.add(new StringField("id", "0", Field.Store.YES));
+    doc.add(new BinaryDocValuesField("dv1", new BytesRef()));
+    iw.addDocument(doc);
+    doc = new Document();
+    doc.add(new StringField("id", "1", Field.Store.YES));
+    iw.addDocument(doc);
+    iw.commit();
+    doc = new Document();
+    doc.add(new StringField("id", "2", Field.Store.YES));
+    doc.add(new BinaryDocValuesField("dv1", new BytesRef("boo")));
+    iw.addDocument(doc);
+    iw.forceMerge(1);
+    iw.close();
+    
+    IndexReader ir = DirectoryReader.open(directory);
+    assertEquals(1, ir.leaves().size());
+    AtomicReader ar = ir.leaves().get(0).reader();
+    BinaryDocValues dv = ar.getBinaryDocValues("dv1");
+    BytesRef ref = new BytesRef();
+    dv.get(0, ref);
+    assertEquals(new BytesRef(), ref);
+    dv.get(1, ref);
+    assertEquals(new BytesRef(), ref);
+    dv.get(2, ref);
+    assertEquals(new BytesRef("boo"), ref);
+    Bits docsWithField = ar.getDocsWithField("dv1");
+    assertTrue(docsWithField.get(0));
+    assertFalse(docsWithField.get(1));
+    assertTrue(docsWithField.get(2));
+    ir.close();
+    directory.close();
+  }
+
+  // LUCENE-4853
+  public void testHugeBinaryValues() throws Exception {
+    Analyzer analyzer = new MockAnalyzer(random());
+    // FSDirectory because SimpleText will consume gobbs of
+    // space when storing big binary values:
+    Directory d = newFSDirectory(_TestUtil.getTempDir("hugeBinaryValues"));
+    boolean doFixed = random().nextBoolean();
+    int numDocs;
+    int fixedLength = 0;
+    if (doFixed) {
+      // Sometimes make all values fixed length since some
+      // codecs have different code paths for this:
+      numDocs = _TestUtil.nextInt(random(), 10, 20);
+      fixedLength = _TestUtil.nextInt(random(), 65537, 256*1024);
+    } else {
+      numDocs = _TestUtil.nextInt(random(), 100, 200);
+    }
+    IndexWriter w = new IndexWriter(d, newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer));
+    List<byte[]> docBytes = new ArrayList<byte[]>();
+    long totalBytes = 0;
+    for(int docID=0;docID<numDocs;docID++) {
+      // we don't use RandomIndexWriter because it might add
+      // more docvalues than we expect !!!!
+
+      // Must be > 64KB in size to ensure more than 2 pages in
+      // PagedBytes would be needed:
+      int numBytes;
+      if (doFixed) {
+        numBytes = fixedLength;
+      } else if (docID == 0 || random().nextInt(5) == 3) {
+        numBytes = _TestUtil.nextInt(random(), 65537, 3*1024*1024);
+      } else {
+        numBytes = _TestUtil.nextInt(random(), 1, 1024*1024);
+      }
+      totalBytes += numBytes;
+      if (totalBytes > 5 * 1024*1024) {
+        break;
+      }
+      byte[] bytes = new byte[numBytes];
+      random().nextBytes(bytes);
+      docBytes.add(bytes);
+      Document doc = new Document();      
+      BytesRef b = new BytesRef(bytes);
+      b.length = bytes.length;
+      doc.add(new BinaryDocValuesField("field", b));
+      doc.add(new StringField("id", ""+docID, Field.Store.YES));
+      try {
+        w.addDocument(doc);
+      } catch (IllegalArgumentException iae) {
+        if (iae.getMessage().indexOf("is too large") == -1) {
+          throw iae;
+        } else {
+          // OK: some codecs can't handle binary DV > 32K
+          assertFalse(codecAcceptsHugeBinaryValues("field"));
+          w.rollback();
+          d.close();
+          return;
+        }
+      }
+    }
+    
+    DirectoryReader r;
+    try {
+      r = w.getReader();
+    } catch (IllegalArgumentException iae) {
+      if (iae.getMessage().indexOf("is too large") == -1) {
+        throw iae;
+      } else {
+        assertFalse(codecAcceptsHugeBinaryValues("field"));
+
+        // OK: some codecs can't handle binary DV > 32K
+        w.rollback();
+        d.close();
+        return;
+      }
+    }
+    w.close();
+
+    AtomicReader ar = SlowCompositeReaderWrapper.wrap(r);
+
+    BinaryDocValues s = FieldCache.DEFAULT.getTerms(ar, "field", false);
+    for(int docID=0;docID<docBytes.size();docID++) {
+      StoredDocument doc = ar.document(docID);
+      BytesRef bytes = new BytesRef();
+      s.get(docID, bytes);
+      byte[] expected = docBytes.get(Integer.parseInt(doc.get("id")));
+      assertEquals(expected.length, bytes.length);
+      assertEquals(new BytesRef(expected), bytes);
+    }
+
+    assertTrue(codecAcceptsHugeBinaryValues("field"));
+
+    ar.close();
+    d.close();
+  }
+
+  // TODO: get this out of here and into the deprecated codecs (4.0, 4.2)
+  public void testHugeBinaryValueLimit() throws Exception {
+    // We only test DVFormats that have a limit
+    assumeFalse("test requires codec with limits on max binary field length", codecAcceptsHugeBinaryValues("field"));
+    Analyzer analyzer = new MockAnalyzer(random());
+    // FSDirectory because SimpleText will consume gobbs of
+    // space when storing big binary values:
+    Directory d = newFSDirectory(_TestUtil.getTempDir("hugeBinaryValues"));
+    boolean doFixed = random().nextBoolean();
+    int numDocs;
+    int fixedLength = 0;
+    if (doFixed) {
+      // Sometimes make all values fixed length since some
+      // codecs have different code paths for this:
+      numDocs = _TestUtil.nextInt(random(), 10, 20);
+      fixedLength = Lucene42DocValuesFormat.MAX_BINARY_FIELD_LENGTH;
+    } else {
+      numDocs = _TestUtil.nextInt(random(), 100, 200);
+    }
+    IndexWriter w = new IndexWriter(d, newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer));
+    List<byte[]> docBytes = new ArrayList<byte[]>();
+    long totalBytes = 0;
+    for(int docID=0;docID<numDocs;docID++) {
+      // we don't use RandomIndexWriter because it might add
+      // more docvalues than we expect !!!!
+
+      // Must be > 64KB in size to ensure more than 2 pages in
+      // PagedBytes would be needed:
+      int numBytes;
+      if (doFixed) {
+        numBytes = fixedLength;
+      } else if (docID == 0 || random().nextInt(5) == 3) {
+        numBytes = Lucene42DocValuesFormat.MAX_BINARY_FIELD_LENGTH;
+      } else {
+        numBytes = _TestUtil.nextInt(random(), 1, Lucene42DocValuesFormat.MAX_BINARY_FIELD_LENGTH);
+      }
+      totalBytes += numBytes;
+      if (totalBytes > 5 * 1024*1024) {
+        break;
+      }
+      byte[] bytes = new byte[numBytes];
+      random().nextBytes(bytes);
+      docBytes.add(bytes);
+      Document doc = new Document();      
+      BytesRef b = new BytesRef(bytes);
+      b.length = bytes.length;
+      doc.add(new BinaryDocValuesField("field", b));
+      doc.add(new StringField("id", ""+docID, Field.Store.YES));
+      w.addDocument(doc);
+    }
+    
+    DirectoryReader r = w.getReader();
+    w.close();
+
+    AtomicReader ar = SlowCompositeReaderWrapper.wrap(r);
+
+    BinaryDocValues s = FieldCache.DEFAULT.getTerms(ar, "field", false);
+    for(int docID=0;docID<docBytes.size();docID++) {
+      StoredDocument doc = ar.document(docID);
+      BytesRef bytes = new BytesRef();
+      s.get(docID, bytes);
+      byte[] expected = docBytes.get(Integer.parseInt(doc.get("id")));
+      assertEquals(expected.length, bytes.length);
+      assertEquals(new BytesRef(expected), bytes);
+    }
+
+    ar.close();
+    d.close();
+  }
+  
+  /** Tests dv against stored fields with threads (binary/numeric/sorted, no missing) */
+  public void testThreads() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
+    Document doc = new Document();
+    Field idField = new StringField("id", "", Field.Store.NO);
+    Field storedBinField = new StoredField("storedBin", new byte[0]);
+    Field dvBinField = new BinaryDocValuesField("dvBin", new BytesRef());
+    Field dvSortedField = new SortedDocValuesField("dvSorted", new BytesRef());
+    Field storedNumericField = new StoredField("storedNum", "");
+    Field dvNumericField = new NumericDocValuesField("dvNum", 0);
+    doc.add(idField);
+    doc.add(storedBinField);
+    doc.add(dvBinField);
+    doc.add(dvSortedField);
+    doc.add(storedNumericField);
+    doc.add(dvNumericField);
+    
+    // index some docs
+    int numDocs = atLeast(300);
+    for (int i = 0; i < numDocs; i++) {
+      idField.setStringValue(Integer.toString(i));
+      int length = _TestUtil.nextInt(random(), 0, 8);
+      byte buffer[] = new byte[length];
+      random().nextBytes(buffer);
+      storedBinField.setBytesValue(buffer);
+      dvBinField.setBytesValue(buffer);
+      dvSortedField.setBytesValue(buffer);
+      long numericValue = random().nextLong();
+      storedNumericField.setStringValue(Long.toString(numericValue));
+      dvNumericField.setLongValue(numericValue);
+      writer.addDocument(doc);
+      if (random().nextInt(31) == 0) {
+        writer.commit();
+      }
+    }
+    
+    // delete some docs
+    int numDeletions = random().nextInt(numDocs/10);
+    for (int i = 0; i < numDeletions; i++) {
+      int id = random().nextInt(numDocs);
+      writer.deleteDocuments(new Term("id", Integer.toString(id)));
+    }
+    writer.close();
+    
+    // compare
+    final DirectoryReader ir = DirectoryReader.open(dir);
+    int numThreads = _TestUtil.nextInt(random(), 2, 7);
+    Thread threads[] = new Thread[numThreads];
+    final CountDownLatch startingGun = new CountDownLatch(1);
+    
+    for (int i = 0; i < threads.length; i++) {
+      threads[i] = new Thread() {
+        @Override
+        public void run() {
+          try {
+            startingGun.await();
+            for (AtomicReaderContext context : ir.leaves()) {
+              AtomicReader r = context.reader();
+              BinaryDocValues binaries = r.getBinaryDocValues("dvBin");
+              SortedDocValues sorted = r.getSortedDocValues("dvSorted");
+              NumericDocValues numerics = r.getNumericDocValues("dvNum");
+              for (int j = 0; j < r.maxDoc(); j++) {
+                BytesRef binaryValue = r.document(j).getBinaryValue("storedBin");
+                BytesRef scratch = new BytesRef();
+                binaries.get(j, scratch);
+                assertEquals(binaryValue, scratch);
+                sorted.get(j, scratch);
+                assertEquals(binaryValue, scratch);
+                String expected = r.document(j).get("storedNum");
+                assertEquals(Long.parseLong(expected), numerics.get(j));
+              }
+            }
+            _TestUtil.checkReader(ir);
+          } catch (Exception e) {
+            throw new RuntimeException(e);
+          }
+        }
+      };
+      threads[i].start();
+    }
+    startingGun.countDown();
+    for (Thread t : threads) {
+      t.join();
+    }
+    ir.close();
+    dir.close();
+  }
+  
+  /** Tests dv against stored fields with threads (all types + missing) */
+  public void testThreads2() throws Exception {
+    assumeTrue("Codec does not support getDocsWithField", defaultCodecSupportsDocsWithField());
+    assumeTrue("Codec does not support SORTED_SET", defaultCodecSupportsSortedSet());
+    Directory dir = newDirectory();
+    IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
+    Field idField = new StringField("id", "", Field.Store.NO);
+    Field storedBinField = new StoredField("storedBin", new byte[0]);
+    Field dvBinField = new BinaryDocValuesField("dvBin", new BytesRef());
+    Field dvSortedField = new SortedDocValuesField("dvSorted", new BytesRef());
+    Field storedNumericField = new StoredField("storedNum", "");
+    Field dvNumericField = new NumericDocValuesField("dvNum", 0);
+    
+    // index some docs
+    int numDocs = atLeast(300);
+    for (int i = 0; i < numDocs; i++) {
+      idField.setStringValue(Integer.toString(i));
+      int length = _TestUtil.nextInt(random(), 0, 8);
+      byte buffer[] = new byte[length];
+      random().nextBytes(buffer);
+      storedBinField.setBytesValue(buffer);
+      dvBinField.setBytesValue(buffer);
+      dvSortedField.setBytesValue(buffer);
+      long numericValue = random().nextLong();
+      storedNumericField.setStringValue(Long.toString(numericValue));
+      dvNumericField.setLongValue(numericValue);
+      Document doc = new Document();
+      doc.add(idField);
+      if (random().nextInt(4) > 0) {
+        doc.add(storedBinField);
+        doc.add(dvBinField);
+        doc.add(dvSortedField);
+      }
+      if (random().nextInt(4) > 0) {
+        doc.add(storedNumericField);
+        doc.add(dvNumericField);
+      }
+      int numSortedSetFields = random().nextInt(3);
+      Set<String> values = new TreeSet<String>();
+      for (int j = 0; j < numSortedSetFields; j++) {
+        values.add(_TestUtil.randomSimpleString(random()));
+      }
+      for (String v : values) {
+        doc.add(new SortedSetDocValuesField("dvSortedSet", new BytesRef(v)));
+        doc.add(new StoredField("storedSortedSet", v));
+      }
+      writer.addDocument(doc);
+      if (random().nextInt(31) == 0) {
+        writer.commit();
+      }
+    }
+    
+    // delete some docs
+    int numDeletions = random().nextInt(numDocs/10);
+    for (int i = 0; i < numDeletions; i++) {
+      int id = random().nextInt(numDocs);
+      writer.deleteDocuments(new Term("id", Integer.toString(id)));
+    }
+    writer.close();
+    
+    // compare
+    final DirectoryReader ir = DirectoryReader.open(dir);
+    int numThreads = _TestUtil.nextInt(random(), 2, 7);
+    Thread threads[] = new Thread[numThreads];
+    final CountDownLatch startingGun = new CountDownLatch(1);
+    
+    for (int i = 0; i < threads.length; i++) {
+      threads[i] = new Thread() {
+        @Override
+        public void run() {
+          try {
+            startingGun.await();
+            for (AtomicReaderContext context : ir.leaves()) {
+              AtomicReader r = context.reader();
+              BinaryDocValues binaries = r.getBinaryDocValues("dvBin");
+              Bits binaryBits = r.getDocsWithField("dvBin");
+              SortedDocValues sorted = r.getSortedDocValues("dvSorted");
+              Bits sortedBits = r.getDocsWithField("dvSorted");
+              NumericDocValues numerics = r.getNumericDocValues("dvNum");
+              Bits numericBits = r.getDocsWithField("dvNum");
+              SortedSetDocValues sortedSet = r.getSortedSetDocValues("dvSortedSet");
+              Bits sortedSetBits = r.getDocsWithField("dvSortedSet");
+              for (int j = 0; j < r.maxDoc(); j++) {
+                BytesRef binaryValue = r.document(j).getBinaryValue("storedBin");
+                if (binaryValue != null) {
+                  if (binaries != null) {
+                    BytesRef scratch = new BytesRef();
+                    binaries.get(j, scratch);
+                    assertEquals(binaryValue, scratch);
+                    sorted.get(j, scratch);
+                    assertEquals(binaryValue, scratch);
+                    assertTrue(binaryBits.get(j));
+                    assertTrue(sortedBits.get(j));
+                  }
+                } else if (binaries != null) {
+                  assertFalse(binaryBits.get(j));
+                  assertFalse(sortedBits.get(j));
+                  assertEquals(-1, sorted.getOrd(j));
+                }
+               
+                String number = r.document(j).get("storedNum");
+                if (number != null) {
+                  if (numerics != null) {
+                    assertEquals(Long.parseLong(number), numerics.get(j));
+                  }
+                } else if (numerics != null) {
+                  assertFalse(numericBits.get(j));
+                  assertEquals(0, numerics.get(j));
+                }
+                
+                String values[] = r.document(j).getValues("storedSortedSet");
+                if (values.length > 0) {
+                  assertNotNull(sortedSet);
+                  sortedSet.setDocument(j);
+                  for (int i = 0; i < values.length; i++) {
+                    long ord = sortedSet.nextOrd();
+                    assertTrue(ord != SortedSetDocValues.NO_MORE_ORDS);
+                    BytesRef value = new BytesRef();
+                    sortedSet.lookupOrd(ord, value);
+                    assertEquals(values[i], value.utf8ToString());
+                  }
+                  assertEquals(SortedSetDocValues.NO_MORE_ORDS, sortedSet.nextOrd());
+                  assertTrue(sortedSetBits.get(j));
+                } else if (sortedSet != null) {
+                  sortedSet.setDocument(j);
+                  assertEquals(SortedSetDocValues.NO_MORE_ORDS, sortedSet.nextOrd());
+                  assertFalse(sortedSetBits.get(j));
+                }
+              }
+            }
+            _TestUtil.checkReader(ir);
+          } catch (Exception e) {
+            throw new RuntimeException(e);
+          }
+        }
+      };
+      threads[i].start();
+    }
+    startingGun.countDown();
+    for (Thread t : threads) {
+      t.join();
+    }
+    ir.close();
+    dir.close();
+  }
 
+  // LUCENE-5218
+  public void testEmptyBinaryValueOnPageSizes() throws Exception {
+    // Test larger and larger power-of-two sized values,
+    // followed by empty string value:
+    for(int i=0;i<20;i++) {
+      if (i > 14 && codecAcceptsHugeBinaryValues("field") == false) {
+        break;
+      }
+      Directory dir = newDirectory();
+      RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+      BytesRef bytes = new BytesRef();
+      bytes.bytes = new byte[1<<i];
+      bytes.length = 1<<i;
+      for(int j=0;j<4;j++) {
+        Document doc = new Document();
+        doc.add(new BinaryDocValuesField("field", bytes));
+        w.addDocument(doc);
+      }
+      Document doc = new Document();
+      doc.add(new StoredField("id", "5"));
+      doc.add(new BinaryDocValuesField("field", new BytesRef()));
+      w.addDocument(doc);
+      IndexReader r = w.getReader();
+      w.close();
+
+      AtomicReader ar = SlowCompositeReaderWrapper.wrap(r);
+      BinaryDocValues values = ar.getBinaryDocValues("field");
+      BytesRef result = new BytesRef();
+      for(int j=0;j<5;j++) {
+        values.get(0, result);
+        assertTrue(result.length == 0 || result.length == 1<<i);
+      }
+      ar.close();
+      dir.close();
+    }
+  }
+
+  protected boolean codecAcceptsHugeBinaryValues(String field) {
+    return true;
+  }
 }

Modified: lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java (original)
+++ lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java Mon Oct 21 18:58:24 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.lucene46.Lucene46Codec;
+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,208 @@ 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 hasFreqs() {
+      return fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0;
+    }
+
+    @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 allowPayloads && 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) {
+        return null;
+      }
+      if ((flags & DocsAndPositionsEnum.FLAG_OFFSETS) != 0 && maxAllowed.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) < 0) {
+        return null;
+      }
+      if ((flags & DocsAndPositionsEnum.FLAG_PAYLOADS) != 0 && allowPayloads == false) {
+        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;
@@ -440,7 +653,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", maxDoc, false, codec, null, null);
+    SegmentInfo segmentInfo = new SegmentInfo(dir, Constants.LUCENE_MAIN_VERSION, "_0", maxDoc, false, codec, null);
 
     int maxIndexOption = Arrays.asList(IndexOptions.values()).indexOf(maxAllowed);
     if (VERBOSE) {
@@ -489,79 +702,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);
-      }
+    Fields seedFields = new SeedFields(fields, newFieldInfos, maxAllowed, allowPayloads);
 
-      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());
-    }
-
-    fieldsConsumer.close();
+    codec.postingsFormat().fieldsConsumer(writeState).write(seedFields);
 
     if (VERBOSE) {
       System.out.println("TEST: after indexing: files=");
@@ -572,7 +716,7 @@ public abstract class BasePostingsFormat
 
     currentFieldInfos = newFieldInfos;
 
-    SegmentReadState readState = new SegmentReadState(dir, segmentInfo, newFieldInfos, IOContext.DEFAULT);
+    SegmentReadState readState = new SegmentReadState(dir, segmentInfo, newFieldInfos, IOContext.READ);
 
     return codec.postingsFormat().fieldsProducer(readState);
   }
@@ -625,7 +769,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 +1369,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 Lucene46Codec() {
+        @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/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/BaseStoredFieldsFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/BaseStoredFieldsFormatTestCase.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/BaseStoredFieldsFormatTestCase.java (original)
+++ lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/BaseStoredFieldsFormatTestCase.java Mon Oct 21 18:58:24 2013
@@ -31,21 +31,20 @@ import java.util.concurrent.atomic.Atomi
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.StoredFieldsFormat;
-import org.apache.lucene.codecs.compressing.CompressingCodec;
-import org.apache.lucene.codecs.lucene42.Lucene42Codec;
+import org.apache.lucene.codecs.lucene46.Lucene46Codec;
 import org.apache.lucene.codecs.simpletext.SimpleTextCodec;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.DoubleField;
 import org.apache.lucene.document.Field;
+import org.apache.lucene.document.Field.Store;
 import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.FieldType.NumericType;
 import org.apache.lucene.document.FloatField;
 import org.apache.lucene.document.IntField;
 import org.apache.lucene.document.LongField;
 import org.apache.lucene.document.StoredField;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.document.TextField;
-import org.apache.lucene.document.Field.Store;
-import org.apache.lucene.document.FieldType.NumericType;
 import org.apache.lucene.search.FieldCache;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.NumericRangeQuery;
@@ -59,7 +58,6 @@ import org.apache.lucene.store.MockDirec
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util._TestUtil;
-import org.apache.lucene.util.LuceneTestCase.Nightly;
 
 import com.carrotsearch.randomizedtesting.generators.RandomInts;
 import com.carrotsearch.randomizedtesting.generators.RandomPicks;
@@ -78,6 +76,7 @@ public abstract class BaseStoredFieldsFo
    */
   protected abstract Codec getCodec();
 
+  @Override
   public void setUp() throws Exception {
     super.setUp();
     // set the default codec, so adding test cases to this isn't fragile
@@ -85,6 +84,7 @@ public abstract class BaseStoredFieldsFo
     Codec.setDefault(getCodec());
   }
 
+  @Override
   public void tearDown() throws Exception {
     Codec.setDefault(savedCodec); // restore
     super.tearDown();
@@ -502,7 +502,7 @@ public abstract class BaseStoredFieldsFo
     // get another codec, other than the default: so we are merging segments across different codecs
     final Codec otherCodec;
     if ("SimpleText".equals(Codec.getDefault().getName())) {
-      otherCodec = new Lucene42Codec();
+      otherCodec = new Lucene46Codec();
     } else {
       otherCodec = new SimpleTextCodec();
     }

Modified: lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/BaseTermVectorsFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/BaseTermVectorsFormatTestCase.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/BaseTermVectorsFormatTestCase.java (original)
+++ lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/BaseTermVectorsFormatTestCase.java Mon Oct 21 18:58:24 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/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java (original)
+++ lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java Mon Oct 21 18:58:24 2013
@@ -22,6 +22,7 @@ import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.Set;
 
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.FilterIterator;
 
 /**
@@ -137,6 +138,11 @@ public final class FieldFilterAtomicRead
   }
 
   @Override
+  public Bits getDocsWithField(String field) throws IOException {
+    return hasField(field) ? super.getDocsWithField(field) : null;
+  }
+
+  @Override
   public String toString() {
     final StringBuilder sb = new StringBuilder("FieldFilterAtomicReader(reader=");
     sb.append(in).append(", fields=");

Modified: lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java (original)
+++ lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java Mon Oct 21 18:58:24 2013
@@ -27,20 +27,24 @@ import java.util.Map;
 import java.util.Random;
 import java.util.Set;
 
-import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.asserting.AssertingDocValuesFormat;
 import org.apache.lucene.codecs.asserting.AssertingPostingsFormat;
+import org.apache.lucene.codecs.bloom.TestBloomFilteredLucene41Postings;
+import org.apache.lucene.codecs.diskdv.DiskDocValuesFormat;
 import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
 import org.apache.lucene.codecs.lucene41ords.Lucene41WithOrds;
 import org.apache.lucene.codecs.lucene41vargap.Lucene41VarGapDocFreqInterval;
 import org.apache.lucene.codecs.lucene41vargap.Lucene41VarGapFixedInterval;
-import org.apache.lucene.codecs.lucene42.Lucene42Codec;
-import org.apache.lucene.codecs.lucene42.Lucene42DocValuesFormat;
-import org.apache.lucene.codecs.bloom.TestBloomFilteredLucene41Postings;
-import org.apache.lucene.codecs.cheapbastard.CheapBastardDocValuesFormat;
-import org.apache.lucene.codecs.diskdv.DiskDocValuesFormat;
+import org.apache.lucene.codecs.lucene45.Lucene45DocValuesFormat;
+import org.apache.lucene.codecs.lucene46.Lucene46Codec;
 import org.apache.lucene.codecs.memory.DirectPostingsFormat;
+import org.apache.lucene.codecs.memory.FSTOrdPostingsFormat;
+import org.apache.lucene.codecs.memory.FSTOrdPulsing41PostingsFormat;
+import org.apache.lucene.codecs.memory.FSTPostingsFormat;
+import org.apache.lucene.codecs.memory.FSTPulsing41PostingsFormat;
+import org.apache.lucene.codecs.memory.MemoryDocValuesFormat;
 import org.apache.lucene.codecs.memory.MemoryPostingsFormat;
 import org.apache.lucene.codecs.mockintblock.MockFixedIntBlockPostingsFormat;
 import org.apache.lucene.codecs.mockintblock.MockVariableIntBlockPostingsFormat;
@@ -48,8 +52,8 @@ import org.apache.lucene.codecs.mockrand
 import org.apache.lucene.codecs.mocksep.MockSepPostingsFormat;
 import org.apache.lucene.codecs.nestedpulsing.NestedPulsingPostingsFormat;
 import org.apache.lucene.codecs.pulsing.Pulsing41PostingsFormat;
-import org.apache.lucene.codecs.simpletext.SimpleTextPostingsFormat;
 import org.apache.lucene.codecs.simpletext.SimpleTextDocValuesFormat;
+import org.apache.lucene.codecs.simpletext.SimpleTextPostingsFormat;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util._TestUtil;
 
@@ -62,7 +66,7 @@ import org.apache.lucene.util._TestUtil;
  * documents in different orders and the test will still be deterministic
  * and reproducable.
  */
-public class RandomCodec extends Lucene42Codec {
+public class RandomCodec extends Lucene46Codec {
   /** Shuffled list of postings formats to use for new mappings */
   private List<PostingsFormat> formats = new ArrayList<PostingsFormat>();
   
@@ -125,6 +129,10 @@ public class RandomCodec extends Lucene4
 
     add(avoidCodecs,
         new Lucene41PostingsFormat(minItemsPerBlock, maxItemsPerBlock),
+        new FSTPostingsFormat(),
+        new FSTOrdPostingsFormat(),
+        new FSTPulsing41PostingsFormat(1 + random.nextInt(20)),
+        new FSTOrdPulsing41PostingsFormat(1 + random.nextInt(20)),
         new DirectPostingsFormat(LuceneTestCase.rarely(random) ? 1 : (LuceneTestCase.rarely(random) ? Integer.MAX_VALUE : maxItemsPerBlock),
                                  LuceneTestCase.rarely(random) ? 1 : (LuceneTestCase.rarely(random) ? Integer.MAX_VALUE : lowFreqCutoff)),
         new Pulsing41PostingsFormat(1 + random.nextInt(20), minItemsPerBlock, maxItemsPerBlock),
@@ -148,11 +156,11 @@ public class RandomCodec extends Lucene4
         new MemoryPostingsFormat(false, random.nextFloat()));
     
     addDocValues(avoidCodecs,
-        new Lucene42DocValuesFormat(),
+        new Lucene45DocValuesFormat(),
         new DiskDocValuesFormat(),
+        new MemoryDocValuesFormat(),
         new SimpleTextDocValuesFormat(),
-        new AssertingDocValuesFormat(),
-        new CheapBastardDocValuesFormat());
+        new AssertingDocValuesFormat());
 
     Collections.shuffle(formats, random);
     Collections.shuffle(dvFormats, random);

Modified: lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java Mon Oct 21 18:58:24 2013
@@ -221,6 +221,10 @@ public class RandomIndexWriter implement
     w.addIndexes(readers);
   }
   
+  public void updateNumericDocValue(Term term, String field, Long value) throws IOException {
+    w.updateNumericDocValue(term, field, value);
+  }
+  
   public void deleteDocuments(Term term) throws IOException {
     w.deleteDocuments(term);
   }
@@ -340,7 +344,7 @@ public class RandomIndexWriter implement
     w.forceMerge(maxSegmentCount);
   }
   
-  private static final class TestPointInfoStream extends InfoStream {
+  static final class TestPointInfoStream extends InfoStream {
     private final InfoStream delegate;
     private final TestPoint testPoint;
     

Modified: lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java (original)
+++ lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java Mon Oct 21 18:58:24 2013
@@ -45,6 +45,7 @@ import org.apache.lucene.util.FailOnNonB
 import org.apache.lucene.util.LineFileDocs;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.NamedThreadFactory;
+import org.apache.lucene.util.PrintStreamInfoStream;
 import org.apache.lucene.util._TestUtil;
 
 // TODO
@@ -327,6 +328,9 @@ public abstract class ThreadedIndexingAn
       searchThreads[thread] = new Thread() {
           @Override
           public void run() {
+            if (VERBOSE) {
+              System.out.println(Thread.currentThread().getName() + ": launch search thread");
+            }
             while (System.currentTimeMillis() < stopTimeMS) {
               try {
                 final IndexSearcher s = getCurrentSearcher();
@@ -478,6 +482,17 @@ public abstract class ThreadedIndexingAn
       }
       });
 
+    if (VERBOSE) {
+      conf.setInfoStream(new PrintStreamInfoStream(System.out) {
+          @Override
+          public void message(String component, String message) {
+            if ("TP".equals(component)) {
+              return; // ignore test points!
+            }
+            super.message(component, message);
+          }
+        });
+    }
     writer = new IndexWriter(dir, conf);
     _TestUtil.reduceOpenFiles(writer);
 
@@ -498,7 +513,7 @@ public abstract class ThreadedIndexingAn
     final Thread[] indexThreads = launchIndexingThreads(docs, NUM_INDEX_THREADS, stopTime, delIDs, delPackIDs, allSubDocs);
 
     if (VERBOSE) {
-      System.out.println("TEST: DONE start indexing threads [" + (System.currentTimeMillis()-t0) + " ms]");
+      System.out.println("TEST: DONE start " + NUM_INDEX_THREADS + " indexing threads [" + (System.currentTimeMillis()-t0) + " ms]");
     }
 
     // Let index build up a bit
@@ -599,7 +614,7 @@ public abstract class ThreadedIndexingAn
       if (!delIDs.contains(stringID)) {
         final TopDocs hits = s.search(new TermQuery(new Term("docid", stringID)), 1);
         if (hits.totalHits != 1) {
-          System.out.println("doc id=" + stringID + " is not supposed to be deleted, but got hitCount=" + hits.totalHits);
+          System.out.println("doc id=" + stringID + " is not supposed to be deleted, but got hitCount=" + hits.totalHits + "; delIDs=" + delIDs);
           doFail = true;
         }
       }

Modified: lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java (original)
+++ lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java Mon Oct 21 18:58:24 2013
@@ -25,6 +25,7 @@ import junit.framework.Assert;
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
+import org.apache.lucene.index.AllDeletedFilterReader;
 import org.apache.lucene.index.AtomicReader;
 import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.IndexReader;
@@ -38,7 +39,6 @@ import org.apache.lucene.store.MockDirec
 import org.apache.lucene.store.RAMDirectory;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util._TestUtil;
 
 import static org.apache.lucene.util.LuceneTestCase.TEST_VERSION_CURRENT;
 
@@ -132,7 +132,7 @@ public class QueryUtils {
   
   public static void purgeFieldCache(IndexReader r) throws IOException {
     // this is just a hack, to get an atomic reader that contains all subreaders for insanity checks
-    FieldCache.DEFAULT.purge(SlowCompositeReaderWrapper.wrap(r));
+    FieldCache.DEFAULT.purgeByCacheKey(SlowCompositeReaderWrapper.wrap(r).getCoreCacheKey());
   }
   
   /** This is a MultiReader that can be used for randomly wrapping other readers
@@ -193,7 +193,7 @@ public class QueryUtils {
   static final IndexReader[] emptyReaders = new IndexReader[8];
   static {
     try {
-      emptyReaders[0] = makeEmptyIndex(new Random(0), 0);
+      emptyReaders[0] = new MultiReader();
       emptyReaders[4] = makeEmptyIndex(new Random(0), 4);
       emptyReaders[5] = makeEmptyIndex(new Random(0), 5);
       emptyReaders[7] = makeEmptyIndex(new Random(0), 7);
@@ -202,31 +202,18 @@ public class QueryUtils {
     }
   }
 
-  private static DirectoryReader makeEmptyIndex(Random random, final int numDeletedDocs) 
-    throws IOException {
+  private static IndexReader makeEmptyIndex(Random random, final int numDocs) throws IOException {
+    assert numDocs > 0;
     Directory d = new MockDirectoryWrapper(random, new RAMDirectory());
-      IndexWriter w = new IndexWriter(d, new IndexWriterConfig(
-        TEST_VERSION_CURRENT, new MockAnalyzer(random)));
-      for (int i = 0; i < numDeletedDocs; i++) {
-        w.addDocument(new Document());
-      }
-      w.commit();
-      w.deleteDocuments( new MatchAllDocsQuery() );
-      _TestUtil.keepFullyDeletedSegments(w);
-      w.commit();
-
-      if (0 < numDeletedDocs)
-        Assert.assertTrue("writer has no deletions", w.hasDeletions());
-
-      Assert.assertEquals("writer is missing some deleted docs", 
-                          numDeletedDocs, w.maxDoc());
-      Assert.assertEquals("writer has non-deleted docs", 
-                          0, w.numDocs());
-      w.close();
-      DirectoryReader r = DirectoryReader.open(d);
-      Assert.assertEquals("reader has wrong number of deleted docs", 
-                          numDeletedDocs, r.numDeletedDocs());
-      return r;
+    IndexWriter w = new IndexWriter(d, new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)));
+    for (int i = 0; i < numDocs; i++) {
+      w.addDocument(new Document());
+    }
+    w.forceMerge(1);
+    w.commit();
+    w.close();
+    DirectoryReader reader = DirectoryReader.open(d);
+    return new AllDeletedFilterReader(LuceneTestCase.getOnlySegmentReader(reader));
   }
 
   /** alternate scorer skipTo(),skipTo(),next(),next(),skipTo(),skipTo(), etc

Modified: lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryWrapper.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryWrapper.java (original)
+++ lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryWrapper.java Mon Oct 21 18:58:24 2013
@@ -18,7 +18,6 @@ package org.apache.lucene.store;
  */
 
 import java.io.IOException;
-import java.util.Collection;
 
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.util._TestUtil;
@@ -29,15 +28,14 @@ import org.apache.lucene.util._TestUtil;
 // do NOT make any methods in this class synchronized, volatile
 // do NOT import anything from the concurrency package.
 // no randoms, no nothing.
-public class BaseDirectoryWrapper extends Directory {
-  /** our in directory */
-  protected final Directory delegate;
+public class BaseDirectoryWrapper extends FilterDirectory {
   
   private boolean checkIndexOnClose = true;
   private boolean crossCheckTermVectorsOnClose = true;
+  protected volatile boolean isOpen = true;
 
   public BaseDirectoryWrapper(Directory delegate) {
-    this.delegate = delegate;
+    super(delegate);
   }
 
   @Override
@@ -46,7 +44,7 @@ public class BaseDirectoryWrapper extend
     if (checkIndexOnClose && DirectoryReader.indexExists(this)) {
       _TestUtil.checkIndex(this, crossCheckTermVectorsOnClose);
     }
-    delegate.close();
+    super.close();
   }
   
   public boolean isOpen() {
@@ -73,80 +71,13 @@ public class BaseDirectoryWrapper extend
     return crossCheckTermVectorsOnClose;
   }
 
-  // directory methods: delegate
-
-  @Override
-  public String[] listAll() throws IOException {
-    return delegate.listAll();
-  }
-
-  @Override
-  public boolean fileExists(String name) throws IOException {
-    return delegate.fileExists(name);
-  }
-
-  @Override
-  public void deleteFile(String name) throws IOException {
-    delegate.deleteFile(name);
-  }
-
-  @Override
-  public long fileLength(String name) throws IOException {
-    return delegate.fileLength(name);
-  }
-
-  @Override
-  public IndexOutput createOutput(String name, IOContext context) throws IOException {
-    return delegate.createOutput(name, context);
-  }
-
-  @Override
-  public void sync(Collection<String> names) throws IOException {
-    delegate.sync(names);
-  }
-
-  @Override
-  public IndexInput openInput(String name, IOContext context) throws IOException {
-    return delegate.openInput(name, context);
-  }
-
-  @Override
-  public Lock makeLock(String name) {
-    return delegate.makeLock(name);
-  }
-
-  @Override
-  public void clearLock(String name) throws IOException {
-    delegate.clearLock(name);
-  }
-
-  @Override
-  public void setLockFactory(LockFactory lockFactory) throws IOException {
-    delegate.setLockFactory(lockFactory);
-  }
-
-  @Override
-  public LockFactory getLockFactory() {
-    return delegate.getLockFactory();
-  }
-
-  @Override
-  public String getLockID() {
-    return delegate.getLockID();
-  }
-
-  @Override
-  public String toString() {
-    return "BaseDirectoryWrapper(" + delegate.toString() + ")";
-  }
-
   @Override
   public void copy(Directory to, String src, String dest, IOContext context) throws IOException {
-    delegate.copy(to, src, dest, context);
+    in.copy(to, src, dest, context);
   }
 
   @Override
   public IndexInputSlicer createSlicer(String name, IOContext context) throws IOException {
-    return delegate.createSlicer(name, context);
+    return in.createSlicer(name, context);
   }  
 }