You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by cm...@apache.org on 2013/08/11 14:19:39 UTC

svn commit: r1512909 [7/38] - in /lucene/dev/branches/lucene4956: ./ dev-tools/ dev-tools/eclipse/ dev-tools/idea/.idea/libraries/ dev-tools/idea/lucene/suggest/ dev-tools/idea/solr/contrib/dataimporthandler/ dev-tools/idea/solr/core/src/test/ dev-tool...

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java Sun Aug 11 12:19:13 2013
@@ -102,20 +102,23 @@ public final class CompressingTermVector
     numDocs = si.getDocCount();
     IndexInput indexStream = null;
     try {
-      vectorsStream = d.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, VECTORS_EXTENSION), context);
+      // Load the index into memory
       final String indexStreamFN = IndexFileNames.segmentFileName(segment, segmentSuffix, VECTORS_INDEX_EXTENSION);
       indexStream = d.openInput(indexStreamFN, context);
-
       final String codecNameIdx = formatName + CODEC_SFX_IDX;
-      final String codecNameDat = formatName + CODEC_SFX_DAT;
       CodecUtil.checkHeader(indexStream, codecNameIdx, VERSION_START, VERSION_CURRENT);
-      CodecUtil.checkHeader(vectorsStream, codecNameDat, VERSION_START, VERSION_CURRENT);
-      assert CodecUtil.headerLength(codecNameDat) == vectorsStream.getFilePointer();
       assert CodecUtil.headerLength(codecNameIdx) == indexStream.getFilePointer();
-
       indexReader = new CompressingStoredFieldsIndexReader(indexStream, si);
+      indexStream.close();
       indexStream = null;
 
+      // Open the data file and read metadata
+      final String vectorsStreamFN = IndexFileNames.segmentFileName(segment, segmentSuffix, VECTORS_EXTENSION);
+      vectorsStream = d.openInput(vectorsStreamFN, context);
+      final String codecNameDat = formatName + CODEC_SFX_DAT;
+      CodecUtil.checkHeader(vectorsStream, codecNameDat, VERSION_START, VERSION_CURRENT);
+      assert CodecUtil.headerLength(codecNameDat) == vectorsStream.getFilePointer();
+
       packedIntsVersion = vectorsStream.readVInt();
       chunkSize = vectorsStream.readVInt();
       decompressor = compressionMode.newDecompressor();
@@ -161,7 +164,7 @@ public final class CompressingTermVector
   @Override
   public void close() throws IOException {
     if (!closed) {
-      IOUtils.close(vectorsStream, indexReader);
+      IOUtils.close(vectorsStream);
       closed = true;
     }
   }
@@ -821,7 +824,7 @@ public final class CompressingTermVector
     }
 
     @Override
-    public SeekStatus seekCeil(BytesRef text, boolean useCache)
+    public SeekStatus seekCeil(BytesRef text)
         throws IOException {
       if (ord < numTerms && ord >= 0) {
         final int cmp = term().compareTo(text);
@@ -848,16 +851,7 @@ public final class CompressingTermVector
 
     @Override
     public void seekExact(long ord) throws IOException {
-      if (ord < -1 || ord >= numTerms) {
-        throw new IOException("ord is out of range: ord=" + ord + ", numTerms=" + numTerms);
-      }
-      if (ord < this.ord) {
-        reset();
-      }
-      for (int i = this.ord; i < ord; ++i) {
-        next();
-      }
-      assert ord == this.ord();
+      throw new UnsupportedOperationException();
     }
 
     @Override
@@ -867,7 +861,7 @@ public final class CompressingTermVector
 
     @Override
     public long ord() throws IOException {
-      return ord;
+      throw new UnsupportedOperationException();
     }
 
     @Override

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java Sun Aug 11 12:19:13 2013
@@ -45,6 +45,7 @@ import org.apache.lucene.store.IndexOutp
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.GrowableByteArrayDataOutput;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.packed.BlockPackedWriter;

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/BitVector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/BitVector.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/BitVector.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/BitVector.java Sun Aug 11 12:19:13 2013
@@ -26,6 +26,7 @@ import org.apache.lucene.store.Directory
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.BitUtil;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.MutableBits;
 
@@ -166,7 +167,7 @@ final class BitVector implements Cloneab
       int c = 0;
       int end = bits.length;
       for (int i = 0; i < end; i++) {
-        c += BYTE_COUNTS[bits[i] & 0xFF];  // sum bits per byte
+        c += BitUtil.bitCount(bits[i]);  // sum bits per byte
       }
       count = c;
     }
@@ -179,29 +180,12 @@ final class BitVector implements Cloneab
     int c = 0;
     int end = bits.length;
     for (int i = 0; i < end; i++) {
-      c += BYTE_COUNTS[bits[i] & 0xFF];  // sum bits per byte
+      c += BitUtil.bitCount(bits[i]);  // sum bits per byte
     }
     return c;
   }
 
-  private static final byte[] BYTE_COUNTS = {  // table of bits/byte
-    0, 1, 1, 2, 1, 2, 2, 3, 1, 2, 2, 3, 2, 3, 3, 4,
-    1, 2, 2, 3, 2, 3, 3, 4, 2, 3, 3, 4, 3, 4, 4, 5,
-    1, 2, 2, 3, 2, 3, 3, 4, 2, 3, 3, 4, 3, 4, 4, 5,
-    2, 3, 3, 4, 3, 4, 4, 5, 3, 4, 4, 5, 4, 5, 5, 6,
-    1, 2, 2, 3, 2, 3, 3, 4, 2, 3, 3, 4, 3, 4, 4, 5,
-    2, 3, 3, 4, 3, 4, 4, 5, 3, 4, 4, 5, 4, 5, 5, 6,
-    2, 3, 3, 4, 3, 4, 4, 5, 3, 4, 4, 5, 4, 5, 5, 6,
-    3, 4, 4, 5, 4, 5, 5, 6, 4, 5, 5, 6, 5, 6, 6, 7,
-    1, 2, 2, 3, 2, 3, 3, 4, 2, 3, 3, 4, 3, 4, 4, 5,
-    2, 3, 3, 4, 3, 4, 4, 5, 3, 4, 4, 5, 4, 5, 5, 6,
-    2, 3, 3, 4, 3, 4, 4, 5, 3, 4, 4, 5, 4, 5, 5, 6,
-    3, 4, 4, 5, 4, 5, 5, 6, 4, 5, 5, 6, 5, 6, 6, 7,
-    2, 3, 3, 4, 3, 4, 4, 5, 3, 4, 4, 5, 4, 5, 5, 6,
-    3, 4, 4, 5, 4, 5, 5, 6, 4, 5, 5, 6, 5, 6, 6, 7,
-    3, 4, 4, 5, 4, 5, 5, 6, 4, 5, 5, 6, 5, 6, 6, 7,
-    4, 5, 5, 6, 5, 6, 6, 7, 5, 6, 6, 7, 6, 7, 7, 8
-  };
+
 
   private static String CODEC = "BitVector";
 
@@ -294,7 +278,7 @@ final class BitVector implements Cloneab
         output.writeVInt(i-last);
         output.writeByte(bits[i]);
         last = i;
-        numCleared -= (8-BYTE_COUNTS[bits[i] & 0xFF]);
+        numCleared -= (8-BitUtil.bitCount(bits[i]));
         assert numCleared >= 0 || (i == (bits.length-1) && numCleared == -(8-(size&7)));
       }
     }
@@ -399,7 +383,7 @@ final class BitVector implements Cloneab
     while (n>0) {
       last += input.readVInt();
       bits[last] = input.readByte();
-      n -= BYTE_COUNTS[bits[last] & 0xFF];
+      n -= BitUtil.bitCount(bits[last]);
       assert n >= 0;
     }          
   }
@@ -416,7 +400,7 @@ final class BitVector implements Cloneab
     while (numCleared>0) {
       last += input.readVInt();
       bits[last] = input.readByte();
-      numCleared -= 8-BYTE_COUNTS[bits[last] & 0xFF];
+      numCleared -= 8-BitUtil.bitCount(bits[last]);
       assert numCleared >= 0 || (last == (bits.length-1) && numCleared == -(8-(size&7)));
     }
   }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java Sun Aug 11 12:19:13 2013
@@ -258,8 +258,7 @@ public class Lucene40PostingsFormat exte
                                                     state.segmentInfo,
                                                     postings,
                                                     state.context,
-                                                    state.segmentSuffix,
-                                                    state.termsIndexDivisor);
+                                                    state.segmentSuffix);
       success = true;
       return ret;
     } finally {

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java Sun Aug 11 12:19:13 2013
@@ -433,7 +433,7 @@ public class Lucene40TermVectorsReader e
 
     // NOTE: slow!  (linear scan)
     @Override
-    public SeekStatus seekCeil(BytesRef text, boolean useCache)
+    public SeekStatus seekCeil(BytesRef text)
       throws IOException {
       if (nextTerm != 0) {
         final int cmp = text.compareTo(term);

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsWriter.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsWriter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsWriter.java Sun Aug 11 12:19:13 2013
@@ -242,8 +242,8 @@ public final class Lucene40TermVectorsWr
       if (payloads) {
         tvf.writeBytes(payloadData.bytes, payloadData.offset, payloadData.length);
       }
-      for (int i = 0; i < bufferedIndex; i++) {
-        if (offsets) {
+      if (offsets) {
+        for (int i = 0; i < bufferedIndex; i++) {
           tvf.writeVInt(offsetStartBuffer[i] - lastOffset);
           tvf.writeVInt(offsetEndBuffer[i] - offsetStartBuffer[i]);
           lastOffset = offsetEndBuffer[i];

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/package.html?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/package.html (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/package.html Sun Aug 11 12:19:13 2013
@@ -372,13 +372,7 @@ term vectors.</li>
 <a name="Limitations" id="Limitations"></a>
 <h2>Limitations</h2>
 <div>
-<p>When referring to term numbers, Lucene's current implementation uses a Java
-<code>int</code> to hold the term index, which means the
-maximum number of unique terms in any single index segment is ~2.1 billion
-times the term index interval (default 128) = ~274 billion. This is technically
-not a limitation of the index file format, just of Lucene's current
-implementation.</p>
-<p>Similarly, Lucene uses a Java <code>int</code> to refer to
+<p>Lucene uses a Java <code>int</code> to refer to
 document numbers, and the index file format uses an <code>Int32</code>
 on-disk to store document numbers. This is a limitation
 of both the index file format and the current implementation. Eventually these

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java Sun Aug 11 12:19:13 2013
@@ -161,7 +161,7 @@ import org.apache.lucene.util.packed.Pac
  *    <li>SkipFPDelta determines the position of this term's SkipData within the .doc
  *        file. In particular, it is the length of the TermFreq data.
  *        SkipDelta is only stored if DocFreq is not smaller than SkipMinimum
- *        (i.e. 8 in Lucene41PostingsFormat).</li>
+ *        (i.e. 128 in Lucene41PostingsFormat).</li>
  *    <li>SingletonDocID is an optimization when a term only appears in one document. In this case, instead
  *        of writing a file pointer to the .doc file (DocFPDelta), and then a VIntBlock at that location, the 
  *        single document ID is written to the term dictionary.</li>
@@ -439,8 +439,7 @@ public final class Lucene41PostingsForma
                                                     state.segmentInfo,
                                                     postingsReader,
                                                     state.context,
-                                                    state.segmentSuffix,
-                                                    state.termsIndexDivisor);
+                                                    state.segmentSuffix);
       success = true;
       return ret;
     } finally {

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/package.html?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/package.html (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/package.html Sun Aug 11 12:19:13 2013
@@ -381,13 +381,7 @@ the term dictionary. Stored fields are c
 <a name="Limitations" id="Limitations"></a>
 <h2>Limitations</h2>
 <div>
-<p>When referring to term numbers, Lucene's current implementation uses a Java
-<code>int</code> to hold the term index, which means the
-maximum number of unique terms in any single index segment is ~2.1 billion
-times the term index interval (default 128) = ~274 billion. This is technically
-not a limitation of the index file format, just of Lucene's current
-implementation.</p>
-<p>Similarly, Lucene uses a Java <code>int</code> to refer to
+<p>Lucene uses a Java <code>int</code> to refer to
 document numbers, and the index file format uses an <code>Int32</code>
 on-disk to store document numbers. This is a limitation
 of both the index file format and the current implementation. Eventually these

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java Sun Aug 11 12:19:13 2013
@@ -245,7 +245,7 @@ class Lucene42DocValuesConsumer extends 
     meta.writeVInt(field.number);
     meta.writeByte(FST);
     meta.writeLong(data.getFilePointer());
-    PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
+    PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
     Builder<Long> builder = new Builder<Long>(INPUT_TYPE.BYTE1, outputs);
     IntsRef scratch = new IntsRef();
     long ord = 0;

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesFormat.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesFormat.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesFormat.java Sun Aug 11 12:19:13 2013
@@ -120,16 +120,33 @@ import org.apache.lucene.util.packed.Blo
  * </ol>
  */
 public final class Lucene42DocValuesFormat extends DocValuesFormat {
-
-  /** Sole constructor */
+  final float acceptableOverheadRatio;
+  
+  /** 
+   * Calls {@link #Lucene42DocValuesFormat(float) 
+   * Lucene42DocValuesFormat(PackedInts.DEFAULT)} 
+   */
   public Lucene42DocValuesFormat() {
+    this(PackedInts.DEFAULT);
+  }
+  
+  /**
+   * Creates a new Lucene42DocValuesFormat with the specified
+   * <code>acceptableOverheadRatio</code> for NumericDocValues.
+   * @param acceptableOverheadRatio compression parameter for numerics. 
+   *        Currently this is only used when the number of unique values is small.
+   *        
+   * @lucene.experimental
+   */
+  public Lucene42DocValuesFormat(float acceptableOverheadRatio) {
     super("Lucene42");
+    this.acceptableOverheadRatio = acceptableOverheadRatio;
   }
 
   @Override
   public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
     // note: we choose DEFAULT here (its reasonably fast, and for small bpv has tiny waste)
-    return new Lucene42DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION, PackedInts.DEFAULT);
+    return new Lucene42DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION, acceptableOverheadRatio);
   }
   
   @Override

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java Sun Aug 11 12:19:13 2013
@@ -278,7 +278,7 @@ class Lucene42DocValuesProducer extends 
       instance = fstInstances.get(field.number);
       if (instance == null) {
         data.seek(entry.offset);
-        instance = new FST<Long>(data, PositiveIntOutputs.getSingleton(true));
+        instance = new FST<Long>(data, PositiveIntOutputs.getSingleton());
         fstInstances.put(field.number, instance);
       }
     }
@@ -352,7 +352,7 @@ class Lucene42DocValuesProducer extends 
       instance = fstInstances.get(field.number);
       if (instance == null) {
         data.seek(entry.offset);
-        instance = new FST<Long>(data, PositiveIntOutputs.getSingleton(true));
+        instance = new FST<Long>(data, PositiveIntOutputs.getSingleton());
         fstInstances.put(field.number, instance);
       }
     }
@@ -490,7 +490,7 @@ class Lucene42DocValuesProducer extends 
     }
 
     @Override
-    public SeekStatus seekCeil(BytesRef text, boolean useCache) throws IOException {
+    public SeekStatus seekCeil(BytesRef text) throws IOException {
       if (in.seekCeil(text) == null) {
         return SeekStatus.END;
       } else if (term().equals(text)) {
@@ -503,7 +503,7 @@ class Lucene42DocValuesProducer extends 
     }
 
     @Override
-    public boolean seekExact(BytesRef text, boolean useCache) throws IOException {
+    public boolean seekExact(BytesRef text) throws IOException {
       if (in.seekExact(text) == null) {
         return false;
       } else {

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsFormat.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsFormat.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsFormat.java Sun Aug 11 12:19:13 2013
@@ -42,14 +42,32 @@ import org.apache.lucene.util.packed.Pac
  * @see Lucene42DocValuesFormat
  */
 public final class Lucene42NormsFormat extends NormsFormat {
+  final float acceptableOverheadRatio;
 
-  /** Sole constructor */
-  public Lucene42NormsFormat() {}
+  /** 
+   * Calls {@link #Lucene42NormsFormat(float) 
+   * Lucene42DocValuesFormat(PackedInts.FASTEST)} 
+   */
+  public Lucene42NormsFormat() {
+    // note: we choose FASTEST here (otherwise our norms are half as big but 15% slower than previous lucene)
+    this(PackedInts.FASTEST);
+  }
+  
+  /**
+   * Creates a new Lucene42DocValuesFormat with the specified
+   * <code>acceptableOverheadRatio</code> for NumericDocValues.
+   * @param acceptableOverheadRatio compression parameter for numerics. 
+   *        Currently this is only used when the number of unique values is small.
+   *        
+   * @lucene.experimental
+   */
+  public Lucene42NormsFormat(float acceptableOverheadRatio) {
+    this.acceptableOverheadRatio = acceptableOverheadRatio;
+  }
   
   @Override
   public DocValuesConsumer normsConsumer(SegmentWriteState state) throws IOException {
-    // note: we choose FASTEST here (otherwise our norms are half as big but 15% slower than previous lucene)
-    return new Lucene42DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION, PackedInts.FASTEST);
+    return new Lucene42DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION, acceptableOverheadRatio);
   }
   
   @Override

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/package.html?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/package.html (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/package.html Sun Aug 11 12:19:13 2013
@@ -384,13 +384,7 @@ on multi-valued fields.</li>
 <a name="Limitations" id="Limitations"></a>
 <h2>Limitations</h2>
 <div>
-<p>When referring to term numbers, Lucene's current implementation uses a Java
-<code>int</code> to hold the term index, which means the
-maximum number of unique terms in any single index segment is ~2.1 billion
-times the term index interval (default 128) = ~274 billion. This is technically
-not a limitation of the index file format, just of Lucene's current
-implementation.</p>
-<p>Similarly, Lucene uses a Java <code>int</code> to refer to
+<p>Lucene uses a Java <code>int</code> to refer to
 document numbers, and the index file format uses an <code>Int32</code>
 on-disk to store document numbers. This is a limitation
 of both the index file format and the current implementation. Eventually these

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/document/Field.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/document/Field.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/document/Field.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/document/Field.java Sun Aug 11 12:19:13 2013
@@ -75,7 +75,6 @@ public class Field implements IndexableF
   protected TokenStream tokenStream;
 
   private transient TokenStream internalTokenStream;
-  private transient ReusableStringReader internalReader;
 
   /**
    * Field's boost
@@ -552,56 +551,12 @@ public class Field implements IndexableF
     } else if (readerValue() != null) {
       return analyzer.tokenStream(name(), readerValue());
     } else if (stringValue() != null) {
-      if (internalReader == null) {
-        internalReader = new ReusableStringReader();
-      }
-      internalReader.setValue(stringValue());
-      return analyzer.tokenStream(name(), internalReader);
+      return analyzer.tokenStream(name(), stringValue());
     }
 
     throw new IllegalArgumentException("Field must have either TokenStream, String, Reader or Number value");
   }
   
-  static final class ReusableStringReader extends Reader {
-    private int pos = 0, size = 0;
-    private String s = null;
-    
-    void setValue(String s) {
-      this.s = s;
-      this.size = s.length();
-      this.pos = 0;
-    }
-    
-    @Override
-    public int read() {
-      if (pos < size) {
-        return s.charAt(pos++);
-      } else {
-        s = null;
-        return -1;
-      }
-    }
-    
-    @Override
-    public int read(char[] c, int off, int len) {
-      if (pos < size) {
-        len = Math.min(len, size-pos);
-        s.getChars(pos, pos+len, c, off);
-        pos += len;
-        return len;
-      } else {
-        s = null;
-        return -1;
-      }
-    }
-    
-    @Override
-    public void close() {
-      pos = size; // this prevents NPE when reading after close!
-      s = null;
-    }
-  }
-  
   static final class StringTokenStream extends TokenStream {
     private final CharTermAttribute termAttribute = addAttribute(CharTermAttribute.class);
     private final OffsetAttribute offsetAttribute = addAttribute(OffsetAttribute.class);

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java Sun Aug 11 12:19:13 2013
@@ -78,7 +78,7 @@ public abstract class AtomicReader exten
       return 0;
     }
     final TermsEnum termsEnum = terms.iterator(null);
-    if (termsEnum.seekExact(term.bytes(), true)) {
+    if (termsEnum.seekExact(term.bytes())) {
       return termsEnum.docFreq();
     } else {
       return 0;
@@ -101,7 +101,7 @@ public abstract class AtomicReader exten
       return 0;
     }
     final TermsEnum termsEnum = terms.iterator(null);
-    if (termsEnum.seekExact(term.bytes(), true)) {
+    if (termsEnum.seekExact(term.bytes())) {
       return termsEnum.totalTermFreq();
     } else {
       return 0;
@@ -156,7 +156,7 @@ public abstract class AtomicReader exten
       final Terms terms = fields.terms(term.field());
       if (terms != null) {
         final TermsEnum termsEnum = terms.iterator(null);
-        if (termsEnum.seekExact(term.bytes(), true)) {
+        if (termsEnum.seekExact(term.bytes())) {
           return termsEnum.docs(getLiveDocs(), null);
         }
       }
@@ -176,7 +176,7 @@ public abstract class AtomicReader exten
       final Terms terms = fields.terms(term.field());
       if (terms != null) {
         final TermsEnum termsEnum = terms.iterator(null);
-        if (termsEnum.seekExact(term.bytes(), true)) {
+        if (termsEnum.seekExact(term.bytes())) {
           return termsEnum.docsAndPositions(getLiveDocs(), null);
         }
       }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java Sun Aug 11 12:19:13 2013
@@ -26,7 +26,8 @@ import org.apache.lucene.util.ByteBlockP
 import org.apache.lucene.util.ByteBlockPool;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.Counter;
-import org.apache.lucene.util.packed.AppendingLongBuffer;
+import org.apache.lucene.util.packed.AppendingDeltaPackedLongBuffer;
+import org.apache.lucene.util.packed.PackedInts;
 
 import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
 
@@ -36,14 +37,14 @@ import static org.apache.lucene.util.Byt
 class BinaryDocValuesWriter extends DocValuesWriter {
 
   private final ByteBlockPool pool;
-  private final AppendingLongBuffer lengths;
+  private final AppendingDeltaPackedLongBuffer lengths;
   private final FieldInfo fieldInfo;
   private int addedValues = 0;
 
   public BinaryDocValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
     this.fieldInfo = fieldInfo;
     this.pool = new ByteBlockPool(new DirectTrackingAllocator(iwBytesUsed));
-    this.lengths = new AppendingLongBuffer();
+    this.lengths = new AppendingDeltaPackedLongBuffer(PackedInts.COMPACT);
   }
 
   public void addValue(int docID, BytesRef value) {
@@ -90,7 +91,7 @@ class BinaryDocValuesWriter extends DocV
   // iterates over the values we have in ram
   private class BytesIterator implements Iterator<BytesRef> {
     final BytesRef value = new BytesRef();
-    final AppendingLongBuffer.Iterator lengthsIterator = lengths.iterator();
+    final AppendingDeltaPackedLongBuffer.Iterator lengthsIterator = lengths.iterator();
     final int size = (int) lengths.size();
     final int maxDoc;
     int upto;

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BufferedDeletes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BufferedDeletes.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BufferedDeletes.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BufferedDeletes.java Sun Aug 11 12:19:13 2013
@@ -33,9 +33,9 @@ import org.apache.lucene.util.RamUsageEs
  * deletes are pushed (on flush in DocumentsWriter), these
  * deletes are converted to a FrozenDeletes instance. */
 
-// NOTE: we are sync'd by BufferedDeletes, ie, all access to
-// instances of this class is via sync'd methods on
-// BufferedDeletes
+// NOTE: instances of this class are accessed either via a private
+// instance on DocumentWriterPerThread, or via sync'd code by
+// DocumentsWriterDeleteQueue
 
 class BufferedDeletes {
 
@@ -136,6 +136,9 @@ class BufferedDeletes {
     }
 
     terms.put(term, Integer.valueOf(docIDUpto));
+    // note that if current != null then it means there's already a buffered
+    // delete on that term, therefore we seem to over-count. this over-counting
+    // is done to respect IndexWriterConfig.setMaxBufferedDeleteTerms.
     numTermDeletes.incrementAndGet();
     if (current == null) {
       bytesUsed.addAndGet(BYTES_PER_DEL_TERM + term.bytes.length + (RamUsageEstimator.NUM_BYTES_CHAR * term.field().length()));

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java Sun Aug 11 12:19:13 2013
@@ -390,7 +390,7 @@ class BufferedDeletesStream {
 
       // System.out.println("  term=" + term);
 
-      if (termsEnum.seekExact(term.bytes(), false)) {
+      if (termsEnum.seekExact(term.bytes())) {
         // we don't need term frequencies for this
         DocsEnum docsEnum = termsEnum.docs(rld.getLiveDocs(), docs, DocsEnum.FLAG_NONE);
         //System.out.println("BDS: got docsEnum=" + docsEnum);

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java Sun Aug 11 12:19:13 2013
@@ -30,8 +30,8 @@ import java.util.Map;
 
 import org.apache.lucene.codecs.BlockTreeTermsReader;
 import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.PostingsFormat; // javadocs
-import org.apache.lucene.document.FieldType; // for javadocs
+import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.index.CheckIndex.Status.DocValuesStatus;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.store.Directory;
@@ -159,19 +159,6 @@ public class CheckIndex {
        *  segment. */
       public double sizeMB;
 
-      /** Doc store offset, if this segment shares the doc
-       *  store files (stored fields and term vectors) with
-       *  other segments.  This is -1 if it does not share. */
-      public int docStoreOffset = -1;
-    
-      /** String of the shared doc store segment, or null if
-       *  this segment does not share the doc store files. */
-      public String docStoreSegment;
-
-      /** True if the shared doc store files are compound file
-       *  format. */
-      public boolean docStoreCompoundFile;
-
       /** True if this segment has pending deletions. */
       public boolean hasDeletions;
 
@@ -297,10 +284,21 @@ public class CheckIndex {
       DocValuesStatus() {
       }
 
-      /** Number of documents tested. */
-      public int docCount;
       /** Total number of docValues tested. */
       public long totalValueFields;
+      
+      /** Total number of numeric fields */
+      public long totalNumericFields;
+      
+      /** Total number of binary fields */
+      public long totalBinaryFields;
+      
+      /** Total number of sorted fields */
+      public long totalSortedFields;
+      
+      /** Total number of sortedset fields */
+      public long totalSortedSetFields;
+      
       /** Exception thrown during doc values test (null on success) */
       public Throwable error = null;
     }
@@ -463,11 +461,11 @@ public class CheckIndex {
 
     if (onlySegments != null) {
       result.partial = true;
-      if (infoStream != null)
+      if (infoStream != null) {
         infoStream.print("\nChecking only these segments:");
-      for (String s : onlySegments) {
-        if (infoStream != null)
+        for (String s : onlySegments) {
           infoStream.print(" " + s);
+        }
       }
       result.segmentsChecked.addAll(onlySegments);
       msg(infoStream, ":");
@@ -535,7 +533,7 @@ public class CheckIndex {
         }
         if (infoStream != null)
           infoStream.print("    test: open reader.........");
-        reader = new SegmentReader(info, DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR, IOContext.DEFAULT);
+        reader = new SegmentReader(info, IOContext.DEFAULT);
 
         segInfoStat.openReaderPassed = true;
 
@@ -1117,7 +1115,7 @@ public class CheckIndex {
             long totDocCountNoDeletes = 0;
             long totDocFreq = 0;
             for(int i=0;i<seekCount;i++) {
-              if (!termsEnum.seekExact(seekTerms[i], true)) {
+              if (!termsEnum.seekExact(seekTerms[i])) {
                 throw new RuntimeException("seek to existing term " + seekTerms[i] + " failed");
               }
               
@@ -1272,7 +1270,7 @@ public class CheckIndex {
       for (FieldInfo fieldInfo : reader.getFieldInfos()) {
         if (fieldInfo.hasDocValues()) {
           status.totalValueFields++;
-          checkDocValues(fieldInfo, reader, infoStream);
+          checkDocValues(fieldInfo, reader, infoStream, status);
         } else {
           if (reader.getBinaryDocValues(fieldInfo.name) != null ||
               reader.getNumericDocValues(fieldInfo.name) != null ||
@@ -1283,7 +1281,11 @@ public class CheckIndex {
         }
       }
 
-      msg(infoStream, "OK [" + status.docCount + " total doc count; " + status.totalValueFields + " docvalues fields]");
+      msg(infoStream, "OK [" + status.totalValueFields + " docvalues fields; "
+                             + status.totalBinaryFields + " BINARY; " 
+                             + status.totalNumericFields + " NUMERIC; "
+                             + status.totalSortedFields + " SORTED; "
+                             + status.totalSortedSetFields + " SORTED_SET]");
     } catch (Throwable e) {
       msg(infoStream, "ERROR [" + String.valueOf(e.getMessage()) + "]");
       status.error = e;
@@ -1382,9 +1384,10 @@ public class CheckIndex {
     }
   }
   
-  private static void checkDocValues(FieldInfo fi, AtomicReader reader, PrintStream infoStream) throws Exception {
+  private static void checkDocValues(FieldInfo fi, AtomicReader reader, PrintStream infoStream, DocValuesStatus status) throws Exception {
     switch(fi.getDocValuesType()) {
       case SORTED:
+        status.totalSortedFields++;
         checkSortedDocValues(fi.name, reader, reader.getSortedDocValues(fi.name));
         if (reader.getBinaryDocValues(fi.name) != null ||
             reader.getNumericDocValues(fi.name) != null ||
@@ -1393,6 +1396,7 @@ public class CheckIndex {
         }
         break;
       case SORTED_SET:
+        status.totalSortedSetFields++;
         checkSortedSetDocValues(fi.name, reader, reader.getSortedSetDocValues(fi.name));
         if (reader.getBinaryDocValues(fi.name) != null ||
             reader.getNumericDocValues(fi.name) != null ||
@@ -1401,6 +1405,7 @@ public class CheckIndex {
         }
         break;
       case BINARY:
+        status.totalBinaryFields++;
         checkBinaryDocValues(fi.name, reader, reader.getBinaryDocValues(fi.name));
         if (reader.getNumericDocValues(fi.name) != null ||
             reader.getSortedDocValues(fi.name) != null ||
@@ -1409,6 +1414,7 @@ public class CheckIndex {
         }
         break;
       case NUMERIC:
+        status.totalNumericFields++;
         checkNumericDocValues(fi.name, reader, reader.getNumericDocValues(fi.name));
         if (reader.getBinaryDocValues(fi.name) != null ||
             reader.getSortedDocValues(fi.name) != null ||
@@ -1543,7 +1549,7 @@ public class CheckIndex {
                 }
 
                 final DocsEnum postingsDocs2;
-                if (!postingsTermsEnum.seekExact(term, true)) {
+                if (!postingsTermsEnum.seekExact(term)) {
                   throw new RuntimeException("vector term=" + term + " field=" + field + " does not exist in postings; doc=" + j);
                 }
                 postingsPostings = postingsTermsEnum.docsAndPositions(null, postingsPostings);
@@ -1677,7 +1683,7 @@ public class CheckIndex {
    *
    * <p><b>WARNING</b>: Make sure you only call this when the
    *  index is not opened  by any writer. */
-  public void fixIndex(Status result, Codec codec) throws IOException {
+  public void fixIndex(Status result) throws IOException {
     if (result.partial)
       throw new IllegalArgumentException("can only fix an index that was fully checked (this status checked a subset of segments)");
     result.newSegments.changed();
@@ -1732,7 +1738,6 @@ public class CheckIndex {
 
     boolean doFix = false;
     boolean doCrossCheckTermVectors = false;
-    Codec codec = Codec.getDefault(); // only used when fixing
     boolean verbose = false;
     List<String> onlySegments = new ArrayList<String>();
     String indexPath = null;
@@ -1744,13 +1749,6 @@ public class CheckIndex {
         doFix = true;
       } else if ("-crossCheckTermVectors".equals(arg)) {
         doCrossCheckTermVectors = true;
-      } else if ("-codec".equals(arg)) {
-        if (i == args.length-1) {
-          System.out.println("ERROR: missing name for -codec option");
-          System.exit(1);
-        }
-        i++;
-        codec = Codec.forName(args[i]);
       } else if (arg.equals("-verbose")) {
         verbose = true;
       } else if (arg.equals("-segment")) {
@@ -1851,7 +1849,7 @@ public class CheckIndex {
           System.out.println("  " + (5-s) + "...");
         }
         System.out.println("Writing...");
-        checker.fixIndex(result, codec);
+        checker.fixIndex(result);
         System.out.println("OK");
         System.out.println("Wrote new segments file \"" + result.newSegments.getSegmentsFileName() + "\"");
       }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java Sun Aug 11 12:19:13 2013
@@ -30,12 +30,11 @@ import java.util.Comparator;
  *  separate thread.
  *
  *  <p>Specify the max number of threads that may run at
- *  once with {@link #setMaxThreadCount}.</p>
+ *  once, and the maximum number of simultaneous merges
+ *  with {@link #setMaxMergesAndThreads}.</p>
  *
- *  <p>Separately specify the maximum number of simultaneous
- *  merges with {@link #setMaxMergeCount}.  If the number of
- *  merges exceeds the max number of threads then the
- *  largest merges are paused until one of the smaller
+ *  <p>If the number of merges exceeds the max number of threads 
+ *  then the largest merges are paused until one of the smaller
  *  merges completes.</p>
  *
  *  <p>If more than {@link #getMaxMergeCount} merges are
@@ -49,21 +48,29 @@ public class ConcurrentMergeScheduler ex
 
   /** List of currently active {@link MergeThread}s. */
   protected List<MergeThread> mergeThreads = new ArrayList<MergeThread>();
+  
+  /** 
+   * Default {@code maxThreadCount}.
+   * We default to 1: tests on spinning-magnet drives showed slower
+   * indexing performance if more than one merge thread runs at
+   * once (though on an SSD it was faster)
+   */
+  public static final int DEFAULT_MAX_THREAD_COUNT = 1;
+  
+  /** Default {@code maxMergeCount}. */
+  public static final int DEFAULT_MAX_MERGE_COUNT = 2;
 
   // Max number of merge threads allowed to be running at
   // once.  When there are more merges then this, we
   // forcefully pause the larger ones, letting the smaller
   // ones run, up until maxMergeCount merges at which point
   // we forcefully pause incoming threads (that presumably
-  // are the ones causing so much merging).  We default to 1
-  // here: tests on spinning-magnet drives showed slower
-  // indexing perf if more than one merge thread runs at
-  // once (though on an SSD it was faster):
-  private int maxThreadCount = 1;
+  // are the ones causing so much merging).
+  private int maxThreadCount = DEFAULT_MAX_THREAD_COUNT;
 
   // Max number of merges we accept before forcefully
   // throttling the incoming threads
-  private int maxMergeCount = 2;
+  private int maxMergeCount = DEFAULT_MAX_MERGE_COUNT;
 
   /** {@link Directory} that holds the index. */
   protected Directory dir;
@@ -80,43 +87,40 @@ public class ConcurrentMergeScheduler ex
   public ConcurrentMergeScheduler() {
   }
 
-  /** Sets the max # simultaneous merge threads that should
-   *  be running at once.  This must be <= {@link
-   *  #setMaxMergeCount}. */
-  public void setMaxThreadCount(int count) {
-    if (count < 1) {
-      throw new IllegalArgumentException("count should be at least 1");
+  /**
+   * Sets the maximum number of merge threads and simultaneous merges allowed.
+   * 
+   * @param maxMergeCount the max # simultaneous merges that are allowed.
+   *       If a merge is necessary yet we already have this many
+   *       threads running, the incoming thread (that is calling
+   *       add/updateDocument) will block until a merge thread
+   *       has completed.  Note that we will only run the
+   *       smallest <code>maxThreadCount</code> merges at a time.
+   * @param maxThreadCount the max # simultaneous merge threads that should
+   *       be running at once.  This must be &lt;= <code>maxMergeCount</code>
+   */
+  public void setMaxMergesAndThreads(int maxMergeCount, int maxThreadCount) {
+    if (maxThreadCount < 1) {
+      throw new IllegalArgumentException("maxThreadCount should be at least 1");
     }
-    if (count > maxMergeCount) {
-      throw new IllegalArgumentException("count should be <= maxMergeCount (= " + maxMergeCount + ")");
+    if (maxMergeCount < 1) {
+      throw new IllegalArgumentException("maxMergeCount should be at least 1");
     }
-    maxThreadCount = count;
+    if (maxThreadCount > maxMergeCount) {
+      throw new IllegalArgumentException("maxThreadCount should be <= maxMergeCount (= " + maxMergeCount + ")");
+    }
+    this.maxThreadCount = maxThreadCount;
+    this.maxMergeCount = maxMergeCount;
   }
 
   /** Returns {@code maxThreadCount}.
    *
-   * @see #setMaxThreadCount(int) */
+   * @see #setMaxMergesAndThreads(int, int) */
   public int getMaxThreadCount() {
     return maxThreadCount;
   }
 
-  /** Sets the max # simultaneous merges that are allowed.
-   *  If a merge is necessary yet we already have this many
-   *  threads running, the incoming thread (that is calling
-   *  add/updateDocument) will block until a merge thread
-   *  has completed.  Note that we will only run the
-   *  smallest {@link #setMaxThreadCount} merges at a time. */
-  public void setMaxMergeCount(int count) {
-    if (count < 1) {
-      throw new IllegalArgumentException("count should be at least 1");
-    }
-    if (count < maxThreadCount) {
-      throw new IllegalArgumentException("count should be >= maxThreadCount (= " + maxThreadCount + ")");
-    }
-    maxMergeCount = count;
-  }
-
-  /** See {@link #setMaxMergeCount}. */
+  /** See {@link #setMaxMergesAndThreads}. */
   public int getMaxMergeCount() {
     return maxMergeCount;
   }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java Sun Aug 11 12:19:13 2013
@@ -19,6 +19,7 @@ package org.apache.lucene.index;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.nio.file.NoSuchFileException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
@@ -51,9 +52,6 @@ import org.apache.lucene.store.NoSuchDir
 */
 public abstract class DirectoryReader extends BaseCompositeReader<AtomicReader> {
 
-  /** Default termInfosIndexDivisor. */
-  public static final int DEFAULT_TERMS_INDEX_DIVISOR = 1;
-
   /** The index directory. */
   protected final Directory directory;
   
@@ -63,29 +61,7 @@ public abstract class DirectoryReader ex
    * @throws IOException if there is a low-level IO error
    */
   public static DirectoryReader open(final Directory directory) throws IOException {
-    return StandardDirectoryReader.open(directory, null, DEFAULT_TERMS_INDEX_DIVISOR);
-  }
-  
-  /** Expert: Returns a IndexReader reading the index in the given
-   *  Directory with the given termInfosIndexDivisor.
-   * @param directory the index directory
-   * @param termInfosIndexDivisor Subsamples which indexed
-   *  terms are loaded into RAM. This has the same effect as {@link
-   *  IndexWriterConfig#setTermIndexInterval} except that setting
-   *  must be done at indexing time while this setting can be
-   *  set per reader.  When set to N, then one in every
-   *  N*termIndexInterval terms in the index is loaded into
-   *  memory.  By setting this to a value > 1 you can reduce
-   *  memory usage, at the expense of higher latency when
-   *  loading a TermInfo.  The default value is 1.  Set this
-   *  to -1 to skip loading the terms index entirely.
-   *  <b>NOTE:</b> divisor settings &gt; 1 do not apply to all PostingsFormat
-   *  implementations, including the default one in this release. It only makes
-   *  sense for terms indexes that can efficiently re-sample terms at load time.
-   * @throws IOException if there is a low-level IO error
-   */
-  public static DirectoryReader open(final Directory directory, int termInfosIndexDivisor) throws IOException {
-    return StandardDirectoryReader.open(directory, null, termInfosIndexDivisor);
+    return StandardDirectoryReader.open(directory, null);
   }
   
   /**
@@ -117,29 +93,7 @@ public abstract class DirectoryReader ex
    * @throws IOException if there is a low-level IO error
    */
   public static DirectoryReader open(final IndexCommit commit) throws IOException {
-    return StandardDirectoryReader.open(commit.getDirectory(), commit, DEFAULT_TERMS_INDEX_DIVISOR);
-  }
-
-  /** Expert: returns an IndexReader reading the index in the given
-   *  {@link IndexCommit} and termInfosIndexDivisor.
-   * @param commit the commit point to open
-   * @param termInfosIndexDivisor Subsamples which indexed
-   *  terms are loaded into RAM. This has the same effect as {@link
-   *  IndexWriterConfig#setTermIndexInterval} except that setting
-   *  must be done at indexing time while this setting can be
-   *  set per reader.  When set to N, then one in every
-   *  N*termIndexInterval terms in the index is loaded into
-   *  memory.  By setting this to a value > 1 you can reduce
-   *  memory usage, at the expense of higher latency when
-   *  loading a TermInfo.  The default value is 1.  Set this
-   *  to -1 to skip loading the terms index entirely.
-   *  <b>NOTE:</b> divisor settings &gt; 1 do not apply to all PostingsFormat
-   *  implementations, including the default one in this release. It only makes
-   *  sense for terms indexes that can efficiently re-sample terms at load time.
-   * @throws IOException if there is a low-level IO error
-   */
-  public static DirectoryReader open(final IndexCommit commit, int termInfosIndexDivisor) throws IOException {
-    return StandardDirectoryReader.open(commit.getDirectory(), commit, termInfosIndexDivisor);
+    return StandardDirectoryReader.open(commit.getDirectory(), commit);
   }
 
   /**
@@ -291,7 +245,7 @@ public abstract class DirectoryReader ex
           // IOException allowed to throw there, in case
           // segments_N is corrupt
           sis.read(dir, fileName);
-        } catch (FileNotFoundException fnfe) {
+        } catch (FileNotFoundException | NoSuchFileException fnfe) {
           // LUCENE-948: on NFS (and maybe others), if
           // you have writers switching back and forth
           // between machines, it's very likely that the

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocConsumer.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocConsumer.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocConsumer.java Sun Aug 11 12:19:13 2013
@@ -24,5 +24,4 @@ abstract class DocConsumer {
   abstract void finishDocument() throws IOException;
   abstract void flush(final SegmentWriteState state) throws IOException;
   abstract void abort();
-  abstract void doAfterFlush();
 }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java Sun Aug 11 12:19:13 2013
@@ -144,15 +144,6 @@ final class DocFieldProcessor extends Do
     return fields;
   }
 
-  /** In flush we reset the fieldHash to not maintain per-field state
-   *  across segments */
-  @Override
-  void doAfterFlush() {
-    fieldHash = new DocFieldProcessorPerField[2];
-    hashMask = 1;
-    totalFieldCount = 0;
-  }
-
   private void rehash() {
     final int newHashSize = (fieldHash.length*2);
     assert newHashSize > fieldHash.length;

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessorPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessorPerField.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessorPerField.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessorPerField.java Sun Aug 11 12:19:13 2013
@@ -17,12 +17,7 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
-import java.util.HashMap;
-import java.util.Map;
-
 import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.RamUsageEstimator;
 
 /**
@@ -33,19 +28,16 @@ final class DocFieldProcessorPerField {
 
   final DocFieldConsumerPerField consumer;
   final FieldInfo fieldInfo;
-  private final Counter bytesUsed;
 
   DocFieldProcessorPerField next;
   int lastGen = -1;
 
   int fieldCount;
   IndexableField[] fields = new IndexableField[1];
-  private final Map<FieldInfo,String> dvFields = new HashMap<FieldInfo,String>();
 
   public DocFieldProcessorPerField(final DocFieldProcessor docFieldProcessor, final FieldInfo fieldInfo) {
     this.consumer = docFieldProcessor.consumer.addField(fieldInfo);
     this.fieldInfo = fieldInfo;
-    this.bytesUsed = docFieldProcessor.bytesUsed;
   }
 
   public void addField(IndexableField field) {

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocTermOrds.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocTermOrds.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocTermOrds.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocTermOrds.java Sun Aug 11 12:19:13 2013
@@ -659,7 +659,7 @@ public class DocTermOrds {
     }
 
     @Override
-    public SeekStatus seekCeil(BytesRef target, boolean useCache) throws IOException {
+    public SeekStatus seekCeil(BytesRef target) throws IOException {
 
       // already here
       if (term != null && term.equals(target)) {
@@ -729,7 +729,7 @@ public class DocTermOrds {
         //System.out.println("  do seek term=" + base.utf8ToString());
         ord = idx << indexIntervalBits;
         delta = (int) (targetOrd - ord);
-        final TermsEnum.SeekStatus seekStatus = termsEnum.seekCeil(base, true);
+        final TermsEnum.SeekStatus seekStatus = termsEnum.seekCeil(base);
         assert seekStatus == TermsEnum.SeekStatus.FOUND;
       } else {
         //System.out.println("seek w/in block");

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java Sun Aug 11 12:19:13 2013
@@ -29,15 +29,11 @@ import org.apache.lucene.index.Documents
 import org.apache.lucene.index.DocumentsWriterPerThread.IndexingChain;
 import org.apache.lucene.index.DocumentsWriterPerThreadPool.ThreadState;
 import org.apache.lucene.index.FieldInfos.FieldNumbers;
-import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.FlushInfo;
-import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.InfoStream;
-import org.apache.lucene.util.MutableBits;
 
 /**
  * This class accepts multiple added documents and directly
@@ -114,6 +110,7 @@ final class DocumentsWriter {
   List<String> newFiles;
 
   final IndexWriter indexWriter;
+  final LiveIndexWriterConfig indexWriterConfig;
 
   private AtomicInteger numDocsInRAM = new AtomicInteger(0);
 
@@ -144,6 +141,7 @@ final class DocumentsWriter {
     this.indexWriter = writer;
     this.infoStream = config.getInfoStream();
     this.similarity = config.getSimilarity();
+    this.indexWriterConfig = writer.getConfig();
     this.perThreadPool = config.getIndexerThreadPool();
     this.chain = config.getIndexingChain();
     this.perThreadPool.initialize(this, globalFieldNumbers, config);
@@ -517,7 +515,7 @@ final class DocumentsWriter {
     // buffer, force them all to apply now. This is to
     // prevent too-frequent flushing of a long tail of
     // tiny segments:
-    final double ramBufferSizeMB = indexWriter.getConfig().getRAMBufferSizeMB();
+    final double ramBufferSizeMB = indexWriterConfig.getRAMBufferSizeMB();
     if (ramBufferSizeMB != IndexWriterConfig.DISABLE_AUTO_FLUSH &&
         flushControl.getDeleteBytesUsed() > (1024*1024*ramBufferSizeMB/2)) {
       if (infoStream.isEnabled("DW")) {

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java Sun Aug 11 12:19:13 2013
@@ -112,7 +112,7 @@ class DocumentsWriterPerThread {
 
     // Only called by asserts
     public boolean testPoint(String name) {
-      return docWriter.writer.testPoint(name);
+      return docWriter.testPoint(name);
     }
 
     public void clear() {
@@ -194,6 +194,7 @@ class DocumentsWriterPerThread {
   private final NumberFormat nf = NumberFormat.getInstance(Locale.ROOT);
   final Allocator byteBlockAllocator;
   final IntBlockPool.Allocator intBlockAllocator;
+  private final LiveIndexWriterConfig indexWriterConfig;
 
   
   public DocumentsWriterPerThread(Directory directory, DocumentsWriter parent,
@@ -203,6 +204,7 @@ class DocumentsWriterPerThread {
     this.parent = parent;
     this.fieldInfos = fieldInfos;
     this.writer = parent.indexWriter;
+    this.indexWriterConfig = parent.indexWriterConfig;
     this.infoStream = parent.infoStream;
     this.codec = parent.codec;
     this.docState = new DocState(this, infoStream);
@@ -232,6 +234,13 @@ class DocumentsWriterPerThread {
     aborting = true;
   }
   
+  final boolean testPoint(String message) {
+    if (infoStream.isEnabled("TP")) {
+      infoStream.message("TP", message);
+    }
+    return true;
+  }
+  
   boolean checkAndResetHasAborted() {
     final boolean retval = hasAborted;
     hasAborted = false;
@@ -239,7 +248,7 @@ class DocumentsWriterPerThread {
   }
 
   public void updateDocument(IndexDocument doc, Analyzer analyzer, Term delTerm) throws IOException {
-    assert writer.testPoint("DocumentsWriterPerThread addDocument start");
+    assert testPoint("DocumentsWriterPerThread addDocument start");
     assert deleteQueue != null;
     docState.doc = doc;
     docState.analyzer = analyzer;
@@ -292,7 +301,7 @@ class DocumentsWriterPerThread {
   }
   
   public int updateDocuments(Iterable<? extends IndexDocument> docs, Analyzer analyzer, Term delTerm) throws IOException {
-    assert writer.testPoint("DocumentsWriterPerThread addDocuments start");
+    assert testPoint("DocumentsWriterPerThread addDocuments start");
     assert deleteQueue != null;
     docState.analyzer = analyzer;
     if (segmentInfo == null) {
@@ -394,8 +403,8 @@ class DocumentsWriterPerThread {
     ++numDocsInRAM;
   }
 
-  // Buffer a specific docID for deletion.  Currently only
-  // used when we hit a exception when adding a document
+  // Buffer a specific docID for deletion. Currently only
+  // used when we hit an exception when adding a document
   void deleteDocID(int docIDUpto) {
     pendingDeletes.addDocID(docIDUpto);
     // NOTE: we do not trigger flush here.  This is
@@ -428,7 +437,6 @@ class DocumentsWriterPerThread {
   /** Reset after a flush */
   private void doAfterFlush() {
     segmentInfo = null;
-    consumer.doAfterFlush();
     directory.getCreatedFiles().clear();
     fieldInfos = new FieldInfos.Builder(fieldInfos.globalFieldNumbers);
     parent.subtractFlushedNumDocs(numDocsInRAM);
@@ -460,7 +468,6 @@ class DocumentsWriterPerThread {
     assert deleteSlice == null : "all deletes must be applied in prepareFlush";
     segmentInfo.setDocCount(numDocsInRAM);
     flushState = new SegmentWriteState(infoStream, directory, segmentInfo, fieldInfos.finish(),
-        writer.getConfig().getTermIndexInterval(),
         pendingDeletes, new IOContext(new FlushInfo(numDocsInRAM, bytesUsed())));
     final double startMBUsed = parent.flushControl.netBytes() / 1024. / 1024.;
 
@@ -561,7 +568,7 @@ class DocumentsWriterPerThread {
 
     boolean success = false;
     try {
-      if (writer.useCompoundFile(newSegment)) {
+      if (indexWriterConfig.getUseCompoundFile()) {
 
         // Now build compound file
         Collection<String> oldFiles = IndexWriter.createCompoundFile(infoStream, directory, MergeState.CheckAbort.NONE, newSegment.info, context);

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java Sun Aug 11 12:19:13 2013
@@ -147,7 +147,10 @@ abstract class DocumentsWriterPerThreadP
   @Override
   public DocumentsWriterPerThreadPool clone() {
     // We should only be cloned before being used:
-    assert numThreadStatesActive == 0;
+    if (numThreadStatesActive != 0) {
+      throw new IllegalStateException("clone this object before it is used!");
+    }
+    
     DocumentsWriterPerThreadPool clone;
     try {
       clone = (DocumentsWriterPerThreadPool) super.clone();

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java Sun Aug 11 12:19:13 2013
@@ -157,8 +157,8 @@ public class FilterAtomicReader extends 
     }
 
     @Override
-    public SeekStatus seekCeil(BytesRef text, boolean useCache) throws IOException {
-      return in.seekCeil(text, useCache);
+    public SeekStatus seekCeil(BytesRef text) throws IOException {
+      return in.seekCeil(text);
     }
 
     @Override

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FilteredTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FilteredTermsEnum.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FilteredTermsEnum.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FilteredTermsEnum.java Sun Aug 11 12:19:13 2013
@@ -154,7 +154,7 @@ public abstract class FilteredTermsEnum 
    *         support seeking.
    */
   @Override
-  public boolean seekExact(BytesRef term, boolean useCache) throws IOException {
+  public boolean seekExact(BytesRef term) throws IOException {
     throw new UnsupportedOperationException(getClass().getName()+" does not support seeking");
   }
 
@@ -163,7 +163,7 @@ public abstract class FilteredTermsEnum 
    *         support seeking.
    */
   @Override
-  public SeekStatus seekCeil(BytesRef term, boolean useCache) throws IOException {
+  public SeekStatus seekCeil(BytesRef term) throws IOException {
     throw new UnsupportedOperationException(getClass().getName()+" does not support seeking");
   }
 
@@ -222,7 +222,7 @@ public abstract class FilteredTermsEnum 
         //System.out.println("  seek to t=" + (t == null ? "null" : t.utf8ToString()) + " tenum=" + tenum);
         // Make sure we always seek forward:
         assert actualTerm == null || t == null || getComparator().compare(t, actualTerm) > 0: "curTerm=" + actualTerm + " seekTerm=" + t;
-        if (t == null || tenum.seekCeil(t, false) == SeekStatus.END) {
+        if (t == null || tenum.seekCeil(t) == SeekStatus.END) {
           // no more terms to seek to or enum exhausted
           //System.out.println("  return null");
           return null;

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java Sun Aug 11 12:19:13 2013
@@ -20,23 +20,32 @@ package org.apache.lucene.index;
 import org.apache.lucene.index.DocumentsWriterPerThreadPool.ThreadState;
 
 /**
- * Default {@link FlushPolicy} implementation that flushes based on RAM used,
- * document count and number of buffered deletes depending on the IndexWriter's
- * {@link IndexWriterConfig}.
+ * Default {@link FlushPolicy} implementation that flushes new segments based on
+ * RAM used and document count depending on the IndexWriter's
+ * {@link IndexWriterConfig}. It also applies pending deletes based on the
+ * number of buffered delete terms.
  * 
  * <ul>
- * <li>{@link #onDelete(DocumentsWriterFlushControl, DocumentsWriterPerThreadPool.ThreadState)} - flushes
- * based on the global number of buffered delete terms iff
- * {@link IndexWriterConfig#getMaxBufferedDeleteTerms()} is enabled</li>
- * <li>{@link #onInsert(DocumentsWriterFlushControl, DocumentsWriterPerThreadPool.ThreadState)} - flushes
- * either on the number of documents per {@link DocumentsWriterPerThread} (
+ * <li>
+ * {@link #onDelete(DocumentsWriterFlushControl, DocumentsWriterPerThreadPool.ThreadState)}
+ * - applies pending delete operations based on the global number of buffered
+ * delete terms iff {@link IndexWriterConfig#getMaxBufferedDeleteTerms()} is
+ * enabled</li>
+ * <li>
+ * {@link #onInsert(DocumentsWriterFlushControl, DocumentsWriterPerThreadPool.ThreadState)}
+ * - flushes either on the number of documents per
+ * {@link DocumentsWriterPerThread} (
  * {@link DocumentsWriterPerThread#getNumDocsInRAM()}) or on the global active
  * memory consumption in the current indexing session iff
  * {@link IndexWriterConfig#getMaxBufferedDocs()} or
  * {@link IndexWriterConfig#getRAMBufferSizeMB()} is enabled respectively</li>
- * <li>{@link #onUpdate(DocumentsWriterFlushControl, DocumentsWriterPerThreadPool.ThreadState)} - calls
- * {@link #onInsert(DocumentsWriterFlushControl, DocumentsWriterPerThreadPool.ThreadState)} and
- * {@link #onDelete(DocumentsWriterFlushControl, DocumentsWriterPerThreadPool.ThreadState)} in order</li>
+ * <li>
+ * {@link #onUpdate(DocumentsWriterFlushControl, DocumentsWriterPerThreadPool.ThreadState)}
+ * - calls
+ * {@link #onInsert(DocumentsWriterFlushControl, DocumentsWriterPerThreadPool.ThreadState)}
+ * and
+ * {@link #onDelete(DocumentsWriterFlushControl, DocumentsWriterPerThreadPool.ThreadState)}
+ * in order</li>
  * </ul>
  * All {@link IndexWriterConfig} settings are used to mark
  * {@link DocumentsWriterPerThread} as flush pending during indexing with

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FlushPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FlushPolicy.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FlushPolicy.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FlushPolicy.java Sun Aug 11 12:19:13 2013
@@ -32,18 +32,19 @@ import org.apache.lucene.util.SetOnce;
  * {@link IndexWriterConfig#setRAMBufferSizeMB(double)}</li>
  * <li>Number of RAM resident documents - configured via
  * {@link IndexWriterConfig#setMaxBufferedDocs(int)}</li>
- * <li>Number of buffered delete terms/queries - configured via
- * {@link IndexWriterConfig#setMaxBufferedDeleteTerms(int)}</li>
  * </ul>
- * 
- * The {@link IndexWriter} consults a provided {@link FlushPolicy} to control the
- * flushing process. The policy is informed for each added or
- * updated document as well as for each delete term. Based on the
- * {@link FlushPolicy}, the information provided via {@link ThreadState} and
+ * The policy also applies pending delete operations (by term and/or query),
+ * given the threshold set in
+ * {@link IndexWriterConfig#setMaxBufferedDeleteTerms(int)}.
+ * <p>
+ * {@link IndexWriter} consults the provided {@link FlushPolicy} to control the
+ * flushing process. The policy is informed for each added or updated document
+ * as well as for each delete term. Based on the {@link FlushPolicy}, the
+ * information provided via {@link ThreadState} and
  * {@link DocumentsWriterFlushControl}, the {@link FlushPolicy} decides if a
- * {@link DocumentsWriterPerThread} needs flushing and mark it as
- * flush-pending via
- * {@link DocumentsWriterFlushControl#setFlushPending(DocumentsWriterPerThreadPool.ThreadState)}.
+ * {@link DocumentsWriterPerThread} needs flushing and mark it as flush-pending
+ * via {@link DocumentsWriterFlushControl#setFlushPending}, or if deletes need
+ * to be applied.
  * 
  * @see ThreadState
  * @see DocumentsWriterFlushControl

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java Sun Aug 11 12:19:13 2013
@@ -415,7 +415,7 @@ final class FreqProxTermsWriterPerField 
       // which all share the same term.  Now we must
       // interleave the docID streams.
       int docFreq = 0;
-      long totTF = 0;
+      long totalTermFreq = 0;
       int docID = 0;
 
       while(true) {
@@ -483,7 +483,7 @@ final class FreqProxTermsWriterPerField 
           }
         }
 
-        totTF += termFreq;
+        totalTermFreq += termFreq;
         
         // Carefully copy over the prox + payload info,
         // changing the format to match Lucene's segment
@@ -540,8 +540,8 @@ final class FreqProxTermsWriterPerField 
         }
         postingsConsumer.finishDoc();
       }
-      termsConsumer.finishTerm(text, new TermStats(docFreq, writeTermFreq ? totTF : -1));
-      sumTotalTermFreq += totTF;
+      termsConsumer.finishTerm(text, new TermStats(docFreq, writeTermFreq ? totalTermFreq : -1));
+      sumTotalTermFreq += totalTermFreq;
       sumDocFreq += docFreq;
     }
 

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/IndexDeletionPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/IndexDeletionPolicy.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/IndexDeletionPolicy.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/IndexDeletionPolicy.java Sun Aug 11 12:19:13 2013
@@ -20,6 +20,8 @@ package org.apache.lucene.index;
 import java.util.List;
 import java.io.IOException;
 
+import org.apache.lucene.store.Directory;
+
 /**
  * <p>Expert: policy for deletion of stale {@link IndexCommit index commits}. 
  * 
@@ -46,6 +48,10 @@ import java.io.IOException;
  * target="top"
  * href="http://issues.apache.org/jira/browse/LUCENE-710">LUCENE-710</a>
  * for details.</p>
+ *
+ * <p>Implementers of sub-classes should make sure that {@link #clone()}
+ * returns an independent instance able to work with any other {@link IndexWriter}
+ * or {@link Directory} instance.</p>
  */
 
 public abstract class IndexDeletionPolicy implements Cloneable {
@@ -72,6 +78,8 @@ public abstract class IndexDeletionPolic
    * @param commits List of current 
    * {@link IndexCommit point-in-time commits},
    *  sorted by age (the 0th one is the oldest commit).
+   *  Note that for a new index this method is invoked with
+   *  an empty list.
    */
   public abstract void onInit(List<? extends IndexCommit> commits) throws IOException;
 

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java Sun Aug 11 12:19:13 2013
@@ -20,6 +20,7 @@ package org.apache.lucene.index;
 import java.io.Closeable;
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.nio.file.NoSuchFileException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -170,7 +171,7 @@ final class IndexFileDeleter implements 
             SegmentInfos sis = new SegmentInfos();
             try {
               sis.read(directory, fileName);
-            } catch (FileNotFoundException e) {
+            } catch (FileNotFoundException | NoSuchFileException e) {
               // LUCENE-948: on NFS (and maybe others), if
               // you have writers switching back and forth
               // between machines, it's very likely that the

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java Sun Aug 11 12:19:13 2013
@@ -214,7 +214,7 @@ public class IndexWriter implements Clos
   private final Analyzer analyzer;    // how to analyze text
 
   private volatile long changeCount; // increments every time a change is completed
-  private long lastCommitChangeCount; // last changeCount that was committed
+  private volatile long lastCommitChangeCount; // last changeCount that was committed
 
   private List<SegmentInfoPerCommit> rollbackSegments;      // list of segmentInfo we will fallback to if the commit fails
 
@@ -630,10 +630,13 @@ public class IndexWriter implements Clos
 
   /**
    * Constructs a new IndexWriter per the settings given in <code>conf</code>.
-   * Note that the passed in {@link IndexWriterConfig} is
-   * privately cloned; if you need to make subsequent "live"
-   * changes to the configuration use {@link #getConfig}.
+   * If you want to make "live" changes to this writer instance, use
+   * {@link #getConfig()}.
+   * 
    * <p>
+   * <b>NOTE:</b> after ths writer is created, the given configuration instance
+   * cannot be passed to another writer. If you intend to do so, you should
+   * {@link IndexWriterConfig#clone() clone} it beforehand.
    * 
    * @param d
    *          the index directory. The index is either created or appended
@@ -648,7 +651,8 @@ public class IndexWriter implements Clos
    *           IO error
    */
   public IndexWriter(Directory d, IndexWriterConfig conf) throws IOException {
-    config = new LiveIndexWriterConfig(conf.clone());
+    conf.setIndexWriter(this); // prevent reuse by other instances
+    config = new LiveIndexWriterConfig(conf);
     directory = d;
     analyzer = config.getAnalyzer();
     infoStream = config.getInfoStream();
@@ -2269,10 +2273,6 @@ public class IndexWriter implements Clos
     }
   }
 
-  synchronized boolean useCompoundFile(SegmentInfoPerCommit segmentInfo) throws IOException {
-    return mergePolicy.useCompoundFile(segmentInfos, segmentInfo);
-  }
-
   private synchronized void resetMergeExceptions() {
     mergeExceptions = new ArrayList<MergePolicy.OneMerge>();
     mergeGen++;
@@ -2428,15 +2428,16 @@ public class IndexWriter implements Clos
    * close the writer. See <a href="#OOME">above</a> for details.
    * 
    * <p>
+   * <b>NOTE:</b> empty segments are dropped by this method and not added to this
+   * index.
+   * 
+   * <p>
    * <b>NOTE:</b> this method merges all given {@link IndexReader}s in one
    * merge. If you intend to merge a large number of readers, it may be better
    * to call this method multiple times, each time with a small set of readers.
    * In principle, if you use a merge policy with a {@code mergeFactor} or
    * {@code maxMergeAtOnce} parameter, you should pass that many readers in one
-   * call. Also, if the given readers are {@link DirectoryReader}s, they can be
-   * opened with {@code termIndexInterval=-1} to save RAM, since during merge
-   * the in-memory structure is not used. See
-   * {@link DirectoryReader#open(Directory, int)}.
+   * call.
    * 
    * <p>
    * <b>NOTE</b>: if you call {@link #close(boolean)} with <tt>false</tt>, which
@@ -2461,11 +2462,20 @@ public class IndexWriter implements Clos
       String mergedName = newSegmentName();
       final List<AtomicReader> mergeReaders = new ArrayList<AtomicReader>();
       for (IndexReader indexReader : readers) {
-        numDocs += indexReader.numDocs();
-        for (AtomicReaderContext ctx : indexReader.leaves()) {
-          mergeReaders.add(ctx.reader());
+        if (indexReader.numDocs() > 0) {
+          numDocs += indexReader.numDocs();
+          for (AtomicReaderContext ctx : indexReader.leaves()) {
+            if (ctx.reader().numDocs() > 0) { // drop empty (or all deleted) segments
+              mergeReaders.add(ctx.reader());
+            }
+          }
         }
       }
+      
+      if (mergeReaders.isEmpty()) { // no segments with documents to add
+        return;
+      }
+      
       final IOContext context = new IOContext(new MergeInfo(numDocs, -1, true, -1));
 
       // TODO: somehow we should fix this merge so it's
@@ -2475,7 +2485,7 @@ public class IndexWriter implements Clos
       SegmentInfo info = new SegmentInfo(directory, Constants.LUCENE_MAIN_VERSION, mergedName, -1,
                                          false, codec, null, null);
 
-      SegmentMerger merger = new SegmentMerger(mergeReaders, info, infoStream, trackingDir, config.getTermIndexInterval(),
+      SegmentMerger merger = new SegmentMerger(mergeReaders, info, infoStream, trackingDir,
                                                MergeState.CheckAbort.NONE, globalFieldNumberMap, context);
 
       MergeState mergeState;
@@ -2826,6 +2836,11 @@ public class IndexWriter implements Clos
     commitInternal();
   }
 
+  /** Returns true if there are changes that have not been committed */
+  public final boolean hasUncommittedChanges() {
+    return changeCount != lastCommitChangeCount;
+  }
+
   private final void commitInternal() throws IOException {
 
     if (infoStream.isEnabled("IW")) {
@@ -2865,8 +2880,8 @@ public class IndexWriter implements Clos
         if (infoStream.isEnabled("IW")) {
           infoStream.message("IW", "commit: wrote segments file \"" + pendingCommit.getSegmentsFileName() + "\"");
         }
-        lastCommitChangeCount = pendingCommitChangeCount;
         segmentInfos.updateGeneration(pendingCommit);
+        lastCommitChangeCount = pendingCommitChangeCount;
         rollbackSegments = pendingCommit.createBackupSegmentInfos();
         deleter.checkpoint(pendingCommit, true);
       } finally {
@@ -3652,7 +3667,7 @@ public class IndexWriter implements Clos
         // Hold onto the "live" reader; we will use this to
         // commit merged deletes
         final ReadersAndLiveDocs rld = readerPool.get(info, true);
-        SegmentReader reader = rld.getMergeReader(context);
+        SegmentReader reader = rld.getReader(context);
         assert reader != null;
 
         // Carefully pull the most recent live docs:
@@ -3709,7 +3724,7 @@ public class IndexWriter implements Clos
       // we pass merge.getMergeReaders() instead of merge.readers to allow the
       // OneMerge to return a view over the actual segments to merge
       final SegmentMerger merger = new SegmentMerger(merge.getMergeReaders(),
-          merge.info.info, infoStream, dirWrapper, config.getTermIndexInterval(),
+          merge.info.info, infoStream, dirWrapper,
           checkAbort, globalFieldNumberMap, context);
 
       merge.checkAborted(directory);
@@ -4162,7 +4177,10 @@ public class IndexWriter implements Clos
   //   startCommitMergeDeletes
   //   startMergeInit
   //   DocumentsWriter.ThreadState.init start
-  boolean testPoint(String name) {
+  private final boolean testPoint(String message) {
+    if (infoStream.isEnabled("TP")) {
+      infoStream.message("TP", message);
+    }
     return true;
   }