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 [11/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-too...

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/AbstractAppendingLongBuffer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/AbstractAppendingLongBuffer.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/AbstractAppendingLongBuffer.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/AbstractAppendingLongBuffer.java Sun Aug 11 12:19:13 2013
@@ -17,50 +17,68 @@ package org.apache.lucene.util.packed;
  * limitations under the License.
  */
 
-import java.util.Arrays;
-
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.RamUsageEstimator;
 
-/** Common functionality shared by {@link AppendingLongBuffer} and {@link MonotonicAppendingLongBuffer}. */
+import java.util.Arrays;
+
+import static org.apache.lucene.util.packed.PackedInts.checkBlockSize;
+
+/** Common functionality shared by {@link AppendingDeltaPackedLongBuffer} and {@link MonotonicAppendingLongBuffer}. */
 abstract class AbstractAppendingLongBuffer {
 
-  static final int BLOCK_BITS = 10;
-  static final int MAX_PENDING_COUNT = 1 << BLOCK_BITS;
-  static final int BLOCK_MASK = MAX_PENDING_COUNT - 1;
-
-  long[] minValues;
-  PackedInts.Reader[] deltas;
-  private long deltasBytes;
+  static final int MIN_PAGE_SIZE = 64;
+  // More than 1M doesn't really makes sense with these appending buffers
+  // since their goal is to try to have small numbers of bits per value
+  static final int MAX_PAGE_SIZE = 1 << 20;
+
+  final int pageShift, pageMask;
+  PackedInts.Reader[] values;
+  private long valuesBytes;
   int valuesOff;
   long[] pending;
   int pendingOff;
+  float acceptableOverheadRatio;
 
-  AbstractAppendingLongBuffer(int initialBlockCount) {
-    minValues = new long[16];
-    deltas = new PackedInts.Reader[16];
-    pending = new long[MAX_PENDING_COUNT];
+  AbstractAppendingLongBuffer(int initialBlockCount, int pageSize, float acceptableOverheadRatio) {
+    values = new PackedInts.Reader[initialBlockCount];
+    pending = new long[pageSize];
+    pageShift = checkBlockSize(pageSize, MIN_PAGE_SIZE, MAX_PAGE_SIZE);
+    pageMask = pageSize - 1;
     valuesOff = 0;
     pendingOff = 0;
+    this.acceptableOverheadRatio = acceptableOverheadRatio;
+  }
+
+  final int pageSize() {
+    return pageMask + 1;
   }
 
   /** Get the number of values that have been added to the buffer. */
   public final long size() {
-    return valuesOff * (long) MAX_PENDING_COUNT + pendingOff;
+    long size = pendingOff;
+    if (valuesOff > 0) {
+      size += values[valuesOff - 1].size();
+    }
+    if (valuesOff > 1) {
+      size += (long) (valuesOff - 1) * pageSize();
+    }
+    return size;
   }
 
   /** Append a value to this buffer. */
   public final void add(long l) {
-    if (pendingOff == MAX_PENDING_COUNT) {
+    if (pending == null) {
+      throw new IllegalStateException("This buffer is frozen");
+    }
+    if (pendingOff == pending.length) {
       // check size
-      if (deltas.length == valuesOff) {
+      if (values.length == valuesOff) {
         final int newLength = ArrayUtil.oversize(valuesOff + 1, 8);
         grow(newLength);
       }
       packPendingValues();
-      if (deltas[valuesOff] != null) {
-        deltasBytes += deltas[valuesOff].ramBytesUsed();
-      }
+      valuesBytes += values[valuesOff].ramBytesUsed();
       ++valuesOff;
       // reset pending buffer
       pendingOff = 0;
@@ -69,57 +87,90 @@ abstract class AbstractAppendingLongBuff
   }
 
   void grow(int newBlockCount) {
-    minValues = Arrays.copyOf(minValues, newBlockCount);
-    deltas = Arrays.copyOf(deltas, newBlockCount);
+    values = Arrays.copyOf(values, newBlockCount);
   }
 
   abstract void packPendingValues();
 
   /** Get a value from this buffer. */
   public final long get(long index) {
-    if (index < 0 || index >= size()) {
-      throw new IndexOutOfBoundsException("" + index);
-    }
-    int block = (int) (index >> BLOCK_BITS);
-    int element = (int) (index & BLOCK_MASK);
+    assert index >= 0 && index < size();
+    final int block = (int) (index >> pageShift);
+    final int element = (int) (index & pageMask);
     return get(block, element);
   }
 
+  /**
+   * Bulk get: read at least one and at most <code>len</code> longs starting
+   * from <code>index</code> into <code>arr[off:off+len]</code> and return
+   * the actual number of values that have been read.
+   */
+  public final int get(long index, long[] arr, int off, int len) {
+    assert len > 0 : "len must be > 0 (got " + len + ")";
+    assert index >= 0 && index < size();
+    assert off + len <= arr.length;
+
+    int block = (int) (index >> pageShift);
+    int element = (int) (index & pageMask);
+    return get(block, element, arr, off, len);
+  }
+
+
   abstract long get(int block, int element);
 
-  abstract Iterator iterator();
+  abstract int get(int block, int element, long[] arr, int off, int len);
 
-  abstract class Iterator {
+
+  /** Return an iterator over the values of this buffer. */
+  public Iterator iterator() {
+    return new Iterator();
+  }
+
+  final public class Iterator {
 
     long[] currentValues;
     int vOff, pOff;
+    int currentCount; // number of entries of the current page
 
     Iterator() {
       vOff = pOff = 0;
       if (valuesOff == 0) {
         currentValues = pending;
+        currentCount = pendingOff;
       } else {
-        currentValues = new long[MAX_PENDING_COUNT];
+        currentValues = new long[values[0].size()];
         fillValues();
       }
     }
 
-    abstract void fillValues();
+    void fillValues() {
+      if (vOff == valuesOff) {
+        currentValues = pending;
+        currentCount = pendingOff;
+      } else {
+        currentCount = values[vOff].size();
+        for (int k = 0; k < currentCount; ) {
+          k += get(vOff, k, currentValues, k, currentCount - k);
+        }
+      }
+    }
 
     /** Whether or not there are remaining values. */
     public final boolean hasNext() {
-      return vOff < valuesOff || (vOff == valuesOff && pOff < pendingOff);
+      return pOff < currentCount;
     }
 
     /** Return the next long in the buffer. */
     public final long next() {
       assert hasNext();
       long result = currentValues[pOff++];
-      if (pOff == MAX_PENDING_COUNT) {
+      if (pOff == currentCount) {
         vOff += 1;
         pOff = 0;
         if (vOff <= valuesOff) {
           fillValues();
+        } else {
+          currentCount = 0;
         }
       }
       return result;
@@ -129,22 +180,35 @@ abstract class AbstractAppendingLongBuff
 
   long baseRamBytesUsed() {
     return RamUsageEstimator.NUM_BYTES_OBJECT_HEADER
-        + 3 * RamUsageEstimator.NUM_BYTES_OBJECT_REF // the 3 arrays
-        + 2 * RamUsageEstimator.NUM_BYTES_INT; // the 2 offsets
+        + 2 * RamUsageEstimator.NUM_BYTES_OBJECT_REF // the 2 arrays
+        + 2 * RamUsageEstimator.NUM_BYTES_INT // the 2 offsets
+        + 2 * RamUsageEstimator.NUM_BYTES_INT // pageShift, pageMask
+        + RamUsageEstimator.NUM_BYTES_FLOAT   // acceptable overhead
+        + RamUsageEstimator.NUM_BYTES_LONG; // valuesBytes
   }
 
-  /**
-   * Return the number of bytes used by this instance.
-   */
+  /** Return the number of bytes used by this instance. */
   public long ramBytesUsed() {
     // TODO: this is called per-doc-per-norms/dv-field, can we optimize this?
     long bytesUsed = RamUsageEstimator.alignObjectSize(baseRamBytesUsed())
-        + RamUsageEstimator.NUM_BYTES_LONG // valuesBytes
-        + RamUsageEstimator.sizeOf(pending)
-        + RamUsageEstimator.sizeOf(minValues)
-        + RamUsageEstimator.alignObjectSize(RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + (long) RamUsageEstimator.NUM_BYTES_OBJECT_REF * deltas.length); // values
+        + (pending != null ? RamUsageEstimator.sizeOf(pending) : 0L)
+        + RamUsageEstimator.alignObjectSize(RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + (long) RamUsageEstimator.NUM_BYTES_OBJECT_REF * values.length); // values
+
+    return bytesUsed + valuesBytes;
+  }
 
-    return bytesUsed + deltasBytes;
+  /** Pack all pending values in this buffer. Subsequent calls to {@link #add(long)} will fail. */
+  public void freeze() {
+    if (pendingOff > 0) {
+      if (values.length == valuesOff) {
+        grow(valuesOff + 1); // don't oversize!
+      }
+      packPendingValues();
+      valuesBytes += values[valuesOff].ramBytesUsed();
+      ++valuesOff;
+      pendingOff = 0;
+    }
+    pending = null;
   }
 
 }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/AbstractBlockPackedWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/AbstractBlockPackedWriter.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/AbstractBlockPackedWriter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/AbstractBlockPackedWriter.java Sun Aug 11 12:19:13 2013
@@ -17,6 +17,8 @@ package org.apache.lucene.util.packed;
  * limitations under the License.
  */
 
+import static org.apache.lucene.util.packed.PackedInts.checkBlockSize;
+
 import java.io.IOException;
 import java.util.Arrays;
 
@@ -24,22 +26,11 @@ import org.apache.lucene.store.DataOutpu
 
 abstract class AbstractBlockPackedWriter {
 
+  static final int MIN_BLOCK_SIZE = 64;
   static final int MAX_BLOCK_SIZE = 1 << (30 - 3);
   static final int MIN_VALUE_EQUALS_0 = 1 << 0;
   static final int BPV_SHIFT = 1;
 
-  static void checkBlockSize(int blockSize) {
-    if (blockSize <= 0 || blockSize > MAX_BLOCK_SIZE) {
-      throw new IllegalArgumentException("blockSize must be > 0 and < " + MAX_BLOCK_SIZE + ", got " + blockSize);
-    }
-    if (blockSize < 64) {
-      throw new IllegalArgumentException("blockSize must be >= 64, got " + blockSize);
-    }
-    if ((blockSize & (blockSize - 1)) != 0) {
-      throw new IllegalArgumentException("blockSize must be a power of two, got " + blockSize);
-    }
-  }
-
   static long zigZagEncode(long n) {
     return (n >> 63) ^ (n << 1);
   }
@@ -66,7 +57,7 @@ abstract class AbstractBlockPackedWriter
    * @param blockSize the number of values of a single block, must be a multiple of <tt>64</tt>
    */
   public AbstractBlockPackedWriter(DataOutput out, int blockSize) {
-    checkBlockSize(blockSize);
+    checkBlockSize(blockSize, MIN_BLOCK_SIZE, MAX_BLOCK_SIZE);
     reset(out);
     values = new long[blockSize];
   }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/BlockPackedReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/BlockPackedReader.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/BlockPackedReader.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/BlockPackedReader.java Sun Aug 11 12:19:13 2013
@@ -17,11 +17,14 @@ package org.apache.lucene.util.packed;
  * limitations under the License.
  */
 
+import static org.apache.lucene.util.packed.AbstractBlockPackedWriter.BPV_SHIFT;
+import static org.apache.lucene.util.packed.AbstractBlockPackedWriter.MAX_BLOCK_SIZE;
+import static org.apache.lucene.util.packed.AbstractBlockPackedWriter.MIN_BLOCK_SIZE;
+import static org.apache.lucene.util.packed.AbstractBlockPackedWriter.MIN_VALUE_EQUALS_0;
 import static org.apache.lucene.util.packed.BlockPackedReaderIterator.readVLong;
 import static org.apache.lucene.util.packed.BlockPackedReaderIterator.zigZagDecode;
-import static org.apache.lucene.util.packed.BlockPackedWriter.BPV_SHIFT;
-import static org.apache.lucene.util.packed.BlockPackedWriter.MIN_VALUE_EQUALS_0;
-import static org.apache.lucene.util.packed.BlockPackedWriter.checkBlockSize;
+import static org.apache.lucene.util.packed.PackedInts.checkBlockSize;
+import static org.apache.lucene.util.packed.PackedInts.numBlocks;
 
 import java.io.IOException;
 
@@ -40,14 +43,10 @@ public final class BlockPackedReader {
 
   /** Sole constructor. */
   public BlockPackedReader(IndexInput in, int packedIntsVersion, int blockSize, long valueCount, boolean direct) throws IOException {
-    checkBlockSize(blockSize);
     this.valueCount = valueCount;
-    blockShift = Integer.numberOfTrailingZeros(blockSize);
+    blockShift = checkBlockSize(blockSize, MIN_BLOCK_SIZE, MAX_BLOCK_SIZE);
     blockMask = blockSize - 1;
-    final int numBlocks = (int) (valueCount / blockSize) + (valueCount % blockSize == 0 ? 0 : 1);
-    if ((long) numBlocks * blockSize < valueCount) {
-      throw new IllegalArgumentException("valueCount is too large for this block size");
-    }
+    final int numBlocks = numBlocks(valueCount, blockSize);
     long[] minValues = null;
     subReaders = new PackedInts.Reader[numBlocks];
     for (int i = 0; i < numBlocks; ++i) {

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/BlockPackedReaderIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/BlockPackedReaderIterator.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/BlockPackedReaderIterator.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/BlockPackedReaderIterator.java Sun Aug 11 12:19:13 2013
@@ -17,9 +17,13 @@ package org.apache.lucene.util.packed;
  * limitations under the License.
  */
 
-import static org.apache.lucene.util.packed.BlockPackedWriter.BPV_SHIFT;
-import static org.apache.lucene.util.packed.BlockPackedWriter.MIN_VALUE_EQUALS_0;
-import static org.apache.lucene.util.packed.BlockPackedWriter.checkBlockSize;
+import static org.apache.lucene.util.packed.AbstractBlockPackedWriter.BPV_SHIFT;
+import static org.apache.lucene.util.packed.AbstractBlockPackedWriter.MAX_BLOCK_SIZE;
+import static org.apache.lucene.util.packed.AbstractBlockPackedWriter.MIN_BLOCK_SIZE;
+import static org.apache.lucene.util.packed.AbstractBlockPackedWriter.MIN_VALUE_EQUALS_0;
+import static org.apache.lucene.util.packed.BlockPackedReaderIterator.readVLong;
+import static org.apache.lucene.util.packed.BlockPackedReaderIterator.zigZagDecode;
+import static org.apache.lucene.util.packed.PackedInts.checkBlockSize;
 
 import java.io.EOFException;
 import java.io.IOException;
@@ -87,7 +91,7 @@ public final class BlockPackedReaderIter
    *                  been used to write the stream
    */
   public BlockPackedReaderIterator(DataInput in, int packedIntsVersion, int blockSize, long valueCount) {
-    checkBlockSize(blockSize);
+    checkBlockSize(blockSize, MIN_BLOCK_SIZE, MAX_BLOCK_SIZE);
     this.packedIntsVersion = packedIntsVersion;
     this.blockSize = blockSize;
     this.values = new long[blockSize];

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/GrowableWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/GrowableWriter.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/GrowableWriter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/GrowableWriter.java Sun Aug 11 12:19:13 2013
@@ -20,24 +20,35 @@ package org.apache.lucene.util.packed;
 import java.io.IOException;
 
 import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.RamUsageEstimator;
 
 /**     
  * Implements {@link PackedInts.Mutable}, but grows the
  * bit count of the underlying packed ints on-demand.
+ * <p>Beware that this class will accept to set negative values but in order
+ * to do this, it will grow the number of bits per value to 64.
  *
  * <p>@lucene.internal</p>
  */
-
 public class GrowableWriter implements PackedInts.Mutable {
 
-  private long currentMaxValue;
+  private long currentMask;
   private PackedInts.Mutable current;
   private final float acceptableOverheadRatio;
 
+  /**
+   * @param startBitsPerValue       the initial number of bits per value, may grow depending on the data
+   * @param valueCount              the number of values
+   * @param acceptableOverheadRatio an acceptable overhead ratio
+   */
   public GrowableWriter(int startBitsPerValue, int valueCount, float acceptableOverheadRatio) {
     this.acceptableOverheadRatio = acceptableOverheadRatio;
     current = PackedInts.getMutable(valueCount, startBitsPerValue, this.acceptableOverheadRatio);
-    currentMaxValue = PackedInts.maxValue(current.getBitsPerValue());
+    currentMask = mask(current.getBitsPerValue());
+  }
+
+  private static long mask(int bitsPerValue) {
+    return bitsPerValue == 64 ? ~0L : PackedInts.maxValue(bitsPerValue);
   }
 
   @Override
@@ -70,16 +81,16 @@ public class GrowableWriter implements P
   }
 
   private void ensureCapacity(long value) {
-    assert value >= 0;
-    if (value <= currentMaxValue) {
+    if ((value & currentMask) == value) {
       return;
     }
-    final int bitsRequired = PackedInts.bitsRequired(value);
+    final int bitsRequired = value < 0 ? 64 : PackedInts.bitsRequired(value);
+    assert bitsRequired > current.getBitsPerValue();
     final int valueCount = size();
     PackedInts.Mutable next = PackedInts.getMutable(valueCount, bitsRequired, acceptableOverheadRatio);
     PackedInts.copy(current, 0, next, 0, valueCount, PackedInts.DEFAULT_BUFFER_SIZE);
     current = next;
-    currentMaxValue = PackedInts.maxValue(current.getBitsPerValue());
+    currentMask = mask(current.getBitsPerValue());
   }
 
   @Override
@@ -109,6 +120,10 @@ public class GrowableWriter implements P
   public int set(int index, long[] arr, int off, int len) {
     long max = 0;
     for (int i = off, end = off + len; i < end; ++i) {
+      // bitwise or is nice because either all values are positive and the
+      // or-ed result will require as many bits per value as the max of the
+      // values, or one of them is negative and the result will be negative,
+      // forcing GrowableWriter to use 64 bits per value
       max |= arr[i];
     }
     ensureCapacity(max);
@@ -123,7 +138,12 @@ public class GrowableWriter implements P
 
   @Override
   public long ramBytesUsed() {
-    return current.ramBytesUsed();
+    return RamUsageEstimator.alignObjectSize(
+        RamUsageEstimator.NUM_BYTES_OBJECT_HEADER
+        + RamUsageEstimator.NUM_BYTES_OBJECT_REF
+        + RamUsageEstimator.NUM_BYTES_LONG
+        + RamUsageEstimator.NUM_BYTES_FLOAT)
+        + current.ramBytesUsed();
   }
 
   @Override

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/MonotonicAppendingLongBuffer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/MonotonicAppendingLongBuffer.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/MonotonicAppendingLongBuffer.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/MonotonicAppendingLongBuffer.java Sun Aug 11 12:19:13 2013
@@ -17,14 +17,15 @@ package org.apache.lucene.util.packed;
  * limitations under the License.
  */
 
-import java.util.Arrays;
-
 import org.apache.lucene.util.RamUsageEstimator;
 
+import java.util.Arrays;
+
 /**
  * Utility class to buffer signed longs in memory, which is optimized for the
  * case where the sequence is monotonic, although it can encode any sequence of
  * arbitrary longs. It only supports appending.
+ *
  * @lucene.internal
  */
 public final class MonotonicAppendingLongBuffer extends AbstractAppendingLongBuffer {
@@ -32,28 +33,77 @@ public final class MonotonicAppendingLon
   static long zigZagDecode(long n) {
     return ((n >>> 1) ^ -(n & 1));
   }
-  
+
   static long zigZagEncode(long n) {
     return (n >> 63) ^ (n << 1);
   }
 
-  private float[] averages;
+  float[] averages;
+  long[] minValues;
 
-  /** Sole constructor. */
+  /**
+   * @param initialPageCount        the initial number of pages
+   * @param pageSize                the size of a single page
+   * @param acceptableOverheadRatio an acceptable overhead ratio per value
+   */
+  public MonotonicAppendingLongBuffer(int initialPageCount, int pageSize, float acceptableOverheadRatio) {
+    super(initialPageCount, pageSize, acceptableOverheadRatio);
+    averages = new float[values.length];
+    minValues = new long[values.length];
+  }
+
+  /**
+   * Create an {@link MonotonicAppendingLongBuffer} with initialPageCount=16,
+   * pageSize=1024 and acceptableOverheadRatio={@link PackedInts#DEFAULT}
+   */
   public MonotonicAppendingLongBuffer() {
-    super(16);
-    averages = new float[16];
+    this(16, 1024, PackedInts.DEFAULT);
   }
-  
+
+  /**
+   * Create an {@link AppendingDeltaPackedLongBuffer} with initialPageCount=16,
+   * pageSize=1024
+   */
+  public MonotonicAppendingLongBuffer(float acceptableOverheadRatio) {
+    this(16, 1024, acceptableOverheadRatio);
+  }
+
+
+  @Override
   long get(int block, int element) {
     if (block == valuesOff) {
       return pending[element];
     } else {
       final long base = minValues[block] + (long) (averages[block] * (long) element);
-      if (deltas[block] == null) {
+      if (values[block] == null) {
         return base;
       } else {
-        return base + zigZagDecode(deltas[block].get(element));
+        return base + zigZagDecode(values[block].get(element));
+      }
+    }
+  }
+
+  @Override
+  int get(int block, int element, long[] arr, int off, int len) {
+    if (block == valuesOff) {
+      int sysCopyToRead = Math.min(len, pendingOff - element);
+      System.arraycopy(pending, element, arr, off, sysCopyToRead);
+      return sysCopyToRead;
+    } else {
+      if (values[block] == null) {
+        int toFill = Math.min(len, pending.length - element);
+        for (int r = 0; r < toFill; r++, off++, element++) {
+          arr[off] = minValues[block] + (long) (averages[block] * (long) element);
+        }
+        return toFill;
+      } else {
+
+    /* packed block */
+        int read = values[block].get(element, arr, off, len);
+        for (int r = 0; r < read; r++, off++, element++) {
+          arr[off] = minValues[block] + (long) (averages[block] * (long) element) + zigZagDecode(arr[off]);
+        }
+        return read;
       }
     }
   }
@@ -62,20 +112,20 @@ public final class MonotonicAppendingLon
   void grow(int newBlockCount) {
     super.grow(newBlockCount);
     this.averages = Arrays.copyOf(averages, newBlockCount);
+    this.minValues = Arrays.copyOf(minValues, newBlockCount);
   }
 
   @Override
   void packPendingValues() {
-    assert pendingOff == MAX_PENDING_COUNT;
-
+    assert pendingOff > 0;
     minValues[valuesOff] = pending[0];
-    averages[valuesOff] = (float) (pending[BLOCK_MASK] - pending[0]) / BLOCK_MASK;
+    averages[valuesOff] = pendingOff == 1 ? 0 : (float) (pending[pendingOff - 1] - pending[0]) / (pendingOff - 1);
 
-    for (int i = 0; i < MAX_PENDING_COUNT; ++i) {
+    for (int i = 0; i < pendingOff; ++i) {
       pending[i] = zigZagEncode(pending[i] - minValues[valuesOff] - (long) (averages[valuesOff] * (long) i));
     }
     long maxDelta = 0;
-    for (int i = 0; i < MAX_PENDING_COUNT; ++i) {
+    for (int i = 0; i < pendingOff; ++i) {
       if (pending[i] < 0) {
         maxDelta = -1;
         break;
@@ -83,57 +133,28 @@ public final class MonotonicAppendingLon
         maxDelta = Math.max(maxDelta, pending[i]);
       }
     }
-    if (maxDelta != 0) {
+    if (maxDelta == 0) {
+      values[valuesOff] = new PackedInts.NullReader(pendingOff);
+    } else {
       final int bitsRequired = maxDelta < 0 ? 64 : PackedInts.bitsRequired(maxDelta);
-      final PackedInts.Mutable mutable = PackedInts.getMutable(pendingOff, bitsRequired, PackedInts.COMPACT);
+      final PackedInts.Mutable mutable = PackedInts.getMutable(pendingOff, bitsRequired, acceptableOverheadRatio);
       for (int i = 0; i < pendingOff; ) {
         i += mutable.set(i, pending, i, pendingOff - i);
       }
-      deltas[valuesOff] = mutable;
-    }
-  }
-
-  /** Return an iterator over the values of this buffer. */
-  public Iterator iterator() {
-    return new Iterator();
-  }
-
-  /** A long iterator. */
-  public final class Iterator extends AbstractAppendingLongBuffer.Iterator {
-
-    Iterator() {
-      super();
-    }
-
-    void fillValues() {
-      if (vOff == valuesOff) {
-        currentValues = pending;
-      } else if (deltas[vOff] == null) {
-        for (int k = 0; k < MAX_PENDING_COUNT; ++k) {
-          currentValues[k] = minValues[vOff] + (long) (averages[vOff] * (long) k);
-        }
-      } else {
-        for (int k = 0; k < MAX_PENDING_COUNT; ) {
-          k += deltas[vOff].get(k, currentValues, k, MAX_PENDING_COUNT - k);
-        }
-        for (int k = 0; k < MAX_PENDING_COUNT; ++k) {
-          currentValues[k] = minValues[vOff] + (long) (averages[vOff] * (long) k) + zigZagDecode(currentValues[k]);
-        }
-      }
+      values[valuesOff] = mutable;
     }
-
   }
 
   @Override
   long baseRamBytesUsed() {
     return super.baseRamBytesUsed()
-        + RamUsageEstimator.NUM_BYTES_OBJECT_REF; // the additional array
+        + 2 * RamUsageEstimator.NUM_BYTES_OBJECT_REF; // 2 additional arrays
   }
 
   @Override
   public long ramBytesUsed() {
     return super.ramBytesUsed()
-        + RamUsageEstimator.sizeOf(averages);
+        + RamUsageEstimator.sizeOf(averages) + RamUsageEstimator.sizeOf(minValues);
   }
 
 }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/MonotonicBlockPackedReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/MonotonicBlockPackedReader.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/MonotonicBlockPackedReader.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/MonotonicBlockPackedReader.java Sun Aug 11 12:19:13 2013
@@ -17,8 +17,11 @@ package org.apache.lucene.util.packed;
  * limitations under the License.
  */
 
-import static org.apache.lucene.util.packed.AbstractBlockPackedWriter.checkBlockSize;
+import static org.apache.lucene.util.packed.AbstractBlockPackedWriter.MAX_BLOCK_SIZE;
+import static org.apache.lucene.util.packed.AbstractBlockPackedWriter.MIN_BLOCK_SIZE;
 import static org.apache.lucene.util.packed.BlockPackedReaderIterator.zigZagDecode;
+import static org.apache.lucene.util.packed.PackedInts.checkBlockSize;
+import static org.apache.lucene.util.packed.PackedInts.numBlocks;
 
 import java.io.IOException;
 
@@ -39,14 +42,10 @@ public final class MonotonicBlockPackedR
 
   /** Sole constructor. */
   public MonotonicBlockPackedReader(IndexInput in, int packedIntsVersion, int blockSize, long valueCount, boolean direct) throws IOException {
-    checkBlockSize(blockSize);
     this.valueCount = valueCount;
-    blockShift = Integer.numberOfTrailingZeros(blockSize);
+    blockShift = checkBlockSize(blockSize, MIN_BLOCK_SIZE, MAX_BLOCK_SIZE);
     blockMask = blockSize - 1;
-    final int numBlocks = (int) (valueCount / blockSize) + (valueCount % blockSize == 0 ? 0 : 1);
-    if ((long) numBlocks * blockSize < valueCount) {
-      throw new IllegalArgumentException("valueCount is too large for this block size");
-    }
+    final int numBlocks = numBlocks(valueCount, blockSize);
     minValues = new long[numBlocks];
     averages = new float[numBlocks];
     subReaders = new PackedInts.Reader[numBlocks];
@@ -79,5 +78,10 @@ public final class MonotonicBlockPackedR
     final int idx = (int) (index & blockMask);
     return minValues[block] + (long) (idx * averages[block]) + zigZagDecode(subReaders[block].get(idx));
   }
+  
+  /** Returns the number of values */
+  public long size() {
+    return valueCount;
+  }
 
 }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/Packed64SingleBlock.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/Packed64SingleBlock.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/Packed64SingleBlock.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/Packed64SingleBlock.java Sun Aug 11 12:19:13 2013
@@ -586,4 +586,4 @@ abstract class Packed64SingleBlock exten
 
   }
 
-}
\ No newline at end of file
+}

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java Sun Aug 11 12:19:13 2013
@@ -18,12 +18,14 @@ package org.apache.lucene.util.packed;
  */
 
 import java.io.IOException;
+import java.util.Arrays;
 
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.LongsRef;
+import org.apache.lucene.util.RamUsageEstimator;
 
 /**
  * Simplistic compression for array of unsigned long values.
@@ -213,6 +215,11 @@ public class PackedInts {
       this.format = format;
       this.bitsPerValue = bitsPerValue;
     }
+
+    @Override
+    public String toString() {
+      return "FormatAndBits(format=" + format + " bitsPerValue=" + bitsPerValue + ")";
+    }
   }
 
   /**
@@ -698,7 +705,11 @@ public class PackedInts {
 
     @Override
     public int get(int index, long[] arr, int off, int len) {
-      return 0;
+      assert len > 0 : "len must be > 0 (got " + len + ")";
+      assert index >= 0 && index < valueCount;
+      len = Math.min(len, valueCount - index);
+      Arrays.fill(arr, off, off + len, 0);
+      return len;
     }
 
     @Override
@@ -713,7 +724,7 @@ public class PackedInts {
 
     @Override
     public long ramBytesUsed() {
-      return 0;
+      return RamUsageEstimator.alignObjectSize(RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + RamUsageEstimator.NUM_BYTES_INT);
     }
 
     @Override
@@ -1036,14 +1047,21 @@ public class PackedInts {
    */
   public static Mutable getMutable(int valueCount,
       int bitsPerValue, float acceptableOverheadRatio) {
-    assert valueCount >= 0;
-
     final FormatAndBits formatAndBits = fastestFormatAndBits(valueCount, bitsPerValue, acceptableOverheadRatio);
-    switch (formatAndBits.format) {
+    return getMutable(valueCount, formatAndBits.bitsPerValue, formatAndBits.format);
+  }
+
+  /** Same as {@link #getMutable(int, int, float)} with a pre-computed number
+   *  of bits per value and format.
+   *  @lucene.internal */
+  public static Mutable getMutable(int valueCount,
+      int bitsPerValue, PackedInts.Format format) {
+    assert valueCount >= 0;
+    switch (format) {
       case PACKED_SINGLE_BLOCK:
-        return Packed64SingleBlock.create(valueCount, formatAndBits.bitsPerValue);
+        return Packed64SingleBlock.create(valueCount, bitsPerValue);
       case PACKED:
-        switch (formatAndBits.bitsPerValue) {
+        switch (bitsPerValue) {
           case 8:
             return new Direct8(valueCount);
           case 16:
@@ -1063,7 +1081,7 @@ public class PackedInts {
             }
             break;
         }
-        return new Packed64(valueCount, formatAndBits.bitsPerValue);
+        return new Packed64(valueCount, bitsPerValue);
       default:
         throw new AssertionError();
     }
@@ -1198,33 +1216,39 @@ public class PackedInts {
       for (int i = 0; i < len; ++i) {
         dest.set(destPos++, src.get(srcPos++));
       }
-    } else {
+    } else if (len > 0) {
       // use bulk operations
-      long[] buf = new long[Math.min(capacity, len)];
-      int remaining = 0;
-      while (len > 0) {
-        final int read = src.get(srcPos, buf, remaining, Math.min(len, buf.length - remaining));
-        assert read > 0;
-        srcPos += read;
-        len -= read;
-        remaining += read;
-        final int written = dest.set(destPos, buf, 0, remaining);
-        assert written > 0;
-        destPos += written;
-        if (written < remaining) {
-          System.arraycopy(buf, written, buf, 0, remaining - written);
-        }
-        remaining -= written;
-      }
-      while (remaining > 0) {
-        final int written = dest.set(destPos, buf, 0, remaining);
-        destPos += written;
-        remaining -= written;
-        System.arraycopy(buf, written, buf, 0, remaining);
+      final long[] buf = new long[Math.min(capacity, len)];
+      copy(src, srcPos, dest, destPos, len, buf);
+    }
+  }
+
+  /** Same as {@link #copy(Reader, int, Mutable, int, int, int)} but using a pre-allocated buffer. */
+  static void copy(Reader src, int srcPos, Mutable dest, int destPos, int len, long[] buf) {
+    assert buf.length > 0;
+    int remaining = 0;
+    while (len > 0) {
+      final int read = src.get(srcPos, buf, remaining, Math.min(len, buf.length - remaining));
+      assert read > 0;
+      srcPos += read;
+      len -= read;
+      remaining += read;
+      final int written = dest.set(destPos, buf, 0, remaining);
+      assert written > 0;
+      destPos += written;
+      if (written < remaining) {
+        System.arraycopy(buf, written, buf, 0, remaining - written);
       }
+      remaining -= written;
+    }
+    while (remaining > 0) {
+      final int written = dest.set(destPos, buf, 0, remaining);
+      destPos += written;
+      remaining -= written;
+      System.arraycopy(buf, written, buf, 0, remaining);
     }
   }
-  
+
   /**
    * Expert: reads only the metadata from a stream. This is useful to later
    * restore a stream or open a direct reader via 
@@ -1261,4 +1285,26 @@ public class PackedInts {
     }    
   }
 
-}
\ No newline at end of file
+  /** Check that the block size is a power of 2, in the right bounds, and return
+   *  its log in base 2. */
+  static int checkBlockSize(int blockSize, int minBlockSize, int maxBlockSize) {
+    if (blockSize < minBlockSize || blockSize > maxBlockSize) {
+      throw new IllegalArgumentException("blockSize must be >= " + minBlockSize + " and <= " + maxBlockSize + ", got " + blockSize);
+    }
+    if ((blockSize & (blockSize - 1)) != 0) {
+      throw new IllegalArgumentException("blockSize must be a power of two, got " + blockSize);
+    }
+    return Integer.numberOfTrailingZeros(blockSize);
+  }
+
+  /** Return the number of blocks required to store <code>size</code> values on
+   *  <code>blockSize</code>. */
+  static int numBlocks(long size, int blockSize) {
+    final int numBlocks = (int) (size / blockSize) + (size % blockSize == 0 ? 0 : 1);
+    if ((long) numBlocks * blockSize < size) {
+      throw new IllegalArgumentException("size is too large for this block size");
+    }
+    return numBlocks;
+  }
+
+}

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/gen_Direct.py
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/gen_Direct.py?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/gen_Direct.py (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/gen_Direct.py Sun Aug 11 12:19:13 2013
@@ -87,6 +87,7 @@ if __name__ == '__main__':
     return values[index]%s;
   }
 
+  @Override
   public void set(final int index, final long value) {
     values[index] = %s(value);
   }
@@ -100,6 +101,7 @@ if __name__ == '__main__':
         + RamUsageEstimator.sizeOf(values);
   }
 
+  @Override
   public void clear() {
     Arrays.fill(values, %s0L);
   }
@@ -128,6 +130,7 @@ if __name__ == '__main__':
     return gets;
   }
 
+  @Override
   public int set(int index, long[] arr, int off, int len) {
     assert len > 0 : "len must be > 0 (got " + len + ")";
     assert index >= 0 && index < valueCount;
@@ -158,6 +161,7 @@ if __name__ == '__main__':
     return gets;
   }
 
+  @Override
   public int set(int index, long[] arr, int off, int len) {
     assert len > 0 : "len must be > 0 (got " + len + ")";
     assert index >= 0 && index < valueCount;

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/gen_Packed64SingleBlock.py
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/gen_Packed64SingleBlock.py?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/gen_Packed64SingleBlock.py (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/gen_Packed64SingleBlock.py Sun Aug 11 12:19:13 2013
@@ -77,8 +77,13 @@ abstract class Packed64SingleBlock exten
     Arrays.fill(blocks, 0L);
   }
 
+  @Override
   public long ramBytesUsed() {
-    return RamUsageEstimator.sizeOf(blocks);
+    return RamUsageEstimator.alignObjectSize(
+        RamUsageEstimator.NUM_BYTES_OBJECT_HEADER
+        + 2 * RamUsageEstimator.NUM_BYTES_INT     // valueCount,bitsPerValue
+        + RamUsageEstimator.NUM_BYTES_OBJECT_REF) // blocks ref
+        + RamUsageEstimator.sizeOf(blocks);
   }
 
   @Override
@@ -106,8 +111,8 @@ abstract class Packed64SingleBlock exten
     // bulk get
     assert index %% valuesPerBlock == 0;
     final PackedInts.Decoder decoder = BulkOperation.of(PackedInts.Format.PACKED_SINGLE_BLOCK, bitsPerValue);
-    assert decoder.blockCount() == 1;
-    assert decoder.valueCount() == valuesPerBlock;
+    assert decoder.longBlockCount() == 1;
+    assert decoder.longValueCount() == valuesPerBlock;
     final int blockIndex = index / valuesPerBlock;
     final int nblocks = (index + len) / valuesPerBlock - blockIndex;
     decoder.decode(blocks, blockIndex, arr, off, nblocks);
@@ -150,8 +155,8 @@ abstract class Packed64SingleBlock exten
     // bulk set
     assert index %% valuesPerBlock == 0;
     final BulkOperation op = BulkOperation.of(PackedInts.Format.PACKED_SINGLE_BLOCK, bitsPerValue);
-    assert op.blockCount() == 1;
-    assert op.valueCount() == valuesPerBlock;
+    assert op.longBlockCount() == 1;
+    assert op.longValueCount() == valuesPerBlock;
     final int blockIndex = index / valuesPerBlock;
     final int nblocks = (index + len) / valuesPerBlock - blockIndex;
     op.encode(arr, off, blocks, blockIndex, nblocks);

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/package.html?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/package.html (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/package.html Sun Aug 11 12:19:13 2013
@@ -47,14 +47,22 @@
         <li>Same as PackedInts.Mutable but grows the number of bits per values when needed.</li>
         <li>Useful to build a PackedInts.Mutable from a read-once stream of longs.</li>
     </ul></li>
-    <li><b>{@link org.apache.lucene.util.packed.AppendingLongBuffer}</b><ul>
+    <li><b>{@link org.apache.lucene.util.packed.PagedGrowableWriter}</b><ul>
+        <li>Slices data into fixed-size blocks stored in GrowableWriters.</li>
+        <li>Supports more than 2B values.</li>
+        <li>You should use Appending(Delta)PackedLongBuffer instead if you don't need random write access.</li>
+    </ul></li>
+    <li><b>{@link org.apache.lucene.util.packed.AppendingDeltaPackedLongBuffer}</b><ul>
         <li>Can store any sequence of longs.</li>
         <li>Compression is good when values are close to each other.</li>
         <li>Supports random reads, but only sequential writes.</li>
         <li>Can address up to 2^42 values.</li>
     </ul></li>
+    <li><b>{@link org.apache.lucene.util.packed.AppendingPackedLongBuffer}</b><ul>
+        <li>Same as AppendingDeltaPackedLongBuffer but assumes values are 0-based.</li>
+    </ul></li>
     <li><b>{@link org.apache.lucene.util.packed.MonotonicAppendingLongBuffer}</b><ul>
-        <li>Same as AppendingLongBuffer except that compression is good when the stream is a succession of affine functions.</li>
+        <li>Same as AppendingDeltaPackedLongBuffer except that compression is good when the stream is a succession of affine functions.</li>
     </ul></li>
 </ul>
 

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/TestSearch.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/TestSearch.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/TestSearch.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/TestSearch.java Sun Aug 11 12:19:13 2013
@@ -112,10 +112,7 @@ public class TestSearch extends LuceneTe
       Analyzer analyzer = new MockAnalyzer(random);
       IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
       MergePolicy mp = conf.getMergePolicy();
-      if (mp instanceof LogMergePolicy) {
-        ((LogMergePolicy) mp).setUseCompoundFile(useCompoundFile);
-      }
-      
+      mp.setNoCFSRatio(useCompoundFile ? 1.0 : 0.0);
       IndexWriter writer = new IndexWriter(directory, conf);
 
       String[] docs = {
@@ -130,7 +127,7 @@ public class TestSearch extends LuceneTe
       for (int j = 0; j < docs.length; j++) {
         Document d = new Document();
         d.add(newTextField("contents", docs[j], Field.Store.YES));
-        d.add(newStringField("id", ""+j, Field.Store.NO));
+        d.add(new IntField("id", j, Field.Store.NO));
         writer.addDocument(d);
       }
       writer.close();

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/TestSearchForDuplicates.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/TestSearchForDuplicates.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/TestSearchForDuplicates.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/TestSearchForDuplicates.java Sun Aug 11 12:19:13 2013
@@ -72,9 +72,7 @@ public class TestSearchForDuplicates ext
       Analyzer analyzer = new MockAnalyzer(random);
       IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
       final MergePolicy mp = conf.getMergePolicy();
-      if (mp instanceof LogMergePolicy) {
-        ((LogMergePolicy) mp).setUseCompoundFile(useCompoundFiles);
-      }
+      mp.setNoCFSRatio(useCompoundFiles ? 1.0 : 0.0);
       IndexWriter writer = new IndexWriter(directory, conf);
       if (VERBOSE) {
         System.out.println("TEST: now build index MAX_DOCS=" + MAX_DOCS);
@@ -83,7 +81,7 @@ public class TestSearchForDuplicates ext
       for (int j = 0; j < MAX_DOCS; j++) {
         Document d = new Document();
         d.add(newTextField(PRIORITY_FIELD, HIGH_PRIORITY, Field.Store.YES));
-        d.add(newTextField(ID_FIELD, Integer.toString(j), Field.Store.YES));
+        d.add(new IntField(ID_FIELD, j, Field.Store.YES));
         writer.addDocument(d);
       }
       writer.close();

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/analysis/TestMockAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/analysis/TestMockAnalyzer.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/analysis/TestMockAnalyzer.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/analysis/TestMockAnalyzer.java Sun Aug 11 12:19:13 2013
@@ -1,7 +1,9 @@
 package org.apache.lucene.analysis;
 
+import java.io.Reader;
 import java.io.StringReader;
 import java.util.Arrays;
+import java.util.Random;
 
 import org.apache.lucene.util._TestUtil;
 import org.apache.lucene.util.automaton.Automaton;
@@ -96,7 +98,7 @@ public class TestMockAnalyzer extends Ba
     String testString = "t";
     
     Analyzer analyzer = new MockAnalyzer(random());
-    TokenStream stream = analyzer.tokenStream("dummy", new StringReader(testString));
+    TokenStream stream = analyzer.tokenStream("dummy", testString);
     stream.reset();
     while (stream.incrementToken()) {
       // consume
@@ -128,4 +130,29 @@ public class TestMockAnalyzer extends Ba
       ts.close();
     }
   }
+  
+  public void testWrapReader() throws Exception {
+    // LUCENE-5153: test that wrapping an analyzer's reader is allowed
+    final Random random = random();
+    
+    Analyzer a = new AnalyzerWrapper() {
+      
+      @Override
+      protected Reader wrapReader(String fieldName, Reader reader) {
+        return new MockCharFilter(reader, 7);
+      }
+      
+      @Override
+      protected TokenStreamComponents wrapComponents(String fieldName, TokenStreamComponents components) {
+        return components;
+      }
+      
+      @Override
+      protected Analyzer getWrappedAnalyzer(String fieldName) {
+        return new MockAnalyzer(random);
+      }
+    };
+    
+    checkOneTerm(a, "abc", "aabc");
+  }
 }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java Sun Aug 11 12:19:13 2013
@@ -49,6 +49,7 @@ public class TestCompressingStoredFields
     iwConf.setCodec(CompressingCodec.randomInstance(random()));
     // disable CFS because this test checks file names
     iwConf.setMergePolicy(newLogMergePolicy(false));
+    iwConf.setUseCompoundFile(false);
     RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwConf);
 
     final Document validDoc = new Document();

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestCompressingTermVectorsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestCompressingTermVectorsFormat.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestCompressingTermVectorsFormat.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestCompressingTermVectorsFormat.java Sun Aug 11 12:19:13 2013
@@ -1,7 +1,18 @@
 package org.apache.lucene.codecs.compressing;
 
 import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.AtomicReader;
 import org.apache.lucene.index.BaseTermVectorsFormatTestCase;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.index.TermsEnum.SeekStatus;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -27,4 +38,35 @@ public class TestCompressingTermVectorsF
     return CompressingCodec.randomInstance(random());
   }
   
+  // https://issues.apache.org/jira/browse/LUCENE-5156
+  public void testNoOrds() throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    Document doc = new Document();
+    FieldType ft = new FieldType(TextField.TYPE_NOT_STORED);
+    ft.setStoreTermVectors(true);
+    doc.add(new Field("foo", "this is a test", ft));
+    iw.addDocument(doc);
+    AtomicReader ir = getOnlySegmentReader(iw.getReader());
+    Terms terms = ir.getTermVector(0, "foo");
+    assertNotNull(terms);
+    TermsEnum termsEnum = terms.iterator(null);
+    assertEquals(SeekStatus.FOUND, termsEnum.seekCeil(new BytesRef("this")));
+    try {
+      termsEnum.ord();
+      fail();
+    } catch (UnsupportedOperationException expected) {
+      // expected exception
+    }
+    
+    try {
+      termsEnum.seekExact(0);
+      fail();
+    } catch (UnsupportedOperationException expected) {
+      // expected exception
+    }
+    ir.close();
+    iw.close();
+    dir.close();
+  }
 }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestLucene40PostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestLucene40PostingsFormat.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestLucene40PostingsFormat.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestLucene40PostingsFormat.java Sun Aug 11 12:19:13 2013
@@ -24,7 +24,7 @@ import org.apache.lucene.index.BasePosti
  * Tests Lucene40PostingsFormat
  */
 public class TestLucene40PostingsFormat extends BasePostingsFormatTestCase {
-  private final Codec codec = new Lucene40Codec();
+  private final Codec codec = new Lucene40RWCodec();
 
   @Override
   protected Codec getCodec() {

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestReuseDocsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestReuseDocsEnum.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestReuseDocsEnum.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestReuseDocsEnum.java Sun Aug 11 12:19:13 2013
@@ -162,7 +162,7 @@ public class TestReuseDocsEnum extends L
       return null;
     }
     TermsEnum iterator = terms.iterator(null);
-    if (iterator.seekExact(term, true)) {
+    if (iterator.seekExact(term)) {
       return iterator.docs(bits, null, random().nextBoolean() ? DocsEnum.FLAG_FREQS : DocsEnum.FLAG_NONE);
     }
     return null;

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat2.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat2.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat2.java Sun Aug 11 12:19:13 2013
@@ -46,7 +46,7 @@ public class TestBlockPostingsFormat2 ex
     dir = newFSDirectory(_TestUtil.getTempDir("testDFBlockSize"));
     iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
     iwc.setCodec(_TestUtil.alwaysPostingsFormat(new Lucene41PostingsFormat()));
-    iw = new RandomIndexWriter(random(), dir, iwc);
+    iw = new RandomIndexWriter(random(), dir, iwc.clone());
     iw.setDoRandomForceMerge(false); // we will ourselves
   }
   
@@ -55,7 +55,7 @@ public class TestBlockPostingsFormat2 ex
     iw.close();
     _TestUtil.checkIndex(dir); // for some extra coverage, checkIndex before we forceMerge
     iwc.setOpenMode(OpenMode.APPEND);
-    IndexWriter iw = new IndexWriter(dir, iwc);
+    IndexWriter iw = new IndexWriter(dir, iwc.clone());
     iw.forceMerge(1);
     iw.close();
     dir.close(); // just force a checkindex for now

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat3.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat3.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat3.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat3.java Sun Aug 11 12:19:13 2013
@@ -86,7 +86,7 @@ public class TestBlockPostingsFormat3 ex
     iwc.setCodec(_TestUtil.alwaysPostingsFormat(new Lucene41PostingsFormat())); 
     // TODO we could actually add more fields implemented with different PFs
     // or, just put this test into the usual rotation?
-    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc.clone());
     Document doc = new Document();
     FieldType docsOnlyType = new FieldType(TextField.TYPE_NOT_STORED);
     // turn this on for a cross-check
@@ -138,7 +138,7 @@ public class TestBlockPostingsFormat3 ex
     verify(dir);
     _TestUtil.checkIndex(dir); // for some extra coverage, checkIndex before we forceMerge
     iwc.setOpenMode(OpenMode.APPEND);
-    IndexWriter iw2 = new IndexWriter(dir, iwc);
+    IndexWriter iw2 = new IndexWriter(dir, iwc.clone());
     iw2.forceMerge(1);
     iw2.close();
     verify(dir);
@@ -235,21 +235,21 @@ public class TestBlockPostingsFormat3 ex
       leftEnum = leftTerms.iterator(leftEnum);
       rightEnum = rightTerms.iterator(rightEnum);
       
-      assertEquals(leftEnum.seekExact(b, false), rightEnum.seekExact(b, false));
-      assertEquals(leftEnum.seekExact(b, true), rightEnum.seekExact(b, true));
+      assertEquals(leftEnum.seekExact(b), rightEnum.seekExact(b));
+      assertEquals(leftEnum.seekExact(b), rightEnum.seekExact(b));
       
       SeekStatus leftStatus;
       SeekStatus rightStatus;
       
-      leftStatus = leftEnum.seekCeil(b, false);
-      rightStatus = rightEnum.seekCeil(b, false);
+      leftStatus = leftEnum.seekCeil(b);
+      rightStatus = rightEnum.seekCeil(b);
       assertEquals(leftStatus, rightStatus);
       if (leftStatus != SeekStatus.END) {
         assertEquals(leftEnum.term(), rightEnum.term());
       }
       
-      leftStatus = leftEnum.seekCeil(b, true);
-      rightStatus = rightEnum.seekCeil(b, true);
+      leftStatus = leftEnum.seekCeil(b);
+      rightStatus = rightEnum.seekCeil(b);
       assertEquals(leftStatus, rightStatus);
       if (leftStatus != SeekStatus.END) {
         assertEquals(leftEnum.term(), rightEnum.term());

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat2.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat2.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat2.java Sun Aug 11 12:19:13 2013
@@ -58,7 +58,7 @@ public class TestPerFieldPostingsFormat2
   private IndexWriter newWriter(Directory dir, IndexWriterConfig conf)
       throws IOException {
     LogDocMergePolicy logByteSizeMergePolicy = new LogDocMergePolicy();
-    logByteSizeMergePolicy.setUseCompoundFile(false); // make sure we use plain
+    logByteSizeMergePolicy.setNoCFSRatio(0.0); // make sure we use plain
     // files
     conf.setMergePolicy(logByteSizeMergePolicy);
 
@@ -146,7 +146,7 @@ public class TestPerFieldPostingsFormat2
 
     iwconf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()))
         .setOpenMode(OpenMode.APPEND).setCodec(codec);
-    //((LogMergePolicy) iwconf.getMergePolicy()).setUseCompoundFile(false);
+    //((LogMergePolicy) iwconf.getMergePolicy()).setNoCFSRatio(0.0);
     //((LogMergePolicy) iwconf.getMergePolicy()).setMergeFactor(10);
     iwconf.setMaxBufferedDocs(IndexWriterConfig.DISABLE_AUTO_FLUSH);
 
@@ -192,7 +192,7 @@ public class TestPerFieldPostingsFormat2
     if (VERBOSE) {
       System.out.println("\nTEST: assertQuery " + t);
     }
-    IndexReader reader = DirectoryReader.open(dir, 1);
+    IndexReader reader = DirectoryReader.open(dir);
     IndexSearcher searcher = newSearcher(reader);
     TopDocs search = searcher.search(new TermQuery(t), num + 10);
     assertEquals(num, search.totalHits);

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/document/TestField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/document/TestField.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/document/TestField.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/document/TestField.java Sun Aug 11 12:19:13 2013
@@ -18,11 +18,8 @@ package org.apache.lucene.document;
  */
 
 import java.io.StringReader;
-import java.nio.CharBuffer;
-
 import org.apache.lucene.analysis.CannedTokenStream;
 import org.apache.lucene.analysis.Token;
-import org.apache.lucene.document.Field.ReusableStringReader;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 
@@ -518,39 +515,4 @@ public class TestField extends LuceneTes
     }
   }
   
-  public void testReusableStringReader() throws Exception {
-    ReusableStringReader reader = new ReusableStringReader();
-    assertEquals(-1, reader.read());
-    assertEquals(-1, reader.read(new char[1]));
-    assertEquals(-1, reader.read(new char[2], 1, 1));
-    assertEquals(-1, reader.read(CharBuffer.wrap(new char[2])));
-    
-    reader.setValue("foobar");
-    char[] buf = new char[4];
-    assertEquals(4, reader.read(buf));
-    assertEquals("foob", new String(buf));
-    assertEquals(2, reader.read(buf));
-    assertEquals("ar", new String(buf, 0, 2));
-    assertEquals(-1, reader.read(buf));
-    reader.close();
-
-    reader.setValue("foobar");
-    assertEquals(0, reader.read(buf, 1, 0));
-    assertEquals(3, reader.read(buf, 1, 3));
-    assertEquals("foo", new String(buf, 1, 3));
-    assertEquals(2, reader.read(CharBuffer.wrap(buf, 2, 2)));
-    assertEquals("ba", new String(buf, 2, 2));
-    assertEquals('r', (char) reader.read());
-    assertEquals(-1, reader.read(buf));
-    reader.close();
-
-    reader.setValue("foobar");
-    StringBuilder sb = new StringBuilder();
-    int ch;
-    while ((ch = reader.read()) != -1) {
-      sb.append((char) ch);
-    }
-    reader.close();
-    assertEquals("foobar", sb.toString());    
-  }
 }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.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.Arrays;
 import java.util.List;
@@ -42,6 +43,7 @@ import org.apache.lucene.store.BaseDirec
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.store.RAMDirectory;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util._TestUtil;
 
@@ -596,7 +598,7 @@ public class TestAddIndexes extends Luce
 
     Directory dir = newDirectory();
     LogByteSizeMergePolicy lmp = new LogByteSizeMergePolicy();
-    lmp.setUseCompoundFile(false);
+    lmp.setNoCFSRatio(0.0);
     lmp.setMergeFactor(100);
     IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(
         TEST_VERSION_CURRENT, new MockAnalyzer(random()))
@@ -625,7 +627,7 @@ public class TestAddIndexes extends Luce
     Directory dir2 = newDirectory();
     lmp = new LogByteSizeMergePolicy();
     lmp.setMinMergeMB(0.0001);
-    lmp.setUseCompoundFile(false);
+    lmp.setNoCFSRatio(0.0);
     lmp.setMergeFactor(4);
     writer = new IndexWriter(dir2, newIndexWriterConfig(TEST_VERSION_CURRENT,
         new MockAnalyzer(random()))
@@ -888,7 +890,7 @@ public class TestAddIndexes extends Luce
 
       if (t instanceof AlreadyClosedException || t instanceof MergePolicy.MergeAbortedException || t instanceof NullPointerException) {
         report = !didClose;
-      } else if (t instanceof FileNotFoundException)  {
+      } else if (t instanceof FileNotFoundException || t instanceof NoSuchFileException)  {
         report = !didClose;
       } else if (t instanceof IOException)  {
         Throwable t2 = t.getCause();
@@ -1094,7 +1096,7 @@ public class TestAddIndexes extends Luce
     
     Directory dir = new MockDirectoryWrapper(random(), new RAMDirectory());
     IndexWriterConfig conf = new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())).setMergePolicy(newLogMergePolicy(true));
-    LogMergePolicy lmp = (LogMergePolicy) conf.getMergePolicy();
+    MergePolicy lmp = conf.getMergePolicy();
     // Force creation of CFS:
     lmp.setNoCFSRatio(1.0);
     lmp.setMaxCFSSegmentSizeMB(Double.POSITIVE_INFINITY);
@@ -1204,4 +1206,53 @@ public class TestAddIndexes extends Luce
     r3.close();
     d3.close();
   }
+  
+  public void testAddEmpty() throws Exception {
+    Directory d1 = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), d1);
+    MultiReader empty = new MultiReader();
+    w.addIndexes(empty);
+    w.close();
+    DirectoryReader dr = DirectoryReader.open(d1);
+    for (AtomicReaderContext ctx : dr.leaves()) {
+      assertTrue("empty segments should be dropped by addIndexes", ctx.reader().maxDoc() > 0);
+    }
+    dr.close();
+    d1.close();
+  }
+
+  // Currently it's impossible to end up with a segment with all documents
+  // deleted, as such segments are dropped. Still, to validate that addIndexes
+  // works with such segments, or readers that end up in such state, we fake an
+  // all deleted segment.
+  public void testFakeAllDeleted() throws Exception {
+    Directory src = newDirectory(), dest = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), src);
+    w.addDocument(new Document());
+    IndexReader allDeletedReader = new FilterAtomicReader(w.getReader().leaves().get(0).reader()) {
+      @Override
+      public Bits getLiveDocs() {
+        return new Bits() {
+          @Override public int length() { return 1; }
+          @Override public boolean get(int index) { return false; }
+        };
+      }
+      @Override public boolean hasDeletions() { return true; }
+      @Override public int numDocs() { return 0; }
+    };
+    w.close();
+    
+    w = new RandomIndexWriter(random(), dest);
+    w.addIndexes(allDeletedReader);
+    w.close();
+    DirectoryReader dr = DirectoryReader.open(src);
+    for (AtomicReaderContext ctx : dr.leaves()) {
+      assertTrue("empty segments should be dropped by addIndexes", ctx.reader().maxDoc() > 0);
+    }
+    dr.close();
+    allDeletedReader.close();
+    src.close();
+    dest.close();
+  }
+
 }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestAtomicUpdate.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestAtomicUpdate.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestAtomicUpdate.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestAtomicUpdate.java Sun Aug 11 12:19:13 2013
@@ -25,18 +25,7 @@ import org.apache.lucene.store.*;
 import org.apache.lucene.util.*;
 
 public class TestAtomicUpdate extends LuceneTestCase {
-  private static final class MockIndexWriter extends IndexWriter {
-    public MockIndexWriter(Directory dir, IndexWriterConfig conf) throws IOException {
-      super(dir, conf);
-    }
-
-    @Override
-    boolean testPoint(String name) {
-      if (LuceneTestCase.random().nextInt(4) == 2)
-        Thread.yield();
-      return true;
-    }
-  }
+  
 
   private static abstract class TimedThread extends Thread {
     volatile boolean failed;
@@ -124,7 +113,7 @@ public class TestAtomicUpdate extends Lu
         TEST_VERSION_CURRENT, new MockAnalyzer(random()))
         .setMaxBufferedDocs(7);
     ((TieredMergePolicy) conf.getMergePolicy()).setMaxMergeAtOnce(3);
-    IndexWriter writer = new MockIndexWriter(directory, conf);
+    IndexWriter writer = RandomIndexWriter.mockIndexWriter(directory, conf, random());
 
     // Establish a base index of 100 docs:
     for(int i=0;i<100;i++) {

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java Sun Aug 11 12:19:13 2013
@@ -573,8 +573,7 @@ public class TestBackwardsCompatibility 
     _TestUtil.rmDir(indexDir);
     Directory dir = newFSDirectory(indexDir);
     LogByteSizeMergePolicy mp = new LogByteSizeMergePolicy();
-    mp.setUseCompoundFile(doCFS);
-    mp.setNoCFSRatio(1.0);
+    mp.setNoCFSRatio(doCFS ? 1.0 : 0.0);
     mp.setMaxCFSSegmentSizeMB(Double.POSITIVE_INFINITY);
     // TODO: remove randomness
     IndexWriterConfig conf = new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()))
@@ -593,8 +592,7 @@ public class TestBackwardsCompatibility 
     if (!fullyMerged) {
       // open fresh writer so we get no prx file in the added segment
       mp = new LogByteSizeMergePolicy();
-      mp.setUseCompoundFile(doCFS);
-      mp.setNoCFSRatio(1.0);
+      mp.setNoCFSRatio(doCFS ? 1.0 : 0.0);
       // TODO: remove randomness
       conf = new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()))
         .setMaxBufferedDocs(10).setMergePolicy(mp);
@@ -626,7 +624,7 @@ public class TestBackwardsCompatibility 
     try {
       Directory dir = newFSDirectory(outputDir);
 
-      LogMergePolicy mergePolicy = newLogMergePolicy(true, 10);
+      MergePolicy mergePolicy = newLogMergePolicy(true, 10);
       
       // This test expects all of its segments to be in CFS:
       mergePolicy.setNoCFSRatio(1.0); 
@@ -637,7 +635,7 @@ public class TestBackwardsCompatibility 
           newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())).
               setMaxBufferedDocs(-1).
               setRAMBufferSizeMB(16.0).
-              setMergePolicy(mergePolicy)
+              setMergePolicy(mergePolicy).setUseCompoundFile(true)
       );
       for(int i=0;i<35;i++) {
         addDoc(writer, i);
@@ -649,7 +647,7 @@ public class TestBackwardsCompatibility 
       writer = new IndexWriter(
           dir,
           newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()))
-            .setMergePolicy(NoMergePolicy.NO_COMPOUND_FILES)
+            .setMergePolicy(NoMergePolicy.NO_COMPOUND_FILES).setUseCompoundFile(true)
       );
       Term searchTerm = new Term("id", "7");
       writer.deleteDocuments(searchTerm);

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java Sun Aug 11 12:19:13 2013
@@ -296,7 +296,7 @@ public class TestCodecs extends LuceneTe
     Codec codec = Codec.getDefault();
     final SegmentInfo si = new SegmentInfo(dir, Constants.LUCENE_MAIN_VERSION, SEGMENT, 10000, false, codec, null, null);
 
-    final FieldsProducer reader = codec.postingsFormat().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, newIOContext(random()), DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR));
+    final FieldsProducer reader = codec.postingsFormat().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, newIOContext(random())));
 
     final Iterator<String> fieldsEnum = reader.iterator();
     String fieldName = fieldsEnum.next();
@@ -357,7 +357,7 @@ public class TestCodecs extends LuceneTe
     if (VERBOSE) {
       System.out.println("TEST: now read postings");
     }
-    final FieldsProducer terms = codec.postingsFormat().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, newIOContext(random()), DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR));
+    final FieldsProducer terms = codec.postingsFormat().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, newIOContext(random())));
 
     final Verify[] threads = new Verify[NUM_TEST_THREADS-1];
     for(int i=0;i<NUM_TEST_THREADS-1;i++) {
@@ -655,10 +655,9 @@ public class TestCodecs extends LuceneTe
 
   private void write(final FieldInfos fieldInfos, final Directory dir, final FieldData[] fields) throws Throwable {
 
-    final int termIndexInterval = _TestUtil.nextInt(random(), 13, 27);
     final Codec codec = Codec.getDefault();
     final SegmentInfo si = new SegmentInfo(dir, Constants.LUCENE_MAIN_VERSION, SEGMENT, 10000, false, codec, null, null);
-    final SegmentWriteState state = new SegmentWriteState(InfoStream.getDefault(), dir, si, fieldInfos, termIndexInterval, null, newIOContext(random()));
+    final SegmentWriteState state = new SegmentWriteState(InfoStream.getDefault(), dir, si, fieldInfos, null, newIOContext(random()));
 
     final FieldsConsumer consumer = codec.postingsFormat().fieldsConsumer(state);
     Arrays.sort(fields);

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java Sun Aug 11 12:19:13 2013
@@ -58,6 +58,9 @@ public class TestConcurrentMergeSchedule
         boolean isClose = false;
         StackTraceElement[] trace = new Exception().getStackTrace();
         for (int i = 0; i < trace.length; i++) {
+          if (isDoFlush && isClose) {
+            break;
+          }
           if ("flush".equals(trace[i].getMethodName())) {
             isDoFlush = true;
           }
@@ -302,11 +305,7 @@ public class TestConcurrentMergeSchedule
         }
       }
       };
-    if (maxMergeThreads > cms.getMaxMergeCount()) {
-      cms.setMaxMergeCount(maxMergeCount);
-    }
-    cms.setMaxThreadCount(maxMergeThreads);
-    cms.setMaxMergeCount(maxMergeCount);
+    cms.setMaxMergesAndThreads(maxMergeCount, maxMergeThreads);
     iwc.setMergeScheduler(cms);
     iwc.setMaxBufferedDocs(2);
 
@@ -332,8 +331,7 @@ public class TestConcurrentMergeSchedule
     long totMergedBytes;
 
     public TrackingCMS() {
-      setMaxMergeCount(5);
-      setMaxThreadCount(5);
+      setMaxMergesAndThreads(5, 5);
     }
 
     @Override

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestCustomNorms.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestCustomNorms.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestCustomNorms.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestCustomNorms.java Sun Aug 11 12:19:13 2013
@@ -112,12 +112,7 @@ public class TestCustomNorms extends Luc
     }
 
     @Override
-    public ExactSimScorer exactSimScorer(SimWeight weight, AtomicReaderContext context) throws IOException {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public SloppySimScorer sloppySimScorer(SimWeight weight, AtomicReaderContext context) throws IOException {
+    public SimScorer simScorer(SimWeight weight, AtomicReaderContext context) throws IOException {
       throw new UnsupportedOperationException();
     }
   }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestDeletionPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestDeletionPolicy.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestDeletionPolicy.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestDeletionPolicy.java Sun Aug 11 12:19:13 2013
@@ -225,9 +225,7 @@ public class TestDeletionPolicy extends 
         new MockAnalyzer(random()))
         .setIndexDeletionPolicy(new ExpirationTimeDeletionPolicy(dir, SECONDS));
     MergePolicy mp = conf.getMergePolicy();
-    if (mp instanceof LogMergePolicy) {
-      ((LogMergePolicy) mp).setUseCompoundFile(true);
-    }
+    mp.setNoCFSRatio(1.0);
     IndexWriter writer = new IndexWriter(dir, conf);
     ExpirationTimeDeletionPolicy policy = (ExpirationTimeDeletionPolicy) writer.getConfig().getIndexDeletionPolicy();
     Map<String,String> commitData = new HashMap<String,String>();
@@ -246,9 +244,7 @@ public class TestDeletionPolicy extends 
           new MockAnalyzer(random())).setOpenMode(
           OpenMode.APPEND).setIndexDeletionPolicy(policy);
       mp = conf.getMergePolicy();
-      if (mp instanceof LogMergePolicy) {
-        ((LogMergePolicy) mp).setUseCompoundFile(true);
-      }
+      mp.setNoCFSRatio(1.0);
       writer = new IndexWriter(dir, conf);
       policy = (ExpirationTimeDeletionPolicy) writer.getConfig().getIndexDeletionPolicy();
       for(int j=0;j<17;j++) {
@@ -326,9 +322,7 @@ public class TestDeletionPolicy extends 
           .setMaxBufferedDocs(10)
           .setMergeScheduler(new SerialMergeScheduler());
       MergePolicy mp = conf.getMergePolicy();
-      if (mp instanceof LogMergePolicy) {
-        ((LogMergePolicy) mp).setUseCompoundFile(useCompoundFile);
-      }
+      mp.setNoCFSRatio(useCompoundFile ? 1.0 : 0.0);
       IndexWriter writer = new IndexWriter(dir, conf);
       KeepAllDeletionPolicy policy = (KeepAllDeletionPolicy) writer.getConfig().getIndexDeletionPolicy();
       for(int i=0;i<107;i++) {
@@ -347,9 +341,7 @@ public class TestDeletionPolicy extends 
                                     new MockAnalyzer(random())).setOpenMode(
                                                                     OpenMode.APPEND).setIndexDeletionPolicy(policy);
         mp = conf.getMergePolicy();
-        if (mp instanceof LogMergePolicy) {
-          ((LogMergePolicy) mp).setUseCompoundFile(useCompoundFile);
-        }
+        mp.setNoCFSRatio(useCompoundFile ? 1.0 : 0.0);
         if (VERBOSE) {
           System.out.println("TEST: open writer for forceMerge");
         }
@@ -526,9 +518,7 @@ public class TestDeletionPolicy extends 
           .setIndexDeletionPolicy(new KeepNoneOnInitDeletionPolicy())
           .setMaxBufferedDocs(10);
       MergePolicy mp = conf.getMergePolicy();
-      if (mp instanceof LogMergePolicy) {
-        ((LogMergePolicy) mp).setUseCompoundFile(useCompoundFile);
-      }
+      mp.setNoCFSRatio(useCompoundFile ? 1.0 : 0.0);
       IndexWriter writer = new IndexWriter(dir, conf);
       KeepNoneOnInitDeletionPolicy policy = (KeepNoneOnInitDeletionPolicy) writer.getConfig().getIndexDeletionPolicy();
       for(int i=0;i<107;i++) {
@@ -539,9 +529,7 @@ public class TestDeletionPolicy extends 
       conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()))
           .setOpenMode(OpenMode.APPEND).setIndexDeletionPolicy(policy);
       mp = conf.getMergePolicy();
-      if (mp instanceof LogMergePolicy) {
-        ((LogMergePolicy) mp).setUseCompoundFile(true);
-      }
+      mp.setNoCFSRatio(1.0);
       writer = new IndexWriter(dir, conf);
       policy = (KeepNoneOnInitDeletionPolicy) writer.getConfig().getIndexDeletionPolicy();
       writer.forceMerge(1);
@@ -581,9 +569,7 @@ public class TestDeletionPolicy extends 
             .setIndexDeletionPolicy(policy)
             .setMaxBufferedDocs(10);
         MergePolicy mp = conf.getMergePolicy();
-        if (mp instanceof LogMergePolicy) {
-          ((LogMergePolicy) mp).setUseCompoundFile(useCompoundFile);
-        }
+        mp.setNoCFSRatio(useCompoundFile ? 1.0 : 0.0);
         IndexWriter writer = new IndexWriter(dir, conf);
         policy = (KeepLastNDeletionPolicy) writer.getConfig().getIndexDeletionPolicy();
         for(int i=0;i<17;i++) {
@@ -642,9 +628,7 @@ public class TestDeletionPolicy extends 
           .setIndexDeletionPolicy(new KeepLastNDeletionPolicy(N))
           .setMaxBufferedDocs(10);
       MergePolicy mp = conf.getMergePolicy();
-      if (mp instanceof LogMergePolicy) {
-        ((LogMergePolicy) mp).setUseCompoundFile(useCompoundFile);
-      }
+      mp.setNoCFSRatio(useCompoundFile ? 1.0 : 0.0);
       IndexWriter writer = new IndexWriter(dir, conf);
       KeepLastNDeletionPolicy policy = (KeepLastNDeletionPolicy) writer.getConfig().getIndexDeletionPolicy();
       writer.close();
@@ -658,9 +642,7 @@ public class TestDeletionPolicy extends 
             .setOpenMode(OpenMode.APPEND).setIndexDeletionPolicy(policy)
             .setMaxBufferedDocs(10);
         mp = conf.getMergePolicy();
-        if (mp instanceof LogMergePolicy) {
-          ((LogMergePolicy) mp).setUseCompoundFile(useCompoundFile);
-        }
+        mp.setNoCFSRatio(useCompoundFile ? 1.0 : 0.0);
         writer = new IndexWriter(dir, conf);
         policy = (KeepLastNDeletionPolicy) writer.getConfig().getIndexDeletionPolicy();
         for(int j=0;j<17;j++) {

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java Sun Aug 11 12:19:13 2013
@@ -20,6 +20,7 @@ package org.apache.lucene.index;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.nio.file.NoSuchFileException;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -32,6 +33,7 @@ import org.apache.lucene.codecs.lucene41
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.IntField;
 import org.apache.lucene.document.StoredField;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.document.TextField;
@@ -428,8 +430,8 @@ void assertTermDocsCount(String msg,
     }
     try {
       DirectoryReader.open(fileDirName);
-      fail("opening DirectoryReader on empty directory failed to produce FileNotFoundException");
-    } catch (FileNotFoundException e) {
+      fail("opening DirectoryReader on empty directory failed to produce FileNotFoundException/NoSuchFileException");
+    } catch (FileNotFoundException | NoSuchFileException e) {
       // GOOD
     }
     rmDir(fileDirName);
@@ -470,8 +472,8 @@ public void testFilesOpenClose() throws 
     Directory dir = newFSDirectory(dirFile);
     try {
       DirectoryReader.open(dir);
-      fail("expected FileNotFoundException");
-    } catch (FileNotFoundException e) {
+      fail("expected FileNotFoundException/NoSuchFileException");
+    } catch (FileNotFoundException | NoSuchFileException e) {
       // expected
     }
 
@@ -480,8 +482,8 @@ public void testFilesOpenClose() throws 
     // Make sure we still get a CorruptIndexException (not NPE):
     try {
       DirectoryReader.open(dir);
-      fail("expected FileNotFoundException");
-    } catch (FileNotFoundException e) {
+      fail("expected FileNotFoundException/NoSuchFileException");
+    } catch (FileNotFoundException | NoSuchFileException e) {
       // expected
     }
     
@@ -763,7 +765,7 @@ public void testFilesOpenClose() throws 
             setMergePolicy(newLogMergePolicy(10))
     );
     Document doc = new Document();
-    doc.add(newStringField("number", "17", Field.Store.NO));
+    doc.add(new IntField("number", 17, Field.Store.NO));
     writer.addDocument(doc);
     writer.commit();
   
@@ -820,54 +822,6 @@ public void testFilesOpenClose() throws 
     dir.close();
   }
   
-  // LUCENE-1609: don't load terms index
-  public void testNoTermsIndex() throws Throwable {
-    Directory dir = newDirectory();
-    IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())).setCodec(_TestUtil.alwaysPostingsFormat(new Lucene41PostingsFormat())));
-    Document doc = new Document();
-    doc.add(newTextField("field", "a b c d e f g h i j k l m n o p q r s t u v w x y z", Field.Store.NO));
-    doc.add(newTextField("number", "0 1 2 3 4 5 6 7 8 9", Field.Store.NO));
-    writer.addDocument(doc);
-    writer.addDocument(doc);
-    writer.close();
-  
-    DirectoryReader r = DirectoryReader.open(dir, -1);
-    try {
-      r.docFreq(new Term("field", "f"));
-      fail("did not hit expected exception");
-    } catch (IllegalStateException ise) {
-      // expected
-    }
-  
-    assertEquals(-1, ((SegmentReader) r.leaves().get(0).reader()).getTermInfosIndexDivisor());
-    writer = new IndexWriter(
-        dir,
-        newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())).
-            setCodec(_TestUtil.alwaysPostingsFormat(new Lucene41PostingsFormat())).
-            setMergePolicy(newLogMergePolicy(10))
-    );
-    writer.addDocument(doc);
-    writer.close();
-  
-    // LUCENE-1718: ensure re-open carries over no terms index:
-    DirectoryReader r2 = DirectoryReader.openIfChanged(r);
-    assertNotNull(r2);
-    assertNull(DirectoryReader.openIfChanged(r2));
-    r.close();
-    List<AtomicReaderContext> leaves = r2.leaves();
-    assertEquals(2, leaves.size());
-    for(AtomicReaderContext ctx : leaves) {
-      try {
-        ctx.reader().docFreq(new Term("field", "f"));
-        fail("did not hit expected exception");
-      } catch (IllegalStateException ise) {
-        // expected
-      }
-    }
-    r2.close();
-    dir.close();
-  }
-  
   // LUCENE-2046
   public void testPrepareCommitIsCurrent() throws Throwable {
     Directory dir = newDirectory();