You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ha...@apache.org on 2013/08/13 06:06:27 UTC

svn commit: r1513336 [5/11] - in /lucene/dev/branches/lucene3069/lucene: ./ analysis/ analysis/common/ analysis/common/src/java/org/apache/lucene/analysis/charfilter/ analysis/common/src/java/org/apache/lucene/analysis/hunspell/ analysis/common/src/jav...

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/store/SimpleFSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/store/SimpleFSDirectory.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/store/SimpleFSDirectory.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/store/SimpleFSDirectory.java Tue Aug 13 04:06:18 2013
@@ -56,7 +56,7 @@ public class SimpleFSDirectory extends F
     ensureOpen();
     final File path = new File(directory, name);
     RandomAccessFile raf = new RandomAccessFile(path, "r");
-    return new SimpleFSIndexInput("SimpleFSIndexInput(path=\"" + path.getPath() + "\")", raf, context, getReadChunkSize());
+    return new SimpleFSIndexInput("SimpleFSIndexInput(path=\"" + path.getPath() + "\")", raf, context);
   }
 
   @Override
@@ -75,7 +75,7 @@ public class SimpleFSDirectory extends F
       @Override
       public IndexInput openSlice(String sliceDescription, long offset, long length) {
         return new SimpleFSIndexInput("SimpleFSIndexInput(" + sliceDescription + " in path=\"" + file.getPath() + "\" slice=" + offset + ":" + (offset+length) + ")", descriptor, offset,
-            length, BufferedIndexInput.bufferSize(context), getReadChunkSize());
+            length, BufferedIndexInput.bufferSize(context));
       }
     };
   }
@@ -85,29 +85,31 @@ public class SimpleFSDirectory extends F
    * {@link RandomAccessFile#read(byte[], int, int)}.  
    */
   protected static class SimpleFSIndexInput extends BufferedIndexInput {
+    /**
+     * The maximum chunk size is 8192 bytes, because {@link RandomAccessFile} mallocs
+     * a native buffer outside of stack if the read buffer size is larger.
+     */
+    private static final int CHUNK_SIZE = 8192;
+    
     /** the file channel we will read from */
     protected final RandomAccessFile file;
     /** is this instance a clone and hence does not own the file to close it */
     boolean isClone = false;
-    /** maximum read length on a 32bit JVM to prevent incorrect OOM, see LUCENE-1566 */ 
-    protected final int chunkSize;
     /** start offset: non-zero in the slice case */
     protected final long off;
     /** end offset (start+length) */
     protected final long end;
     
-    public SimpleFSIndexInput(String resourceDesc, RandomAccessFile file, IOContext context, int chunkSize) throws IOException {
+    public SimpleFSIndexInput(String resourceDesc, RandomAccessFile file, IOContext context) throws IOException {
       super(resourceDesc, context);
       this.file = file; 
-      this.chunkSize = chunkSize;
       this.off = 0L;
       this.end = file.length();
     }
     
-    public SimpleFSIndexInput(String resourceDesc, RandomAccessFile file, long off, long length, int bufferSize, int chunkSize) {
+    public SimpleFSIndexInput(String resourceDesc, RandomAccessFile file, long off, long length, int bufferSize) {
       super(resourceDesc, bufferSize);
       this.file = file;
-      this.chunkSize = chunkSize;
       this.off = off;
       this.end = off + length;
       this.isClone = true;
@@ -146,26 +148,16 @@ public class SimpleFSDirectory extends F
         }
 
         try {
-          do {
-            final int readLength;
-            if (total + chunkSize > len) {
-              readLength = len - total;
-            } else {
-              // LUCENE-1566 - work around JVM Bug by breaking very large reads into chunks
-              readLength = chunkSize;
+          while (total < len) {
+            final int toRead = Math.min(CHUNK_SIZE, len - total);
+            final int i = file.read(b, offset + total, toRead);
+            if (i < 0) { // be defensive here, even though we checked before hand, something could have changed
+             throw new EOFException("read past EOF: " + this + " off: " + offset + " len: " + len + " total: " + total + " chunkLen: " + toRead + " end: " + end);
             }
-            final int i = file.read(b, offset + total, readLength);
+            assert i > 0 : "RandomAccessFile.read with non zero-length toRead must always read at least one byte";
             total += i;
-          } while (total < len);
-        } catch (OutOfMemoryError e) {
-          // propagate OOM up and add a hint for 32bit VM Users hitting the bug
-          // with a large chunk size in the fast path.
-          final OutOfMemoryError outOfMemoryError = new OutOfMemoryError(
-              "OutOfMemoryError likely caused by the Sun VM Bug described in "
-              + "https://issues.apache.org/jira/browse/LUCENE-1566; try calling FSDirectory.setReadChunkSize "
-              + "with a value smaller than the current chunk size (" + chunkSize + ")");
-          outOfMemoryError.initCause(e);
-          throw outOfMemoryError;
+          }
+          assert total == len;
         } catch (IOException ioe) {
           throw new IOException(ioe.getMessage() + ": " + this, ioe);
         }

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/GrowableByteArrayDataOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/GrowableByteArrayDataOutput.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/GrowableByteArrayDataOutput.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/GrowableByteArrayDataOutput.java Tue Aug 13 04:06:18 2013
@@ -47,9 +47,7 @@ public final class GrowableByteArrayData
   @Override
   public void writeBytes(byte[] b, int off, int len) {
     final int newLength = length + len;
-    if (newLength > bytes.length) {
-      bytes = ArrayUtil.grow(bytes, newLength);
-    }
+    bytes = ArrayUtil.grow(bytes, newLength);
     System.arraycopy(b, off, bytes, length, len);
     length = newLength;
   }

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/SetOnce.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/SetOnce.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/SetOnce.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/SetOnce.java Tue Aug 13 04:06:18 2013
@@ -28,7 +28,7 @@ import java.util.concurrent.atomic.Atomi
  *
  * @lucene.experimental
  */
-public final class SetOnce<T> {
+public final class SetOnce<T> implements Cloneable {
 
   /** Thrown when {@link SetOnce#set(Object)} is called more than once. */
   public static final class AlreadySetException extends IllegalStateException {
@@ -74,4 +74,10 @@ public final class SetOnce<T> {
   public final T get() {
     return obj;
   }
+  
+  @Override
+  public SetOnce<T> clone() {
+    return obj == null ? new SetOnce<T>() : new SetOnce<T>(obj);
+  }
+  
 }

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/WAH8DocIdSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/WAH8DocIdSet.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/WAH8DocIdSet.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/WAH8DocIdSet.java Tue Aug 13 04:06:18 2013
@@ -27,6 +27,7 @@ import org.apache.lucene.search.DocIdSet
 import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.util.packed.MonotonicAppendingLongBuffer;
+import org.apache.lucene.util.packed.PackedInts;
 
 /**
  * {@link DocIdSet} implementation based on word-aligned hybrid encoding on
@@ -34,31 +35,33 @@ import org.apache.lucene.util.packed.Mon
  * <p>This implementation doesn't support random-access but has a fast
  * {@link DocIdSetIterator} which can advance in logarithmic time thanks to
  * an index.</p>
- * <p>The compression scheme is simplistic and should work well with sparse doc
- * id sets while being only slightly larger than a {@link FixedBitSet} for
- * incompressible sets (overhead&lt;2% in the worst case) in spite of the index.</p>
+ * <p>The compression scheme is simplistic and should work well with sparse and
+ * very dense doc id sets while being only slightly larger than a
+ * {@link FixedBitSet} for incompressible sets (overhead&lt;2% in the worst
+ * case) in spite of the index.</p>
  * <p><b>Format</b>: The format is byte-aligned. An 8-bits word is either clean,
- * meaning composed only of zeros, or dirty, meaning that it contains at least one
- * bit set. The idea is to encode sequences of clean words using run-length
- * encoding and to leave sequences of dirty words as-is.</p>
+ * meaning composed only of zeros or ones, or dirty, meaning that it contains
+ * between 1 and 7 bits set. The idea is to encode sequences of clean words
+ * using run-length encoding and to leave sequences of dirty words as-is.</p>
  * <table>
  *   <tr><th>Token</th><th>Clean length+</th><th>Dirty length+</th><th>Dirty words</th></tr>
  *   <tr><td>1 byte</td><td>0-n bytes</td><td>0-n bytes</td><td>0-n bytes</td></tr>
  * </table>
  * <ul>
- *   <li><b>Token</b> encodes the number of clean words minus 2 on the first 4
- * bits and the number of dirty words minus 1 on the last 4 bits. The
- * higher-order bit is a continuation bit, meaning that the number is incomplete
- * and needs additional bytes to be read.</li>
+ *   <li><b>Token</b> encodes whether clean means full of zeros or ones in the
+ * first bit, the number of clean words minus 2 on the next 3 bits and the
+ * number of dirty words on the last 4 bits. The higher-order bit is a
+ * continuation bit, meaning that the number is incomplete and needs additional
+ * bytes to be read.</li>
  *   <li><b>Clean length+</b>: If clean length has its higher-order bit set,
  * you need to read a {@link DataInput#readVInt() vint}, shift it by 3 bits on
  * the left side and add it to the 3 bits which have been read in the token.</li>
  *   <li><b>Dirty length+</b> works the same way as <b>Clean length+</b> but
- * for the length of dirty words.</li>
+ * on 4 bits and for the length of dirty words.</li>
  *   <li><b>Dirty words</b> are the dirty words, there are <b>Dirty length</b>
  * of them.</li>
  * </ul>
- * <p>This format cannot encode sequences of less than 2 clean words and 1 dirty
+ * <p>This format cannot encode sequences of less than 2 clean words and 0 dirty
  * word. The reason is that if you find a single clean word, you should rather
  * encode it as a dirty word. This takes the same space as starting a new
  * sequence (since you need one byte for the token) but will be lighter to
@@ -66,10 +69,9 @@ import org.apache.lucene.util.packed.Mon
  * sequence may start directly with a dirty word, the clean length is encoded
  * directly, without subtracting 2.</p>
  * <p>There is an additional restriction on the format: the sequence of dirty
- * words must start and end with a non-null word and is not allowed to contain
- * two consecutive null words. This restriction exists to make sure no space is
- * wasted and to make sure iterators can read the next doc ID by reading at most
- * 2 dirty words.</p>
+ * words is not allowed to contain two consecutive clean words. This restriction
+ * exists to make sure no space is wasted and to make sure iterators can read
+ * the next doc ID by reading at most 2 dirty words.</p>
  * @lucene.experimental
  */
 public final class WAH8DocIdSet extends DocIdSet {
@@ -82,9 +84,9 @@ public final class WAH8DocIdSet extends 
   private static final int MIN_INDEX_INTERVAL = 8;
 
   /** Default index interval. */
-  public static final int DEFAULT_INDEX_INTERVAL = MIN_INDEX_INTERVAL;
+  public static final int DEFAULT_INDEX_INTERVAL = 24;
 
-  private static final MonotonicAppendingLongBuffer SINGLE_ZERO_BUFFER = new MonotonicAppendingLongBuffer();
+  private static final MonotonicAppendingLongBuffer SINGLE_ZERO_BUFFER = new MonotonicAppendingLongBuffer(1, 64, PackedInts.COMPACT);
   private static WAH8DocIdSet EMPTY = new WAH8DocIdSet(new byte[0], 0, 1, SINGLE_ZERO_BUFFER, SINGLE_ZERO_BUFFER);
 
   static {
@@ -229,6 +231,7 @@ public final class WAH8DocIdSet extends 
     int numSequences;
     int indexInterval;
     int cardinality;
+    boolean reverse;
 
     WordBuilder() {
       out = new GrowableByteArrayDataOutput(1024);
@@ -255,34 +258,45 @@ public final class WAH8DocIdSet extends 
       return this;
     }
 
-    void writeHeader(int cleanLength) throws IOException {
+    void writeHeader(boolean reverse, int cleanLength, int dirtyLength) throws IOException {
       final int cleanLengthMinus2 = cleanLength - 2;
-      final int dirtyLengthMinus1 = dirtyWords.length - 1;
       assert cleanLengthMinus2 >= 0;
-      assert dirtyLengthMinus1 >= 0;
-      int token = ((cleanLengthMinus2 & 0x07) << 4) | (dirtyLengthMinus1 & 0x07);
-      if (cleanLengthMinus2 > 0x07) {
+      assert dirtyLength >= 0;
+      int token = ((cleanLengthMinus2 & 0x03) << 4) | (dirtyLength & 0x07);
+      if (reverse) {
         token |= 1 << 7;
       }
-      if (dirtyLengthMinus1 > 0x07) {
+      if (cleanLengthMinus2 > 0x03) {
+        token |= 1 << 6;
+      }
+      if (dirtyLength > 0x07) {
         token |= 1 << 3;
       }
       out.writeByte((byte) token);
-      if (cleanLengthMinus2 > 0x07) {
-        out.writeVInt(cleanLengthMinus2 >>> 3);
+      if (cleanLengthMinus2 > 0x03) {
+        out.writeVInt(cleanLengthMinus2 >>> 2);
+      }
+      if (dirtyLength > 0x07) {
+        out.writeVInt(dirtyLength >>> 3);
       }
-      if (dirtyLengthMinus1 > 0x07) {
-        out.writeVInt(dirtyLengthMinus1 >>> 3);
+    }
+
+    private boolean sequenceIsConsistent() {
+      for (int i = 1; i < dirtyWords.length; ++i) {
+        assert dirtyWords.bytes[i-1] != 0 || dirtyWords.bytes[i] != 0;
+        assert dirtyWords.bytes[i-1] != (byte) 0xFF || dirtyWords.bytes[i] != (byte) 0xFF;
       }
+      return true;
     }
 
-    void writeSequence(int cleanLength) {
+    void writeSequence() {
+      assert sequenceIsConsistent();
       try {
-        writeHeader(cleanLength);
-        out.writeBytes(dirtyWords.bytes, dirtyWords.length);
+        writeHeader(reverse, clean, dirtyWords.length);
       } catch (IOException cannotHappen) {
         throw new AssertionError(cannotHappen);
       }
+      out.writeBytes(dirtyWords.bytes, 0, dirtyWords.length);
       dirtyWords.length = 0;
       ++numSequences;
     }
@@ -291,20 +305,57 @@ public final class WAH8DocIdSet extends 
       assert wordNum > lastWordNum;
       assert word != 0;
 
-      if (lastWordNum == -1) {
-        clean = 2 + wordNum; // special case for the 1st sequence
-        dirtyWords.writeByte(word);
+      if (!reverse) {
+        if (lastWordNum == -1) {
+          clean = 2 + wordNum; // special case for the 1st sequence
+          dirtyWords.writeByte(word);
+        } else {
+          switch (wordNum - lastWordNum) {
+            case 1:
+              if (word == (byte) 0xFF && dirtyWords.bytes[dirtyWords.length-1] == (byte) 0xFF) {
+                --dirtyWords.length;
+                writeSequence();
+                reverse = true;
+                clean = 2;
+              } else {
+                dirtyWords.writeByte(word);
+              }
+              break;
+            case 2:
+              dirtyWords.writeByte((byte) 0);
+              dirtyWords.writeByte(word);
+              break;
+            default:
+              writeSequence();
+              clean = wordNum - lastWordNum - 1;
+              dirtyWords.writeByte(word);
+          }
+        }
       } else {
+        assert lastWordNum >= 0;
         switch (wordNum - lastWordNum) {
           case 1:
-            dirtyWords.writeByte(word);
+            if (word == (byte) 0xFF) {
+              if (dirtyWords.length == 0) {
+                ++clean;
+              } else if (dirtyWords.bytes[dirtyWords.length - 1] == (byte) 0xFF) {
+                --dirtyWords.length;
+                writeSequence();
+                clean = 2;
+              } else {
+                dirtyWords.writeByte(word);
+              }
+            } else {
+              dirtyWords.writeByte(word);
+            }
             break;
           case 2:
             dirtyWords.writeByte((byte) 0);
             dirtyWords.writeByte(word);
             break;
           default:
-            writeSequence(clean);
+            writeSequence();
+            reverse = false;
             clean = wordNum - lastWordNum - 1;
             dirtyWords.writeByte(word);
         }
@@ -319,7 +370,7 @@ public final class WAH8DocIdSet extends 
         assert lastWordNum == -1;
         return EMPTY;
       }
-      writeSequence(clean);
+      writeSequence();
       final byte[] data = Arrays.copyOf(out.bytes, out.length);
 
       // Now build the index
@@ -330,9 +381,9 @@ public final class WAH8DocIdSet extends 
       } else {
         final int pageSize = 128;
         final int initialPageCount = (valueCount + pageSize - 1) / pageSize;
-        final MonotonicAppendingLongBuffer positions = new MonotonicAppendingLongBuffer(initialPageCount, pageSize);
-        final MonotonicAppendingLongBuffer wordNums = new MonotonicAppendingLongBuffer(initialPageCount, pageSize);
- 
+        final MonotonicAppendingLongBuffer positions = new MonotonicAppendingLongBuffer(initialPageCount, pageSize, PackedInts.COMPACT);
+        final MonotonicAppendingLongBuffer wordNums = new MonotonicAppendingLongBuffer(initialPageCount, pageSize, PackedInts.COMPACT);
+
         positions.add(0L);
         wordNums.add(0L);
         final Iterator it = new Iterator(data, cardinality, Integer.MAX_VALUE, SINGLE_ZERO_BUFFER, SINGLE_ZERO_BUFFER);
@@ -443,20 +494,43 @@ public final class WAH8DocIdSet extends 
     return new Iterator(data, cardinality, indexInterval, positions, wordNums);
   }
 
-  static int readLength(ByteArrayDataInput in, int len) {
-    if ((len & 0x08) == 0) {
-      // no continuation bit
-      return len;
+  static int readCleanLength(ByteArrayDataInput in, int token) {
+    int len = (token >>> 4) & 0x07;
+    final int startPosition = in.getPosition();
+    if ((len & 0x04) != 0) {
+      len = (len & 0x03) | (in.readVInt() << 2);
+    }
+    if (startPosition != 1) {
+      len += 2;
+    }
+    return len;
+  }
+
+  static int readDirtyLength(ByteArrayDataInput in, int token) {
+    int len = token & 0x0F;
+    if ((len & 0x08) != 0) {
+      len = (len & 0x07) | (in.readVInt() << 3);
     }
-    return (len & 0x07) | (in.readVInt() << 3);
+    return len;
   }
 
   static class Iterator extends DocIdSetIterator {
 
+    /* Using the index can be costly for close targets. */
+    static int indexThreshold(int cardinality, int indexInterval) {
+      // Short sequences encode for 3 words (2 clean words and 1 dirty byte),
+      // don't advance if we are going to read less than 3 x indexInterval
+      // sequences
+      long indexThreshold = 3L * 3 * indexInterval;
+      return (int) Math.min(Integer.MAX_VALUE, indexThreshold);
+    }
+
     final ByteArrayDataInput in;
     final int cardinality;
     final int indexInterval;
     final MonotonicAppendingLongBuffer positions, wordNums;
+    final int indexThreshold;
+    int allOnesLength;
     int dirtyLength;
 
     int wordNum; // byte offset
@@ -477,6 +551,7 @@ public final class WAH8DocIdSet extends 
       bitList = 0;
       sequenceNum = -1;
       docID = -1;
+      indexThreshold = indexThreshold(cardinality, indexInterval);
     }
 
     boolean readSequence() {
@@ -485,40 +560,64 @@ public final class WAH8DocIdSet extends 
         return false;
       }
       final int token = in.readByte() & 0xFF;
-      final int cleanLength = (in.getPosition() == 1 ? 0 : 2) + readLength(in, token >>> 4);
-      wordNum += cleanLength;
-      dirtyLength = 1 + readLength(in, token & 0x0F);
+      if ((token & (1 << 7)) == 0) {
+        final int cleanLength = readCleanLength(in, token);
+        wordNum += cleanLength;
+      } else {
+        allOnesLength = readCleanLength(in, token);
+      }
+      dirtyLength = readDirtyLength(in, token);
+      assert in.length() - in.getPosition() >= dirtyLength : in.getPosition() + " " + in.length() + " " + dirtyLength;
       ++sequenceNum;
       return true;
     }
 
     void skipDirtyBytes(int count) {
       assert count >= 0;
-      assert count <= dirtyLength;
-      in.skipBytes(count);
+      assert count <= allOnesLength + dirtyLength;
       wordNum += count;
-      dirtyLength -= count;
+      if (count <= allOnesLength) {
+        allOnesLength -= count;
+      } else {
+        count -= allOnesLength;
+        allOnesLength = 0;
+        in.skipBytes(count);
+        dirtyLength -= count;
+      }
     }
 
     void skipDirtyBytes() {
+      wordNum += allOnesLength + dirtyLength;
       in.skipBytes(dirtyLength);
-      wordNum += dirtyLength;
+      allOnesLength = 0;
       dirtyLength = 0;
     }
 
     void nextWord() {
-      if (dirtyLength == 0 && !readSequence()) {
+      if (allOnesLength > 0) {
+        word = (byte) 0xFF;
+        ++wordNum;
+        --allOnesLength;
         return;
       }
-      word = in.readByte();
-      if (word == 0) {
+      if (dirtyLength > 0) {
         word = in.readByte();
-        assert word != 0; // there can never be two consecutive null dirty words
         ++wordNum;
         --dirtyLength;
+        if (word != 0) {
+          return;
+        }
+        if (dirtyLength > 0) {
+          word = in.readByte();
+          ++wordNum;
+          --dirtyLength;
+          assert word != 0; // never more than one consecutive 0
+          return;
+        }
+      }
+      if (readSequence()) {
+        nextWord();
       }
-      ++wordNum;
-      --dirtyLength;
     }
 
     int forwardBinarySearch(int targetWordNum) {
@@ -557,20 +656,20 @@ public final class WAH8DocIdSet extends 
     void advanceWord(int targetWordNum) {
       assert targetWordNum > wordNum;
       int delta = targetWordNum - wordNum;
-      if (delta <= dirtyLength + 1) {
-        if (delta > 1) {
-          skipDirtyBytes(delta - 1);
-        }
+      if (delta <= allOnesLength + dirtyLength + 1) {
+        skipDirtyBytes(delta - 1);
       } else {
         skipDirtyBytes();
         assert dirtyLength == 0;
-        // use the index
-        final int i = forwardBinarySearch(targetWordNum);
-        final int position = (int) positions.get(i);
-        if (position > in.getPosition()) { // if the binary search returned a backward offset, don't move
-          wordNum = (int) wordNums.get(i) - 1;
-          in.setPosition(position);
-          sequenceNum = i * indexInterval - 1;
+        if (delta > indexThreshold) {
+          // use the index
+          final int i = forwardBinarySearch(targetWordNum);
+          final int position = (int) positions.get(i);
+          if (position > in.getPosition()) { // if the binary search returned a backward offset, don't move
+            wordNum = (int) wordNums.get(i) - 1;
+            in.setPosition(position);
+            sequenceNum = i * indexInterval - 1;
+          }
         }
 
         while (true) {
@@ -578,7 +677,7 @@ public final class WAH8DocIdSet extends 
             return;
           }
           delta = targetWordNum - wordNum;
-          if (delta <= dirtyLength + 1) {
+          if (delta <= allOnesLength + dirtyLength + 1) {
             if (delta > 1) {
               skipDirtyBytes(delta - 1);
             }

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/fst/FST.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/fst/FST.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/fst/FST.java Tue Aug 13 04:06:18 2013
@@ -171,6 +171,8 @@ public final class FST<T> {
   private final boolean allowArrayArcs;
 
   private Arc<T> cachedRootArcs[];
+  private Arc<T> assertingCachedRootArcs[]; // only set wit assert
+
 
   /** Represents a single arc. */
   public final static class Arc<T> {
@@ -213,7 +215,7 @@ public final class FST<T> {
       }
       return this;
     }
-
+    
     boolean flag(int flag) {
       return FST.flag(flags, flag);
     }
@@ -423,11 +425,18 @@ public final class FST<T> {
       return node;
     }
   }
-
+  
   // Caches first 128 labels
   @SuppressWarnings({"rawtypes","unchecked"})
   private void cacheRootArcs() throws IOException {
     cachedRootArcs = (Arc<T>[]) new Arc[0x80];
+    readRootArcs(cachedRootArcs);
+    
+    assert setAssertingRootArcs(cachedRootArcs);
+    assert assertRootArcs();
+  }
+  
+  public void readRootArcs(Arc<T>[] arcs) throws IOException {
     final Arc<T> arc = new Arc<T>();
     getFirstArc(arc);
     final BytesReader in = getBytesReader();
@@ -436,7 +445,7 @@ public final class FST<T> {
       while(true) {
         assert arc.label != END_LABEL;
         if (arc.label < cachedRootArcs.length) {
-          cachedRootArcs[arc.label] = new Arc<T>().copyFrom(arc);
+          arcs[arc.label] = new Arc<T>().copyFrom(arc);
         } else {
           break;
         }
@@ -447,6 +456,38 @@ public final class FST<T> {
       }
     }
   }
+  
+  @SuppressWarnings({"rawtypes","unchecked"})
+  private boolean setAssertingRootArcs(Arc<T>[] arcs) throws IOException {
+    assertingCachedRootArcs = (Arc<T>[]) new Arc[arcs.length];
+    readRootArcs(assertingCachedRootArcs);
+    return true;
+  }
+  
+  private boolean assertRootArcs() {
+    assert cachedRootArcs != null;
+    assert assertingCachedRootArcs != null;
+    for (int i = 0; i < cachedRootArcs.length; i++) {
+      final Arc<T> root = cachedRootArcs[i];
+      final Arc<T> asserting = assertingCachedRootArcs[i];
+      if (root != null) { 
+        assert root.arcIdx == asserting.arcIdx;
+        assert root.bytesPerArc == asserting.bytesPerArc;
+        assert root.flags == asserting.flags;
+        assert root.label == asserting.label;
+        assert root.nextArc == asserting.nextArc;
+        assert root.nextFinalOutput.equals(asserting.nextFinalOutput);
+        assert root.node == asserting.node;
+        assert root.numArcs == asserting.numArcs;
+        assert root.output.equals(asserting.output);
+        assert root.posArcsStart == asserting.posArcsStart;
+        assert root.target == asserting.target;
+      } else {
+        assert root == null && asserting == null;
+      } 
+    }
+    return true;
+  }
 
   public T getEmptyOutput() {
     return emptyOutput;
@@ -1105,7 +1146,7 @@ public final class FST<T> {
   /** Finds an arc leaving the incoming arc, replacing the arc in place.
    *  This returns null if the arc was not found, else the incoming arc. */
   public Arc<T> findTargetArc(int labelToMatch, Arc<T> follow, Arc<T> arc, BytesReader in) throws IOException {
-    assert cachedRootArcs != null;
+    assert assertRootArcs();
 
     if (labelToMatch == END_LABEL) {
       if (follow.isFinal()) {
@@ -1129,7 +1170,7 @@ public final class FST<T> {
     if (follow.target == startNode && labelToMatch < cachedRootArcs.length) {
       final Arc<T> result = cachedRootArcs[labelToMatch];
       if (result == null) {
-        return result;
+        return null;
       } else {
         arc.copyFrom(result);
         return arc;

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/packed/AbstractAppendingLongBuffer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/packed/AbstractAppendingLongBuffer.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/packed/AbstractAppendingLongBuffer.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/packed/AbstractAppendingLongBuffer.java Tue Aug 13 04:06:18 2013
@@ -17,14 +17,14 @@ package org.apache.lucene.util.packed;
  * limitations under the License.
  */
 
-import static org.apache.lucene.util.packed.PackedInts.checkBlockSize;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.RamUsageEstimator;
 
 import java.util.Arrays;
 
-import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.RamUsageEstimator;
+import static org.apache.lucene.util.packed.PackedInts.checkBlockSize;
 
-/** Common functionality shared by {@link AppendingLongBuffer} and {@link MonotonicAppendingLongBuffer}. */
+/** Common functionality shared by {@link AppendingDeltaPackedLongBuffer} and {@link MonotonicAppendingLongBuffer}. */
 abstract class AbstractAppendingLongBuffer {
 
   static final int MIN_PAGE_SIZE = 64;
@@ -33,21 +33,21 @@ abstract class AbstractAppendingLongBuff
   static final int MAX_PAGE_SIZE = 1 << 20;
 
   final int pageShift, pageMask;
-  long[] minValues;
-  PackedInts.Reader[] deltas;
-  private long deltasBytes;
+  PackedInts.Reader[] values;
+  private long valuesBytes;
   int valuesOff;
   long[] pending;
   int pendingOff;
+  float acceptableOverheadRatio;
 
-  AbstractAppendingLongBuffer(int initialBlockCount, int pageSize) {
-    minValues = new long[initialBlockCount];
-    deltas = new PackedInts.Reader[initialBlockCount];
+  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() {
@@ -58,7 +58,7 @@ abstract class AbstractAppendingLongBuff
   public final long size() {
     long size = pendingOff;
     if (valuesOff > 0) {
-      size += deltas[valuesOff - 1].size();
+      size += values[valuesOff - 1].size();
     }
     if (valuesOff > 1) {
       size += (long) (valuesOff - 1) * pageSize();
@@ -73,12 +73,12 @@ abstract class AbstractAppendingLongBuff
     }
     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();
-      deltasBytes += deltas[valuesOff].ramBytesUsed();
+      valuesBytes += values[valuesOff].ramBytesUsed();
       ++valuesOff;
       // reset pending buffer
       pendingOff = 0;
@@ -87,8 +87,7 @@ 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();
@@ -101,11 +100,33 @@ abstract class AbstractAppendingLongBuff
     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);
+
+
+  /** Return an iterator over the values of this buffer. */
+  public Iterator iterator() {
+    return new Iterator();
+  }
 
-  abstract class Iterator {
+  final public class Iterator {
 
     long[] currentValues;
     int vOff, pOff;
@@ -117,12 +138,22 @@ abstract class AbstractAppendingLongBuff
         currentValues = pending;
         currentCount = pendingOff;
       } else {
-        currentValues = new long[deltas[0].size()];
+        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() {
@@ -149,33 +180,31 @@ abstract class AbstractAppendingLongBuff
 
   long baseRamBytesUsed() {
     return RamUsageEstimator.NUM_BYTES_OBJECT_HEADER
-        + 3 * RamUsageEstimator.NUM_BYTES_OBJECT_REF // the 3 arrays
+        + 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_LONG; // deltasBytes
+        + 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())
         + (pending != null ? RamUsageEstimator.sizeOf(pending) : 0L)
-        + RamUsageEstimator.sizeOf(minValues)
-        + RamUsageEstimator.alignObjectSize(RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + (long) RamUsageEstimator.NUM_BYTES_OBJECT_REF * deltas.length); // values
+        + RamUsageEstimator.alignObjectSize(RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + (long) RamUsageEstimator.NUM_BYTES_OBJECT_REF * values.length); // values
 
-    return bytesUsed + deltasBytes;
+    return bytesUsed + valuesBytes;
   }
 
   /** Pack all pending values in this buffer. Subsequent calls to {@link #add(long)} will fail. */
   public void freeze() {
     if (pendingOff > 0) {
-      if (deltas.length == valuesOff) {
+      if (values.length == valuesOff) {
         grow(valuesOff + 1); // don't oversize!
       }
       packPendingValues();
-      deltasBytes += deltas[valuesOff].ramBytesUsed();
+      valuesBytes += values[valuesOff].ramBytesUsed();
       ++valuesOff;
       pendingOff = 0;
     }

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/packed/MonotonicAppendingLongBuffer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/packed/MonotonicAppendingLongBuffer.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/packed/MonotonicAppendingLongBuffer.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/packed/MonotonicAppendingLongBuffer.java Tue Aug 13 04:06:18 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,36 +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);
   }
 
   float[] averages;
+  long[] minValues;
 
-  /** @param initialPageCount the initial number of pages
-   *  @param pageSize         the size of a single page */
-  public MonotonicAppendingLongBuffer(int initialPageCount, int pageSize) {
-    super(initialPageCount, pageSize);
-    averages = new float[pageSize];
+  /**
+   * @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() {
+    this(16, 1024, PackedInts.DEFAULT);
   }
 
-  /** Create an {@link MonotonicAppendingLongBuffer} with initialPageCount=16
-   *  and pageSize=1024. */
-  public MonotonicAppendingLongBuffer() {
-    this(16, 1024);
+  /**
+   * 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;
       }
     }
   }
@@ -70,6 +112,7 @@ public final class MonotonicAppendingLon
   void grow(int newBlockCount) {
     super.grow(newBlockCount);
     this.averages = Arrays.copyOf(averages, newBlockCount);
+    this.minValues = Arrays.copyOf(minValues, newBlockCount);
   }
 
   @Override
@@ -91,58 +134,27 @@ public final class MonotonicAppendingLon
       }
     }
     if (maxDelta == 0) {
-      deltas[valuesOff] = new  PackedInts.NullReader(pendingOff);
+      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. */
-  @Override
-  public Iterator iterator() {
-    return new Iterator();
-  }
-
-  /** A long iterator. */
-  public final class Iterator extends AbstractAppendingLongBuffer.Iterator {
-
-    Iterator() {
-      super();
-    }
-
-    @Override
-    void fillValues() {
-      if (vOff == valuesOff) {
-        currentValues = pending;
-        currentCount = pendingOff;
-      } else {
-        currentCount = deltas[vOff].size();
-        for (int k = 0; k < currentCount; ) {
-          k += deltas[vOff].get(k, currentValues, k, currentCount - k);
-        }
-        for (int k = 0; k < currentCount; ++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/lucene3069/lucene/core/src/java/org/apache/lucene/util/packed/MonotonicBlockPackedReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/packed/MonotonicBlockPackedReader.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/packed/MonotonicBlockPackedReader.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/packed/MonotonicBlockPackedReader.java Tue Aug 13 04:06:18 2013
@@ -78,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/lucene3069/lucene/core/src/java/org/apache/lucene/util/packed/Packed64SingleBlock.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/packed/Packed64SingleBlock.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/packed/Packed64SingleBlock.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/packed/Packed64SingleBlock.java Tue Aug 13 04:06:18 2013
@@ -586,4 +586,4 @@ abstract class Packed64SingleBlock exten
 
   }
 
-}
\ No newline at end of file
+}

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java Tue Aug 13 04:06:18 2013
@@ -705,6 +705,9 @@ public class PackedInts {
 
     @Override
     public int get(int index, long[] arr, int off, int len) {
+      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;
     }

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/packed/PagedGrowableWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/packed/PagedGrowableWriter.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/packed/PagedGrowableWriter.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/packed/PagedGrowableWriter.java Tue Aug 13 04:06:18 2013
@@ -23,7 +23,7 @@ import org.apache.lucene.util.packed.Pac
 /**
  * A {@link PagedGrowableWriter}. This class slices data into fixed-size blocks
  * which have independent numbers of bits per value and grow on-demand.
- * <p>You should use this class instead of {@link AppendingLongBuffer} only when
+ * <p>You should use this class instead of the {@link AbstractAppendingLongBuffer} related ones only when
  * you need random write-access. Otherwise this class will likely be slower and
  * less memory-efficient.
  * @lucene.internal

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/packed/gen_Direct.py
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/packed/gen_Direct.py?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/packed/gen_Direct.py (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/packed/gen_Direct.py Tue Aug 13 04:06:18 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/lucene3069/lucene/core/src/java/org/apache/lucene/util/packed/gen_Packed64SingleBlock.py
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/packed/gen_Packed64SingleBlock.py?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/packed/gen_Packed64SingleBlock.py (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/packed/gen_Packed64SingleBlock.py Tue Aug 13 04:06:18 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/lucene3069/lucene/core/src/java/org/apache/lucene/util/packed/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/packed/package.html?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/packed/package.html (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/util/packed/package.html Tue Aug 13 04:06:18 2013
@@ -50,16 +50,19 @@
     <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 AppendingLongBuffer instead if you don't need random write access.</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.AppendingLongBuffer}</b><ul>
+    <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/lucene3069/lucene/core/src/test/org/apache/lucene/analysis/TestMockAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/analysis/TestMockAnalyzer.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/analysis/TestMockAnalyzer.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/analysis/TestMockAnalyzer.java Tue Aug 13 04:06:18 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;
@@ -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/lucene3069/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestCompressingTermVectorsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestCompressingTermVectorsFormat.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestCompressingTermVectorsFormat.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestCompressingTermVectorsFormat.java Tue Aug 13 04:06:18 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/lucene3069/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestLucene40PostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestLucene40PostingsFormat.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestLucene40PostingsFormat.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestLucene40PostingsFormat.java Tue Aug 13 04:06:18 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/lucene3069/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestReuseDocsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestReuseDocsEnum.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestReuseDocsEnum.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestReuseDocsEnum.java Tue Aug 13 04:06:18 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/lucene3069/lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat2.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat2.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat2.java Tue Aug 13 04:06:18 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/lucene3069/lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat3.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat3.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat3.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat3.java Tue Aug 13 04:06:18 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/lucene3069/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat2.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat2.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat2.java Tue Aug 13 04:06:18 2013
@@ -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/lucene3069/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java Tue Aug 13 04:06:18 2013
@@ -43,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;
 
@@ -1205,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/lucene3069/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java Tue Aug 13 04:06:18 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/lucene3069/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java Tue Aug 13 04:06:18 2013
@@ -822,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();

Modified: lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/index/TestDoc.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/index/TestDoc.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/index/TestDoc.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/index/TestDoc.java Tue Aug 13 04:06:18 2013
@@ -212,15 +212,15 @@ public class TestDoc extends LuceneTestC
    private SegmentInfoPerCommit merge(Directory dir, SegmentInfoPerCommit si1, SegmentInfoPerCommit si2, String merged, boolean useCompoundFile)
    throws Exception {
       IOContext context = newIOContext(random());
-      SegmentReader r1 = new SegmentReader(si1, DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR, context);
-      SegmentReader r2 = new SegmentReader(si2, DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR, context);
+      SegmentReader r1 = new SegmentReader(si1, context);
+      SegmentReader r2 = new SegmentReader(si2, context);
 
       final Codec codec = Codec.getDefault();
       TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(si1.info.dir);
       final SegmentInfo si = new SegmentInfo(si1.info.dir, Constants.LUCENE_MAIN_VERSION, merged, -1, false, codec, null, null);
 
       SegmentMerger merger = new SegmentMerger(Arrays.<AtomicReader>asList(r1, r2),
-          si, InfoStream.getDefault(), trackingDir, IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL,
+          si, InfoStream.getDefault(), trackingDir,
           MergeState.CheckAbort.NONE, new FieldInfos.FieldNumbers(), context);
 
       MergeState mergeState = merger.merge();
@@ -245,7 +245,7 @@ public class TestDoc extends LuceneTestC
 
    private void printSegment(PrintWriter out, SegmentInfoPerCommit si)
    throws Exception {
-      SegmentReader reader = new SegmentReader(si, DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random()));
+      SegmentReader reader = new SegmentReader(si, newIOContext(random()));
 
       for (int i = 0; i < reader.numDocs(); i++)
         out.println(reader.document(i));

Modified: lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/index/TestDocTermOrds.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/index/TestDocTermOrds.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/index/TestDocTermOrds.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/index/TestDocTermOrds.java Tue Aug 13 04:06:18 2013
@@ -332,7 +332,7 @@ public class TestDocTermOrds extends Luc
         Terms terms = MultiFields.getTerms(r, "field");
         if (terms != null) {
           TermsEnum termsEnum = terms.iterator(null);
-          TermsEnum.SeekStatus result = termsEnum.seekCeil(prefixRef, false);
+          TermsEnum.SeekStatus result = termsEnum.seekCeil(prefixRef);
           if (result != TermsEnum.SeekStatus.END) {
             assertFalse("term=" + termsEnum.term().utf8ToString() + " matches prefix=" + prefixRef.utf8ToString(), StringHelper.startsWith(termsEnum.term(), prefixRef));
           } else {
@@ -454,16 +454,16 @@ public class TestDocTermOrds extends Luc
     assertEquals(SeekStatus.END, termsEnum.seekCeil(new BytesRef("zzz")));
     
     // seekExact()
-    assertTrue(termsEnum.seekExact(new BytesRef("beer"), true));
+    assertTrue(termsEnum.seekExact(new BytesRef("beer")));
     assertEquals("beer", termsEnum.term().utf8ToString());
     assertEquals(0, termsEnum.ord());
-    assertTrue(termsEnum.seekExact(new BytesRef("hello"), true));
+    assertTrue(termsEnum.seekExact(new BytesRef("hello")));
     assertEquals("hello", termsEnum.term().utf8ToString());
     assertEquals(1, termsEnum.ord());
-    assertTrue(termsEnum.seekExact(new BytesRef("world"), true));
+    assertTrue(termsEnum.seekExact(new BytesRef("world")));
     assertEquals("world", termsEnum.term().utf8ToString());
     assertEquals(2, termsEnum.ord());
-    assertFalse(termsEnum.seekExact(new BytesRef("bogus"), true));
+    assertFalse(termsEnum.seekExact(new BytesRef("bogus")));
     
     // seek(ord)
     termsEnum.seekExact(0);

Modified: lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/index/TestDocValuesIndexing.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/index/TestDocValuesIndexing.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/index/TestDocValuesIndexing.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/index/TestDocValuesIndexing.java Tue Aug 13 04:06:18 2013
@@ -558,13 +558,13 @@ public class TestDocValuesIndexing exten
   public void testIllegalTypeChangeAcrossSegments() throws Exception {
     Directory dir = newDirectory();
     IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
-    IndexWriter writer = new IndexWriter(dir, conf);
+    IndexWriter writer = new IndexWriter(dir, conf.clone());
     Document doc = new Document();
     doc.add(new NumericDocValuesField("dv", 0L));
     writer.addDocument(doc);
     writer.close();
 
-    writer = new IndexWriter(dir, conf);
+    writer = new IndexWriter(dir, conf.clone());
     doc = new Document();
     doc.add(new SortedDocValuesField("dv", new BytesRef("foo")));
     try {
@@ -580,13 +580,13 @@ public class TestDocValuesIndexing exten
   public void testTypeChangeAfterCloseAndDeleteAll() throws Exception {
     Directory dir = newDirectory();
     IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
-    IndexWriter writer = new IndexWriter(dir, conf);
+    IndexWriter writer = new IndexWriter(dir, conf.clone());
     Document doc = new Document();
     doc.add(new NumericDocValuesField("dv", 0L));
     writer.addDocument(doc);
     writer.close();
 
-    writer = new IndexWriter(dir, conf);
+    writer = new IndexWriter(dir, conf.clone());
     writer.deleteAll();
     doc = new Document();
     doc.add(new SortedDocValuesField("dv", new BytesRef("foo")));
@@ -629,13 +629,13 @@ public class TestDocValuesIndexing exten
   public void testTypeChangeAfterOpenCreate() throws Exception {
     Directory dir = newDirectory();
     IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
-    IndexWriter writer = new IndexWriter(dir, conf);
+    IndexWriter writer = new IndexWriter(dir, conf.clone());
     Document doc = new Document();
     doc.add(new NumericDocValuesField("dv", 0L));
     writer.addDocument(doc);
     writer.close();
     conf.setOpenMode(IndexWriterConfig.OpenMode.CREATE);
-    writer = new IndexWriter(dir, conf);
+    writer = new IndexWriter(dir, conf.clone());
     doc = new Document();
     doc.add(new SortedDocValuesField("dv", new BytesRef("foo")));
     writer.addDocument(doc);
@@ -646,14 +646,14 @@ public class TestDocValuesIndexing exten
   public void testTypeChangeViaAddIndexes() throws Exception {
     Directory dir = newDirectory();
     IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
-    IndexWriter writer = new IndexWriter(dir, conf);
+    IndexWriter writer = new IndexWriter(dir, conf.clone());
     Document doc = new Document();
     doc.add(new NumericDocValuesField("dv", 0L));
     writer.addDocument(doc);
     writer.close();
 
     Directory dir2 = newDirectory();
-    writer = new IndexWriter(dir2, conf);
+    writer = new IndexWriter(dir2, conf.clone());
     doc = new Document();
     doc.add(new SortedDocValuesField("dv", new BytesRef("foo")));
     writer.addDocument(doc);
@@ -672,14 +672,14 @@ public class TestDocValuesIndexing exten
   public void testTypeChangeViaAddIndexesIR() throws Exception {
     Directory dir = newDirectory();
     IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
-    IndexWriter writer = new IndexWriter(dir, conf);
+    IndexWriter writer = new IndexWriter(dir, conf.clone());
     Document doc = new Document();
     doc.add(new NumericDocValuesField("dv", 0L));
     writer.addDocument(doc);
     writer.close();
 
     Directory dir2 = newDirectory();
-    writer = new IndexWriter(dir2, conf);
+    writer = new IndexWriter(dir2, conf.clone());
     doc = new Document();
     doc.add(new SortedDocValuesField("dv", new BytesRef("foo")));
     writer.addDocument(doc);
@@ -700,14 +700,14 @@ public class TestDocValuesIndexing exten
   public void testTypeChangeViaAddIndexes2() throws Exception {
     Directory dir = newDirectory();
     IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
-    IndexWriter writer = new IndexWriter(dir, conf);
+    IndexWriter writer = new IndexWriter(dir, conf.clone());
     Document doc = new Document();
     doc.add(new NumericDocValuesField("dv", 0L));
     writer.addDocument(doc);
     writer.close();
 
     Directory dir2 = newDirectory();
-    writer = new IndexWriter(dir2, conf);
+    writer = new IndexWriter(dir2, conf.clone());
     writer.addIndexes(dir);
     doc = new Document();
     doc.add(new SortedDocValuesField("dv", new BytesRef("foo")));
@@ -725,14 +725,14 @@ public class TestDocValuesIndexing exten
   public void testTypeChangeViaAddIndexesIR2() throws Exception {
     Directory dir = newDirectory();
     IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
-    IndexWriter writer = new IndexWriter(dir, conf);
+    IndexWriter writer = new IndexWriter(dir, conf.clone());
     Document doc = new Document();
     doc.add(new NumericDocValuesField("dv", 0L));
     writer.addDocument(doc);
     writer.close();
 
     Directory dir2 = newDirectory();
-    writer = new IndexWriter(dir2, conf);
+    writer = new IndexWriter(dir2, conf.clone());
     IndexReader[] readers = new IndexReader[] {DirectoryReader.open(dir)};
     writer.addIndexes(readers);
     readers[0].close();

Modified: lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/index/TestDocsAndPositions.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/index/TestDocsAndPositions.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/index/TestDocsAndPositions.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/index/TestDocsAndPositions.java Tue Aug 13 04:06:18 2013
@@ -95,7 +95,7 @@ public class TestDocsAndPositions extend
     Terms terms = reader.terms(fieldName);
     if (terms != null) {
       TermsEnum te = terms.iterator(null);
-      if (te.seekExact(bytes, true)) {
+      if (te.seekExact(bytes)) {
         return te.docsAndPositions(liveDocs, null);
       }
     }
@@ -341,7 +341,7 @@ public class TestDocsAndPositions extend
     
     // now reuse and check again
     TermsEnum te = r.terms("foo").iterator(null);
-    assertTrue(te.seekExact(new BytesRef("bar"), true));
+    assertTrue(te.seekExact(new BytesRef("bar")));
     disi = _TestUtil.docs(random(), te, null, disi, DocsEnum.FLAG_NONE);
     docid = disi.docID();
     assertEquals(-1, docid);
@@ -366,7 +366,7 @@ public class TestDocsAndPositions extend
     
     // now reuse and check again
     TermsEnum te = r.terms("foo").iterator(null);
-    assertTrue(te.seekExact(new BytesRef("bar"), true));
+    assertTrue(te.seekExact(new BytesRef("bar")));
     disi = te.docsAndPositions(null, disi);
     docid = disi.docID();
     assertEquals(-1, docid);

Modified: lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/index/TestDocumentWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/index/TestDocumentWriter.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/index/TestDocumentWriter.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/index/TestDocumentWriter.java Tue Aug 13 04:06:18 2013
@@ -65,7 +65,7 @@ public class TestDocumentWriter extends 
     SegmentInfoPerCommit info = writer.newestSegment();
     writer.close();
     //After adding the document, we should be able to read it back in
-    SegmentReader reader = new SegmentReader(info, DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random()));
+    SegmentReader reader = new SegmentReader(info, newIOContext(random()));
     assertTrue(reader != null);
     StoredDocument doc = reader.document(0);
     assertTrue(doc != null);
@@ -126,7 +126,7 @@ public class TestDocumentWriter extends 
     writer.commit();
     SegmentInfoPerCommit info = writer.newestSegment();
     writer.close();
-    SegmentReader reader = new SegmentReader(info, DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random()));
+    SegmentReader reader = new SegmentReader(info, newIOContext(random()));
 
     DocsAndPositionsEnum termPositions = MultiFields.getTermPositionsEnum(reader, MultiFields.getLiveDocs(reader),
                                                                           "repeated", new BytesRef("repeated"));
@@ -198,7 +198,7 @@ public class TestDocumentWriter extends 
     writer.commit();
     SegmentInfoPerCommit info = writer.newestSegment();
     writer.close();
-    SegmentReader reader = new SegmentReader(info, DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random()));
+    SegmentReader reader = new SegmentReader(info, newIOContext(random()));
 
     DocsAndPositionsEnum termPositions = MultiFields.getTermPositionsEnum(reader, reader.getLiveDocs(), "f1", new BytesRef("a"));
     assertTrue(termPositions.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
@@ -241,7 +241,7 @@ public class TestDocumentWriter extends 
     writer.commit();
     SegmentInfoPerCommit info = writer.newestSegment();
     writer.close();
-    SegmentReader reader = new SegmentReader(info, DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random()));
+    SegmentReader reader = new SegmentReader(info, newIOContext(random()));
 
     DocsAndPositionsEnum termPositions = reader.termPositionsEnum(new Term("preanalyzed", "term1"));
     assertTrue(termPositions.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);

Modified: lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java Tue Aug 13 04:06:18 2013
@@ -1305,36 +1305,6 @@ public class TestIndexWriter extends Luc
     dir.close();
   }
 
-  public void testIndexDivisor() throws Exception {
-    Directory dir = newDirectory();
-    IndexWriterConfig config = new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
-    config.setTermIndexInterval(2);
-    IndexWriter w = new IndexWriter(dir, config);
-    StringBuilder s = new StringBuilder();
-    // must be > 256
-    for(int i=0;i<300;i++) {
-      s.append(' ').append(i);
-    }
-    Document d = new Document();
-    Field f = newTextField("field", s.toString(), Field.Store.NO);
-    d.add(f);
-    w.addDocument(d);
-
-    AtomicReader r = getOnlySegmentReader(w.getReader());
-    TermsEnum t = r.fields().terms("field").iterator(null);
-    int count = 0;
-    while(t.next() != null) {
-      final DocsEnum docs = _TestUtil.docs(random(), t, null, null, DocsEnum.FLAG_NONE);
-      assertEquals(0, docs.nextDoc());
-      assertEquals(DocIdSetIterator.NO_MORE_DOCS, docs.nextDoc());
-      count++;
-    }
-    assertEquals(300, count);
-    r.close();
-    w.close();
-    dir.close();
-  }
-
   public void testDeleteUnusedFiles() throws Exception {
     for(int iter=0;iter<2;iter++) {
       Directory dir = newMockDirectory(); // relies on windows semantics
@@ -1716,20 +1686,6 @@ public class TestIndexWriter extends Luc
     dir.close();
   }
 
-  // LUCENE-3183
-  public void testEmptyFieldNameTIIOne() throws IOException {
-    Directory dir = newDirectory();
-    IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
-    iwc.setTermIndexInterval(1);
-    iwc.setReaderTermsIndexDivisor(1);
-    IndexWriter writer = new IndexWriter(dir, iwc);
-    Document doc = new Document();
-    doc.add(newTextField("", "a b c", Field.Store.NO));
-    writer.addDocument(doc);
-    writer.close();
-    dir.close();
-  }
-
   public void testDeleteAllNRTLeftoverFiles() throws Exception {
 
     Directory d = new MockDirectoryWrapper(random(), new RAMDirectory());