You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by si...@apache.org on 2012/09/21 19:22:27 UTC

svn commit: r1388574 [20/45] - in /lucene/dev/branches/LUCENE-2878: ./ dev-tools/ dev-tools/eclipse/ dev-tools/eclipse/dot.settings/ dev-tools/idea/ dev-tools/idea/.idea/ dev-tools/idea/.idea/libraries/ dev-tools/idea/lucene/ dev-tools/idea/lucene/anal...

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/Packed64.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/Packed64.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/Packed64.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/Packed64.java Fri Sep 21 17:21:34 2012
@@ -17,12 +17,12 @@ package org.apache.lucene.util.packed;
  * limitations under the License.
  */
 
-import org.apache.lucene.store.DataInput;
-import org.apache.lucene.util.RamUsageEstimator;
-
 import java.io.IOException;
 import java.util.Arrays;
 
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.util.RamUsageEstimator;
+
 /**
  * Space optimized random access capable array of values with a fixed number of
  * bits/value. Values are packed contiguously.
@@ -146,12 +146,12 @@ class Packed64 extends PackedInts.Mutabl
     assert off + len <= arr.length;
 
     final int originalIndex = index;
-    final BulkOperation op = BulkOperation.of(PackedInts.Format.PACKED, bitsPerValue);
+    final PackedInts.Decoder decoder = BulkOperation.of(PackedInts.Format.PACKED, bitsPerValue);
 
     // go to the next block where the value does not span across two blocks
-    final int offsetInBlocks = index % op.values();
+    final int offsetInBlocks = index % decoder.valueCount();
     if (offsetInBlocks != 0) {
-      for (int i = offsetInBlocks; i < op.values() && len > 0; ++i) {
+      for (int i = offsetInBlocks; i < decoder.valueCount() && len > 0; ++i) {
         arr[off++] = get(index++);
         --len;
       }
@@ -161,12 +161,12 @@ class Packed64 extends PackedInts.Mutabl
     }
 
     // bulk get
-    assert index % op.values() == 0;
+    assert index % decoder.valueCount() == 0;
     int blockIndex = (int) ((long) index * bitsPerValue) >>> BLOCK_BITS;
     assert (((long)index * bitsPerValue) & MOD_MASK) == 0;
-    final int iterations = len / op.values();
-    op.get(blocks, blockIndex, arr, off, iterations);
-    final int gotValues = iterations * op.values();
+    final int iterations = len / decoder.valueCount();
+    decoder.decode(blocks, blockIndex, arr, off, iterations);
+    final int gotValues = iterations * decoder.valueCount();
     index += gotValues;
     len -= gotValues;
     assert len >= 0;
@@ -210,12 +210,12 @@ class Packed64 extends PackedInts.Mutabl
     assert off + len <= arr.length;
 
     final int originalIndex = index;
-    final BulkOperation op = BulkOperation.of(PackedInts.Format.PACKED, bitsPerValue);
+    final PackedInts.Encoder encoder = BulkOperation.of(PackedInts.Format.PACKED, bitsPerValue);
 
     // go to the next block where the value does not span across two blocks
-    final int offsetInBlocks = index % op.values();
+    final int offsetInBlocks = index % encoder.valueCount();
     if (offsetInBlocks != 0) {
-      for (int i = offsetInBlocks; i < op.values() && len > 0; ++i) {
+      for (int i = offsetInBlocks; i < encoder.valueCount() && len > 0; ++i) {
         set(index++, arr[off++]);
         --len;
       }
@@ -224,13 +224,13 @@ class Packed64 extends PackedInts.Mutabl
       }
     }
 
-    // bulk get
-    assert index % op.values() == 0;
+    // bulk set
+    assert index % encoder.valueCount() == 0;
     int blockIndex = (int) ((long) index * bitsPerValue) >>> BLOCK_BITS;
     assert (((long)index * bitsPerValue) & MOD_MASK) == 0;
-    final int iterations = len / op.values();
-    op.set(blocks, blockIndex, arr, off, iterations);
-    final int setValues = iterations * op.values();
+    final int iterations = len / encoder.valueCount();
+    encoder.encode(arr, off, blocks, blockIndex, iterations);
+    final int setValues = iterations * encoder.valueCount();
     index += setValues;
     len -= setValues;
     assert len >= 0;

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/Packed64SingleBlock.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/Packed64SingleBlock.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/Packed64SingleBlock.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/Packed64SingleBlock.java Fri Sep 21 17:21:34 2012
@@ -86,12 +86,12 @@ abstract class Packed64SingleBlock exten
 
     // bulk get
     assert index % valuesPerBlock == 0;
-    final BulkOperation op = BulkOperation.of(PackedInts.Format.PACKED_SINGLE_BLOCK, bitsPerValue);
-    assert op.blocks() == 1;
-    assert op.values() == valuesPerBlock;
+    final PackedInts.Decoder decoder = BulkOperation.of(PackedInts.Format.PACKED_SINGLE_BLOCK, bitsPerValue);
+    assert decoder.blockCount() == 1;
+    assert decoder.valueCount() == valuesPerBlock;
     final int blockIndex = index / valuesPerBlock;
     final int nblocks = (index + len) / valuesPerBlock - blockIndex;
-    op.get(blocks, blockIndex, arr, off, nblocks);
+    decoder.decode(blocks, blockIndex, arr, off, nblocks);
     final int diff = nblocks * valuesPerBlock;
     index += diff; len -= diff;
 
@@ -131,11 +131,11 @@ abstract class Packed64SingleBlock exten
     // bulk set
     assert index % valuesPerBlock == 0;
     final BulkOperation op = BulkOperation.of(PackedInts.Format.PACKED_SINGLE_BLOCK, bitsPerValue);
-    assert op.blocks() == 1;
-    assert op.values() == valuesPerBlock;
+    assert op.blockCount() == 1;
+    assert op.valueCount() == valuesPerBlock;
     final int blockIndex = index / valuesPerBlock;
     final int nblocks = (index + len) / valuesPerBlock - blockIndex;
-    op.set(blocks, blockIndex, arr, off, nblocks);
+    op.encode(arr, off, blocks, blockIndex, nblocks);
     final int diff = nblocks * valuesPerBlock;
     index += diff; len -= diff;
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java Fri Sep 21 17:21:34 2012
@@ -18,6 +18,7 @@ package org.apache.lucene.util.packed;
  */
 
 import java.io.Closeable;
+import java.io.IOException;
 
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.store.DataInput;
@@ -25,8 +26,6 @@ import org.apache.lucene.store.DataOutpu
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.LongsRef;
 
-import java.io.IOException;
-
 /**
  * Simplistic compression for array of unsigned long values.
  * Each value is >= 0 and <= a specified maximum value.  The
@@ -66,6 +65,14 @@ public class PackedInts {
   public final static int VERSION_START = 0;
   public final static int VERSION_CURRENT = VERSION_START;
 
+  private static void checkVersion(int version) {
+    if (version < VERSION_START) {
+      throw new IllegalArgumentException("Version is too old, should be at least " + VERSION_START + " (got " + version + ")");
+    } else if (version > VERSION_CURRENT) {
+      throw new IllegalArgumentException("Version is too new, should be at most " + VERSION_CURRENT + " (got " + version + ")");
+    }
+  }
+
   /**
    * A format to write packed ints.
    *
@@ -242,6 +249,146 @@ public class PackedInts {
   }
 
   /**
+   * A decoder for packed integers.
+   */
+  public static interface Decoder {
+
+    /**
+     * The minimum number of long blocks to decode in a single call.
+     */
+    int blockCount();
+
+    /**
+     * The number of values that can be stored in <code>blockCount()</code> long
+     * blocks.
+     */
+    int valueCount();
+
+    /**
+     * Read <code>iterations * blockCount()</code> blocks from <code>blocks</code>,
+     * decode them and write <code>iterations * valueCount()</code> values into
+     * <code>values</code>.
+     *
+     * @param blocks       the long blocks that hold packed integer values
+     * @param blocksOffset the offset where to start reading blocks
+     * @param values       the values buffer
+     * @param valuesOffset the offset where to start writing values
+     * @param iterations   controls how much data to decode
+     */
+    void decode(long[] blocks, int blocksOffset, long[] values, int valuesOffset, int iterations);
+
+    /**
+     * Read <code>8 * iterations * blockCount()</code> blocks from <code>blocks</code>,
+     * decode them and write <code>iterations * valueCount()</code> values into
+     * <code>values</code>.
+     *
+     * @param blocks       the long blocks that hold packed integer values
+     * @param blocksOffset the offset where to start reading blocks
+     * @param values       the values buffer
+     * @param valuesOffset the offset where to start writing values
+     * @param iterations   controls how much data to decode
+     */
+    void decode(byte[] blocks, int blocksOffset, long[] values, int valuesOffset, int iterations);
+
+    /**
+     * Read <code>iterations * blockCount()</code> blocks from <code>blocks</code>,
+     * decode them and write <code>iterations * valueCount()</code> values into
+     * <code>values</code>.
+     *
+     * @param blocks       the long blocks that hold packed integer values
+     * @param blocksOffset the offset where to start reading blocks
+     * @param values       the values buffer
+     * @param valuesOffset the offset where to start writing values
+     * @param iterations   controls how much data to decode
+     */
+    void decode(long[] blocks, int blocksOffset, int[] values, int valuesOffset, int iterations);
+
+    /**
+     * Read <code>8 * iterations * blockCount()</code> blocks from <code>blocks</code>,
+     * decode them and write <code>iterations * valueCount()</code> values into
+     * <code>values</code>.
+     *
+     * @param blocks       the long blocks that hold packed integer values
+     * @param blocksOffset the offset where to start reading blocks
+     * @param values       the values buffer
+     * @param valuesOffset the offset where to start writing values
+     * @param iterations   controls how much data to decode
+     */
+    void decode(byte[] blocks, int blocksOffset, int[] values, int valuesOffset, int iterations);
+
+  }
+
+  /**
+   * An encoder for packed integers.
+   */
+  public static interface Encoder {
+
+    /**
+     * The minimum number of long blocks to encode in a single call.
+     */
+    int blockCount();
+
+    /**
+     * The number of values that can be stored in <code>blockCount()</code> long
+     * blocks.
+     */
+    int valueCount();
+
+    /**
+     * Read <code>iterations * valueCount()</code> values from <code>values</code>,
+     * encode them and write <code>iterations * blockCount()</code> blocks into
+     * <code>blocks</code>.
+     *
+     * @param blocks       the long blocks that hold packed integer values
+     * @param blocksOffset the offset where to start writing blocks
+     * @param values       the values buffer
+     * @param valuesOffset the offset where to start reading values
+     * @param iterations   controls how much data to encode
+     */
+    void encode(long[] values, int valuesOffset, long[] blocks, int blocksOffset, int iterations);
+
+    /**
+     * Read <code>iterations * valueCount()</code> values from <code>values</code>,
+     * encode them and write <code>8 * iterations * blockCount()</code> blocks into
+     * <code>blocks</code>.
+     *
+     * @param blocks       the long blocks that hold packed integer values
+     * @param blocksOffset the offset where to start writing blocks
+     * @param values       the values buffer
+     * @param valuesOffset the offset where to start reading values
+     * @param iterations   controls how much data to encode
+     */
+    void encode(long[] values, int valuesOffset, byte[] blocks, int blocksOffset, int iterations);
+
+    /**
+     * Read <code>iterations * valueCount()</code> values from <code>values</code>,
+     * encode them and write <code>iterations * blockCount()</code> blocks into
+     * <code>blocks</code>.
+     *
+     * @param blocks       the long blocks that hold packed integer values
+     * @param blocksOffset the offset where to start writing blocks
+     * @param values       the values buffer
+     * @param valuesOffset the offset where to start reading values
+     * @param iterations   controls how much data to encode
+     */
+    void encode(int[] values, int valuesOffset, long[] blocks, int blocksOffset, int iterations);
+
+    /**
+     * Read <code>iterations * valueCount()</code> values from <code>values</code>,
+     * encode them and write <code>8 * iterations * blockCount()</code> blocks into
+     * <code>blocks</code>.
+     *
+     * @param blocks       the long blocks that hold packed integer values
+     * @param blocksOffset the offset where to start writing blocks
+     * @param values       the values buffer
+     * @param valuesOffset the offset where to start reading values
+     * @param iterations   controls how much data to encode
+     */
+    void encode(int[] values, int valuesOffset, byte[] blocks, int blocksOffset, int iterations);
+
+  }
+
+  /**
    * A read-only random access array of positive integers.
    * @lucene.internal
    */
@@ -490,8 +637,7 @@ public class PackedInts {
     protected final int valueCount;
     protected final int bitsPerValue;
 
-    protected Writer(DataOutput out, int valueCount, int bitsPerValue)
-      throws IOException {
+    protected Writer(DataOutput out, int valueCount, int bitsPerValue) {
       assert bitsPerValue <= 64;
       assert valueCount >= 0 || valueCount == -1;
       this.out = out;
@@ -529,6 +675,32 @@ public class PackedInts {
   }
 
   /**
+   * Get a {@link Decoder}.
+   *
+   * @param format         the format used to store packed ints
+   * @param version        the compatibility version
+   * @param bitsPerValue   the number of bits per value
+   * @return a decoder
+   */
+  public static Decoder getDecoder(Format format, int version, int bitsPerValue) {
+    checkVersion(version);
+    return BulkOperation.of(format, bitsPerValue);
+  }
+
+  /**
+   * Get an {@link Encoder}.
+   *
+   * @param format         the format used to store packed ints
+   * @param version        the compatibility version
+   * @param bitsPerValue   the number of bits per value
+   * @return an encoder
+   */
+  public static Encoder getEncoder(Format format, int version, int bitsPerValue) {
+    checkVersion(version);
+    return BulkOperation.of(format, bitsPerValue);
+  }
+
+  /**
    * Expert: Restore a {@link Reader} from a stream without reading metadata at
    * the beginning of the stream. This method is useful to restore data from
    * streams which have been created using
@@ -546,6 +718,7 @@ public class PackedInts {
    */
   public static Reader getReaderNoHeader(DataInput in, Format format, int version,
       int valueCount, int bitsPerValue) throws IOException {
+    checkVersion(version);
     switch (format) {
       case PACKED_SINGLE_BLOCK:
         return Packed64SingleBlock.create(in, valueCount, bitsPerValue);
@@ -572,7 +745,7 @@ public class PackedInts {
         }
         return new Packed64(in, valueCount, bitsPerValue);
       default:
-        throw new AssertionError("Unknwown Writer format: " + format);
+        throw new AssertionError("Unknown Writer format: " + format);
     }
   }
 
@@ -607,12 +780,12 @@ public class PackedInts {
    * @param bitsPerValue the number of bits per value
    * @param mem          how much memory the iterator is allowed to use to read-ahead (likely to speed up iteration)
    * @return             a ReaderIterator
-   * @throws IOException
    * @see PackedInts#getWriterNoHeader(DataOutput, Format, int, int, int)
    * @lucene.internal
    */
   public static ReaderIterator getReaderIteratorNoHeader(DataInput in, Format format, int version,
-      int valueCount, int bitsPerValue, int mem) throws IOException {
+      int valueCount, int bitsPerValue, int mem) {
+    checkVersion(version);
     return new PackedReaderIterator(format, valueCount, bitsPerValue, in, mem);
   }
 
@@ -648,11 +821,11 @@ public class PackedInts {
    * @param valueCount   how many values the stream holds
    * @param bitsPerValue the number of bits per value
    * @return a direct Reader
-   * @throws IOException
    * @lucene.internal
    */
   public static Reader getDirectReaderNoHeader(IndexInput in, Format format,
-      int version, int valueCount, int bitsPerValue) throws IOException {
+      int version, int valueCount, int bitsPerValue) {
+    checkVersion(version);
     switch (format) {
       case PACKED:
         return new DirectPackedReader(bitsPerValue, valueCount, in);
@@ -778,13 +951,12 @@ public class PackedInts {
    * @param bitsPerValue the number of bits per value
    * @param mem          how much memory (in bytes) can be used to speed up serialization
    * @return             a Writer
-   * @throws IOException
    * @see PackedInts#getReaderIteratorNoHeader(DataInput, Format, int, int, int, int)
    * @see PackedInts#getReaderNoHeader(DataInput, Format, int, int, int)
    * @lucene.internal
    */
   public static Writer getWriterNoHeader(
-      DataOutput out, Format format, int valueCount, int bitsPerValue, int mem) throws IOException {
+      DataOutput out, Format format, int valueCount, int bitsPerValue, int mem) {
     return new PackedWriter(format, out, valueCount, bitsPerValue, mem);
   }
 
@@ -894,4 +1066,4 @@ public class PackedInts {
     }
   }
 
-}
+}
\ No newline at end of file

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/PackedReaderIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/PackedReaderIterator.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/PackedReaderIterator.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/PackedReaderIterator.java Fri Sep 21 17:21:34 2012
@@ -38,10 +38,10 @@ final class PackedReaderIterator extends
     bulkOperation = BulkOperation.of(format, bitsPerValue);
     iterations = bulkOperation.computeIterations(valueCount, mem);
     assert iterations > 0;
-    nextBlocks = new long[iterations * bulkOperation.blocks()];
-    nextValues = new LongsRef(new long[iterations * bulkOperation.values()], 0, 0);
-    assert iterations * bulkOperation.values() == nextValues.longs.length;
-    assert iterations * bulkOperation.blocks() == nextBlocks.length;
+    nextBlocks = new long[iterations * bulkOperation.blockCount()];
+    nextValues = new LongsRef(new long[iterations * bulkOperation.valueCount()], 0, 0);
+    assert iterations * bulkOperation.valueCount() == nextValues.longs.length;
+    assert iterations * bulkOperation.blockCount() == nextBlocks.length;
     nextValues.offset = nextValues.longs.length;
     position = -1;
   }
@@ -70,7 +70,7 @@ final class PackedReaderIterator extends
         nextBlocks[i] = 0L;
       }
 
-      bulkOperation.get(nextBlocks, 0, nextValues.longs, 0, iterations);
+      bulkOperation.decode(nextBlocks, 0, nextValues.longs, 0, iterations);
       nextValues.offset = 0;
     }
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/PackedWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/PackedWriter.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/PackedWriter.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/PackedWriter.java Fri Sep 21 17:21:34 2012
@@ -21,6 +21,7 @@ import org.apache.lucene.store.DataOutpu
 
 import java.io.EOFException;
 import java.io.IOException;
+import java.util.Arrays;
 
 // Packs high order byte first, to match
 // IndexOutput.writeInt/Long/Short byte order
@@ -29,21 +30,20 @@ final class PackedWriter extends PackedI
 
   boolean finished;
   final PackedInts.Format format;
-  final BulkOperation bulkOperation;
+  final BulkOperation encoder;
   final long[] nextBlocks;
   final long[] nextValues;
   final int iterations;
   int off;
   int written;
 
-  PackedWriter(PackedInts.Format format, DataOutput out, int valueCount, int bitsPerValue, int mem)
-      throws IOException {
+  PackedWriter(PackedInts.Format format, DataOutput out, int valueCount, int bitsPerValue, int mem) {
     super(out, valueCount, bitsPerValue);
     this.format = format;
-    bulkOperation = BulkOperation.of(format, bitsPerValue);
-    iterations = bulkOperation.computeIterations(valueCount, mem);
-    nextBlocks = new long[iterations * bulkOperation.blocks()];
-    nextValues = new long[iterations * bulkOperation.values()];
+    encoder = BulkOperation.of(format, bitsPerValue);
+    iterations = encoder.computeIterations(valueCount, mem);
+    nextBlocks = new long[iterations * encoder.blockCount()];
+    nextValues = new long[iterations * encoder.valueCount()];
     off = 0;
     written = 0;
     finished = false;
@@ -63,8 +63,7 @@ final class PackedWriter extends PackedI
     }
     nextValues[off++] = v;
     if (off == nextValues.length) {
-      flush(nextValues.length);
-      off = 0;
+      flush();
     }
     ++written;
   }
@@ -77,16 +76,17 @@ final class PackedWriter extends PackedI
         add(0L);
       }
     }
-    flush(off);
+    flush();
     finished = true;
   }
 
-  private void flush(int nvalues) throws IOException {
-    bulkOperation.set(nextBlocks, 0, nextValues, 0, iterations);
-    final int blocks = format.nblocks(bitsPerValue, nvalues);
+  private void flush() throws IOException {
+    encoder.encode(nextValues, 0, nextBlocks, 0, iterations);
+    final int blocks = format.nblocks(bitsPerValue, off);
     for (int i = 0; i < blocks; ++i) {
       out.writeLong(nextBlocks[i]);
     }
+    Arrays.fill(nextValues, 0L);
     off = 0;
   }
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/gen_BulkOperation.py
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/gen_BulkOperation.py?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/gen_BulkOperation.py (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/gen_BulkOperation.py Fri Sep 21 17:21:34 2012
@@ -19,6 +19,7 @@ from fractions import gcd
 
 """Code generation for bulk operations"""
 
+MAX_SPECIALIZED_BITS_PER_VALUE = 24;
 PACKED_64_SINGLE_BLOCK_BPV = [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 12, 16, 21, 32]
 OUTPUT_FILE = "BulkOperation.java"
 HEADER = """// This file has been automatically generated, DO NOT EDIT
@@ -42,22 +43,14 @@ package org.apache.lucene.util.packed;
  * limitations under the License.
  */
 
-import java.util.EnumMap;
-
-/**
- * Efficient sequential read/write of packed integers.
- */
-abstract class BulkOperation {
-
-  static final EnumMap<PackedInts.Format, BulkOperation[]> BULK_OPERATIONS = new EnumMap<PackedInts.Format, BulkOperation[]>(PackedInts.Format.class);
+"""
 
-  public static BulkOperation of(PackedInts.Format format, int bitsPerValue) {
-    assert bitsPerValue > 0 && bitsPerValue <= 64;
-    BulkOperation[] ops = BULK_OPERATIONS.get(format);
-    if (ops == null || ops[bitsPerValue] == null) {
-      throw new IllegalArgumentException("format: " + format + ", bitsPerValue: " + bitsPerValue);
+FOOTER="""
+  protected int writeLong(long block, byte[] blocks, int blocksOffset) {
+    for (int j = 1; j <= 8; ++j) {
+      blocks[blocksOffset++] = (byte) (block >>> (64 - (j << 3)));
     }
-    return ops[bitsPerValue];
+    return blocksOffset;
   }
 
   /**
@@ -67,7 +60,7 @@ abstract class BulkOperation {
    *  - 16 bits per value -> b=1, v=4
    *  - 24 bits per value -> b=3, v=8
    *  - 50 bits per value -> b=25, v=32
-   *  - 63 bits per value -> b=63, v = 64
+   *  - 63 bits per value -> b=63, v=64
    *  - ...
    *
    * A bulk read consists in copying <code>iterations*v</code> values that are
@@ -79,186 +72,264 @@ abstract class BulkOperation {
    * <code>ramBudget / (8 * (b + v))</code> (since a long is 8 bytes).
    */
   public final int computeIterations(int valueCount, int ramBudget) {
-    final int iterations = (ramBudget >>> 3) / (blocks() + values());
+    final int iterations = (ramBudget >>> 3) / (blockCount() + valueCount());
     if (iterations == 0) {
       // at least 1
       return 1;
-    } else if ((iterations - 1) * blocks() >= valueCount) {
+    } else if ((iterations - 1) * blockCount() >= valueCount) {
       // don't allocate for more than the size of the reader
-      return (int) Math.ceil((double) valueCount / values());
+      return (int) Math.ceil((double) valueCount / valueCount());
     } else {
       return iterations;
     }
   }
-
-  /**
-   * The minimum number of blocks required to perform a bulk get/set.
-   */
-  public abstract int blocks();
-
-  /**
-   * The number of values that can be stored in <code>blocks()</code> blocks.
-   */
-  public abstract int values();
-
-  /**
-   * Get <code>n * values()</code> values from <code>n * blocks()</code> blocks.
-   */
-  public abstract void get(long[] blocks, int blockIndex, long[] values, int valuesIndex, int iterations);
-
-  /**
-   * Set <code>n * values()</code> values into <code>n * blocks()</code> blocks.
-   */
-  public abstract void set(long[] blocks, int blockIndex, long[] values, int valuesIndex, int iterations);
-
+}
 """
 
-FOOTER = "}"
-
-def packed64singleblock(bpv, f):
-  values = 64 / bpv
-  f.write("\n  static final class Packed64SingleBlockBulkOperation%d extends BulkOperation {\n\n" %bpv)
-  f.write("    public int blocks() {\n")
-  f.write("      return 1;\n")
-  f.write("     }\n\n")
-  f.write("    public int values() {\n")
-  f.write("      return %d;\n" %values)
-  f.write("    }\n\n")
-
-  f.write("    public void get(long[] blocks, int bi, long[] values, int vi, int iterations) {\n")
-  f.write("      assert bi + iterations * blocks() <= blocks.length;\n")
-  f.write("      assert vi + iterations * values() <= values.length;\n")
-  f.write("      for (int i = 0; i < iterations; ++i) {\n")
-  f.write("        final long block = blocks[bi++];\n")
-  mask = (1 << bpv) - 1
-  for i in xrange(values):
-    block_offset = i / values
-    offset_in_block = i % values
-    if i == 0:
-      f.write("        values[vi++] = block & %dL;\n" %mask)
-    elif i == values - 1:
-      f.write("        values[vi++] = block >>> %d;\n" %(i * bpv))
-    else:
-      f.write("        values[vi++] = (block >>> %d) & %dL;\n" %(i * bpv, mask))
-  f.write("      }\n")
-  f.write("    }\n\n")
-
-  f.write("    public void set(long[] blocks, int bi, long[] values, int vi, int iterations) {\n")
-  f.write("      assert bi + iterations * blocks() <= blocks.length;\n")
-  f.write("      assert vi + iterations * values() <= values.length;\n")
-  f.write("      for (int i = 0; i < iterations; ++i) {\n")
-  for i in xrange(values):
-    block_offset = i / values
-    offset_in_block = i % values
-    if i == 0:
-      f.write("        blocks[bi++] = values[vi++]")
-    else:
-      f.write(" | (values[vi++] << %d)" %(i * bpv))
-      if i == values - 1:
-        f.write(";\n")
-  f.write("      }\n")
-  f.write("    }\n")
+def is_power_of_two(n):
+  return n & (n - 1) == 0
 
-  f.write("  }\n")
+def casts(typ):
+  cast_start = "(%s) (" %typ
+  cast_end = ")"
+  if typ == "long":
+    cast_start = ""
+    cast_end = ""
+  return cast_start, cast_end
+
+def hexNoLSuffix(n):
+  # On 32 bit Python values > (1 << 31)-1 will have L appended by hex function:
+  s = hex(n)
+  if s.endswith('L'):
+    s = s[:-1]
+  return s
+
+def masks(bits):
+  if bits == 64:
+    return "", ""
+  return "(", " & %sL)" %(hexNoLSuffix((1 << bits) - 1))
+
+def get_type(bits):
+  if bits == 8:
+    return "byte"
+  elif bits == 16:
+    return "short"
+  elif bits == 32:
+    return "int"
+  elif bits == 64:
+    return "long"
+  else:
+    assert False
 
-def packed64(bpv, f):
+def block_value_count(bpv, bits=64):
   blocks = bpv
-  values = blocks * 64 / bpv
+  values = blocks * bits / bpv
   while blocks % 2 == 0 and values % 2 == 0:
     blocks /= 2
     values /= 2
-  assert values * bpv == 64 * blocks, "%d values, %d blocks, %d bits per value" %(values, blocks, bpv)
+  assert values * bpv == bits * blocks, "%d values, %d blocks, %d bits per value" %(values, blocks, bpv)
+  return (blocks, values)
+
+def packed64(bpv, f):
+  blocks, values = block_value_count(bpv)
   mask = (1 << bpv) - 1
-  f.write("  static final class Packed64BulkOperation%d extends BulkOperation {\n\n" %bpv)
-  f.write("    public int blocks() {\n")
-  f.write("      return %d;\n" %blocks)
-  f.write("    }\n\n")
-  f.write("    public int values() {\n")
-  f.write("      return %d;\n" %values)
-  f.write("    }\n\n")
+
+  f.write("\n")
+  f.write("  public BulkOperationPacked%d() {\n" %bpv)
+  f.write("    super(%d);\n" %bpv)
+  f.write("    assert blockCount() == %d;\n" %blocks)
+  f.write("    assert valueCount() == %d;\n" %values)
+  f.write("  }\n\n")
 
   if bpv == 64:
-    f.write("""    public void get(long[] blocks, int bi, long[] values, int vi, int iterations) {
-      System.arraycopy(blocks, bi, values, vi, iterations);
+    f.write("""    @Override
+    public void decode(long[] blocks, int blocksOffset, long[] values, int valuesOffset, int iterations) {
+      System.arraycopy(blocks, blocksOffset, values, valuesOffset, valueCount() * iterations);
     }
 
-    public void set(long[] blocks, int bi, long[] values, int vi, int iterations) {
-      System.arraycopy(values, bi, blocks, vi, iterations);
+    @Override
+    public void decode(long[] blocks, int blocksOffset, int[] values, int valuesOffset, int iterations) {
+      throw new UnsupportedOperationException();
     }
-  }
-""")
-    return
 
-  f.write("    public void get(long[] blocks, int bi, long[] values, int vi, int iterations) {\n")
-  f.write("      assert bi + iterations * blocks() <= blocks.length;\n")
-  f.write("      assert vi + iterations * values() <= values.length;\n")
-  f.write("      for (int i = 0; i < iterations; ++i) {\n")
-  for i in xrange(0, values):
-    block_offset = i * bpv / 64
-    bit_offset = (i * bpv) % 64
-    if bit_offset == 0:
-      # start of block
-      f.write("        final long block%d = blocks[bi++];\n" %block_offset);
-      f.write("        values[vi++] = block%d >>> %d;\n" %(block_offset, 64 - bpv))
-    elif bit_offset + bpv == 64:
-      # end of block
-      f.write("        values[vi++] = block%d & %dL;\n" %(block_offset, mask))
-    elif bit_offset + bpv < 64:
-      # middle of block
-      f.write("        values[vi++] = (block%d >>> %d) & %dL;\n" %(block_offset, 64 - bit_offset - bpv, mask))
-    else:
-      # value spans across 2 blocks
-      mask1 = (1 << (64 - bit_offset)) -1
-      shift1 = bit_offset + bpv - 64
-      shift2 = 64 - shift1
-      f.write("        final long block%d = blocks[bi++];\n" %(block_offset + 1));
-      f.write("        values[vi++] = ((block%d & %dL) << %d) | (block%d >>> %d);\n" %(block_offset, mask1, shift1, block_offset + 1, shift2))
-  f.write("      }\n")
-  f.write("    }\n\n")
-
-  f.write("    public void set(long[] blocks, int bi, long[] values, int vi, int iterations) {\n")
-  f.write("      assert bi + iterations * blocks() <= blocks.length;\n")
-  f.write("      assert vi + iterations * values() <= values.length;\n")
-  f.write("      for (int i = 0; i < iterations; ++i) {\n")
-  for i in xrange(0, values):
-    block_offset = i * bpv / 64
-    bit_offset = (i * bpv) % 64
-    if bit_offset == 0:
-      # start of block
-      f.write("        blocks[bi++] = (values[vi++] << %d)" %(64 - bpv))
-    elif bit_offset + bpv == 64:
-      # end of block
-      f.write(" | values[vi++];\n")
-    elif bit_offset + bpv < 64:
-      # inside a block
-      f.write(" | (values[vi++] << %d)" %(64 - bit_offset - bpv))
-    else:
-      # value spans across 2 blocks
-      right_bits = bit_offset + bpv - 64
-      f.write(" | (values[vi] >>> %d);\n" %right_bits)
-      f.write("        blocks[bi++] = (values[vi++] << %d)" %(64 - right_bits))
-  f.write("      }\n")
-  f.write("    }\n")
+    @Override
+    public void decode(byte[] blocks, int blocksOffset, int[] values, int valuesOffset, int iterations) {
+      throw new UnsupportedOperationException();
+    }
 
+    @Override
+    public void decode(byte[] blocks, int blocksOffset, long[] values, int valuesOffset, int iterations) {
+      LongBuffer.wrap(values, valuesOffset, iterations * valueCount()).put(ByteBuffer.wrap(blocks, blocksOffset, 8 * iterations * blockCount()).asLongBuffer());
+    }
+""")
+  else:
+    p64_decode(bpv, f, 32)
+    p64_decode(bpv, f, 64)
+
+def p64_decode(bpv, f, bits):
+  blocks, values = block_value_count(bpv)
+  typ = get_type(bits)
+  cast_start, cast_end = casts(typ)
+
+  f.write("  @Override\n")
+  f.write("  public void decode(long[] blocks, int blocksOffset, %s[] values, int valuesOffset, int iterations) {\n" %typ)
+  if bits < bpv:
+    f.write("    throw new UnsupportedOperationException();\n")
+  else:
+    f.write("    for (int i = 0; i < iterations; ++i) {\n")
+    mask = (1 << bpv) - 1
+
+    if is_power_of_two(bpv):
+      f.write("      final long block = blocks[blocksOffset++];\n")
+      f.write("      for (int shift = %d; shift >= 0; shift -= %d) {\n" %(64 - bpv, bpv))
+      f.write("        values[valuesOffset++] = %s(block >>> shift) & %d%s;\n" %(cast_start, mask, cast_end))
+      f.write("      }\n") 
+    else:
+      for i in xrange(0, values):
+        block_offset = i * bpv / 64
+        bit_offset = (i * bpv) % 64
+        if bit_offset == 0:
+          # start of block
+          f.write("      final long block%d = blocks[blocksOffset++];\n" %block_offset);
+          f.write("      values[valuesOffset++] = %sblock%d >>> %d%s;\n" %(cast_start, block_offset, 64 - bpv, cast_end))
+        elif bit_offset + bpv == 64:
+          # end of block
+          f.write("      values[valuesOffset++] = %sblock%d & %dL%s;\n" %(cast_start, block_offset, mask, cast_end))
+        elif bit_offset + bpv < 64:
+          # middle of block
+          f.write("      values[valuesOffset++] = %s(block%d >>> %d) & %dL%s;\n" %(cast_start, block_offset, 64 - bit_offset - bpv, mask, cast_end))
+        else:
+          # value spans across 2 blocks
+          mask1 = (1 << (64 - bit_offset)) -1
+          shift1 = bit_offset + bpv - 64
+          shift2 = 64 - shift1
+          f.write("      final long block%d = blocks[blocksOffset++];\n" %(block_offset + 1));
+          f.write("      values[valuesOffset++] = %s((block%d & %dL) << %d) | (block%d >>> %d)%s;\n" %(cast_start, block_offset, mask1, shift1, block_offset + 1, shift2, cast_end))
+    f.write("    }\n")
   f.write("  }\n\n")
 
+  byte_blocks, byte_values = block_value_count(bpv, 8)
 
+  f.write("  @Override\n")
+  f.write("  public void decode(byte[] blocks, int blocksOffset, %s[] values, int valuesOffset, int iterations) {\n" %typ)
+  if bits < bpv:
+    f.write("    throw new UnsupportedOperationException();\n")
+  else:
+
+    if is_power_of_two(bpv) and bpv < 8:
+      f.write("    for (int j = 0; j < 8 * iterations; ++j) {\n")
+      f.write("      final byte block = blocks[blocksOffset++];\n")
+      for shift in xrange(8 - bpv, 0, -bpv):
+        f.write("      values[valuesOffset++] = (block >>> %d) & %d;\n" %(shift, mask))
+      f.write("      values[valuesOffset++] = block & %d;\n" %mask)
+      f.write("    }\n")
+    elif bpv == 8:
+      f.write("    for (int j = 0; j < 8 * iterations; ++j) {\n")
+      f.write("      values[valuesOffset++] = blocks[blocksOffset++] & 0xFF;\n")
+      f.write("    }\n")
+    elif is_power_of_two(bpv) and bpv > 8:
+      f.write("    for (int j = 0; j < %d * iterations; ++j) {\n" %(64 / bpv))
+      m = bits <= 32 and "0xFF" or "0xFFL"
+      f.write("      values[valuesOffset++] =")
+      for i in xrange(bpv / 8 - 1):
+        f.write(" ((blocks[blocksOffset++] & %s) << %d) |" %(m, bpv - 8))
+      f.write(" (blocks[blocksOffset++] & %s);\n" %m)
+      f.write("    }\n")
+    else:
+      f.write("    for (int i = 0; i < 8 * iterations; ++i) {\n")
+      for i in xrange(0, byte_values):
+        byte_start = i * bpv / 8
+        bit_start = (i * bpv) % 8
+        byte_end = ((i + 1) * bpv - 1) / 8
+        bit_end = ((i + 1) * bpv - 1) % 8
+        shift = lambda b: 8 * (byte_end - b - 1) + 1 + bit_end
+        if bit_start == 0:
+          f.write("      final %s byte%d = blocks[blocksOffset++] & 0xFF;\n" %(typ, byte_start))
+        for b in xrange(byte_start + 1, byte_end + 1):
+          f.write("      final %s byte%d = blocks[blocksOffset++] & 0xFF;\n" %(typ, b))
+        f.write("      values[valuesOffset++] =")
+        if byte_start == byte_end:
+          if bit_start == 0:
+            if bit_end == 7:
+              f.write(" byte%d" %byte_start)
+            else:
+              f.write(" byte%d >>> %d" %(byte_start, 7 - bit_end))
+          else:
+            if bit_end == 7:
+              f.write(" byte%d & %d" %(byte_start, 2 ** (8 - bit_start) - 1))
+            else:
+              f.write(" (byte%d >>> %d) & %d" %(byte_start, 7 - bit_end, 2 ** (bit_end - bit_start + 1) - 1))
+        else:
+          if bit_start == 0:
+            f.write(" (byte%d << %d)" %(byte_start, shift(byte_start)))
+          else:
+            f.write(" ((byte%d & %d) << %d)" %(byte_start, 2 ** (8 - bit_start) - 1, shift(byte_start)))
+          for b in xrange(byte_start + 1, byte_end):
+            f.write(" | (byte%d << %d)" %(b, shift(b)))
+          if bit_end == 7:
+            f.write(" | byte%d" %byte_end)
+          else:
+            f.write(" | (byte%d >>> %d)" %(byte_end, 7 - bit_end))
+        f.write(";\n")
+      f.write("    }\n")
+  f.write("  }\n\n")
 
 if __name__ == '__main__':
-  p64_bpv = [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 12, 16, 21, 32]
   f = open(OUTPUT_FILE, 'w')
   f.write(HEADER)
-  f.write("  static {\n")
-  f.write("    BULK_OPERATIONS.put(PackedInts.Format.PACKED, new BulkOperation[65]);")
+  f.write('\n')
+  f.write('''/**
+ * Efficient sequential read/write of packed integers.
+ */\n''')
+
+  f.write('abstract class BulkOperation implements PackedInts.Decoder, PackedInts.Encoder {\n')
+  f.write('  private static final BulkOperation[] packedBulkOps = new BulkOperation[] {\n')
+    
   for bpv in xrange(1, 65):
-    f.write("    BULK_OPERATIONS.get(PackedInts.Format.PACKED)[%d] = new Packed64BulkOperation%d();\n" %(bpv, bpv))
-  f.write("    BULK_OPERATIONS.put(PackedInts.Format.PACKED_SINGLE_BLOCK, new BulkOperation[65]);\n")
-  for bpv in PACKED_64_SINGLE_BLOCK_BPV:
-    f.write("    BULK_OPERATIONS.get(PackedInts.Format.PACKED_SINGLE_BLOCK)[%d] = new Packed64SingleBlockBulkOperation%d();\n" %(bpv, bpv))
+    if bpv > MAX_SPECIALIZED_BITS_PER_VALUE:
+      f.write('    new BulkOperationPacked(%d),\n' % bpv)
+      continue
+    f2 = open('BulkOperationPacked%d.java' % bpv, 'w')
+    f2.write(HEADER)
+    if bpv == 64:
+      f2.write('import java.nio.LongBuffer;\n')
+      f2.write('import java.nio.ByteBuffer;\n')
+      f2.write('\n')
+    f2.write('''/**
+ * Efficient sequential read/write of packed integers.
+ */\n''')
+    f2.write('final class BulkOperationPacked%d extends BulkOperationPacked {\n' % bpv)
+    packed64(bpv, f2)
+    f2.write('}\n')
+    f2.close()
+    f.write('    new BulkOperationPacked%d(),\n' % bpv)
+    
+  f.write('  };\n')
+  f.write('\n')
+    
+  f.write('  // NOTE: this is sparse (some entries are null):\n')
+  f.write('  private static final BulkOperation[] packedSingleBlockBulkOps = new BulkOperation[] {\n')
+  for bpv in xrange(1, max(PACKED_64_SINGLE_BLOCK_BPV)+1):
+    if bpv in PACKED_64_SINGLE_BLOCK_BPV:
+      f.write('    new BulkOperationPackedSingleBlock(%d),\n' % bpv)
+    else:
+      f.write('    null,\n')
+  f.write('  };\n')
+  f.write('\n')
+      
+  f.write("\n")
+  f.write("  public static BulkOperation of(PackedInts.Format format, int bitsPerValue) {\n")
+  f.write("    switch (format) {\n")
+
+  f.write("    case PACKED:\n")
+  f.write("      assert packedBulkOps[bitsPerValue - 1] != null;\n")
+  f.write("      return packedBulkOps[bitsPerValue - 1];\n")
+  f.write("    case PACKED_SINGLE_BLOCK:\n")
+  f.write("      assert packedSingleBlockBulkOps[bitsPerValue - 1] != null;\n")
+  f.write("      return packedSingleBlockBulkOps[bitsPerValue - 1];\n")
+  f.write("    default:\n")
+  f.write("      throw new AssertionError();\n")
+  f.write("    }\n")
   f.write("  }\n")
-  for bpv in xrange(1, 65):
-    packed64(bpv, f)
-  for bpv in PACKED_64_SINGLE_BLOCK_BPV:
-    packed64singleblock(bpv,f)
   f.write(FOOTER)
   f.close()

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/gen_Packed64SingleBlock.py
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/gen_Packed64SingleBlock.py?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/gen_Packed64SingleBlock.py (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/packed/gen_Packed64SingleBlock.py Fri Sep 21 17:21:34 2012
@@ -105,12 +105,12 @@ abstract class Packed64SingleBlock exten
 
     // bulk get
     assert index %% valuesPerBlock == 0;
-    final BulkOperation op = BulkOperation.of(PackedInts.Format.PACKED_SINGLE_BLOCK, bitsPerValue);
-    assert op.blocks() == 1;
-    assert op.values() == valuesPerBlock;
+    final PackedInts.Decoder decoder = BulkOperation.of(PackedInts.Format.PACKED_SINGLE_BLOCK, bitsPerValue);
+    assert decoder.blockCount() == 1;
+    assert decoder.valueCount() == valuesPerBlock;
     final int blockIndex = index / valuesPerBlock;
     final int nblocks = (index + len) / valuesPerBlock - blockIndex;
-    op.get(blocks, blockIndex, arr, off, nblocks);
+    decoder.decode(blocks, blockIndex, arr, off, nblocks);
     final int diff = nblocks * valuesPerBlock;
     index += diff; len -= diff;
 
@@ -150,11 +150,11 @@ abstract class Packed64SingleBlock exten
     // bulk set
     assert index %% valuesPerBlock == 0;
     final BulkOperation op = BulkOperation.of(PackedInts.Format.PACKED_SINGLE_BLOCK, bitsPerValue);
-    assert op.blocks() == 1;
-    assert op.values() == valuesPerBlock;
+    assert op.blockCount() == 1;
+    assert op.valueCount() == valuesPerBlock;
     final int blockIndex = index / valuesPerBlock;
     final int nblocks = (index + len) / valuesPerBlock - blockIndex;
-    op.set(blocks, blockIndex, arr, off, nblocks);
+    op.encode(arr, off, blocks, blockIndex, nblocks);
     final int diff = nblocks * valuesPerBlock;
     index += diff; len -= diff;
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/overview.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/overview.html?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/overview.html (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/overview.html Fri Sep 21 17:21:34 2012
@@ -130,7 +130,7 @@ adding
 
 <li>
 Create an {@link org.apache.lucene.index.IndexWriter IndexWriter}
-and add documents to it with {@link org.apache.lucene.index.IndexWriter#addDocument(Iterable) addDocument()};</li>
+and add documents to it with {@link org.apache.lucene.index.IndexWriter#addDocument(org.apache.lucene.index.IndexDocument) addDocument()};</li>
 
 <li>
 Call <a href="../queryparser/org/apache/lucene/queryparser/classic/QueryParserBase.html#parse(java.lang.String)">QueryParser.parse()</a>
@@ -152,11 +152,11 @@ index for all the files contained in a d
 queries and searches an index.</li>
 </ul>
 To demonstrate these, try something like:
-<blockquote><tt>> <b>java -cp lucene-core.jar:lucene-demo.jar:lucene-analyzers-common.jar org.apache.lucene.demo.IndexFiles rec.food.recipes/soups</b></tt>
+<blockquote><tt>> <b>java -cp lucene-core.jar:lucene-demo.jar:lucene-analyzers-common.jar org.apache.lucene.demo.IndexFiles -index index -docs rec.food.recipes/soups</b></tt>
 <br><tt>adding rec.food.recipes/soups/abalone-chowder</tt>
 <br><tt>&nbsp; </tt>[ ... ]
 
-<p><tt>> <b>java -cp lucene-core.jar:lucene-demo.jar:lucene-analyzers-common.jar org.apache.lucene.demo.SearchFiles</b></tt>
+<p><tt>> <b>java -cp lucene-core.jar:lucene-demo.jar:lucene-queryparser.jar:lucene-analyzers-common.jar org.apache.lucene.demo.SearchFiles</b></tt>
 <br><tt>Query: <b>chowder</b></tt>
 <br><tt>Searching for: chowder</tt>
 <br><tt>34 total matching documents</tt>

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec Fri Sep 21 17:21:34 2012
@@ -14,5 +14,3 @@
 #  limitations under the License.
 
 org.apache.lucene.codecs.lucene40.Lucene40Codec
-org.apache.lucene.codecs.simpletext.SimpleTextCodec
-org.apache.lucene.codecs.appending.AppendingCodec

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat Fri Sep 21 17:21:34 2012
@@ -14,8 +14,3 @@
 #  limitations under the License.
 
 org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat
-org.apache.lucene.codecs.pulsing.Pulsing40PostingsFormat
-org.apache.lucene.codecs.simpletext.SimpleTextPostingsFormat
-org.apache.lucene.codecs.memory.MemoryPostingsFormat
-org.apache.lucene.codecs.bloom.BloomFilteringPostingsFormat
-org.apache.lucene.codecs.memory.DirectPostingsFormat

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/TestDemo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/TestDemo.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/TestDemo.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/TestDemo.java Fri Sep 21 17:21:34 2012
@@ -25,6 +25,7 @@ import org.apache.lucene.document.Docume
 import org.apache.lucene.document.Field;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.StoredDocument;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.search.*;
@@ -64,7 +65,7 @@ public class TestDemo extends LuceneTest
     assertEquals(1, hits.totalHits);
     // Iterate through the results:
     for (int i = 0; i < hits.scoreDocs.length; i++) {
-      Document hitDoc = isearcher.doc(hits.scoreDocs[i].doc);
+      StoredDocument hitDoc = isearcher.doc(hits.scoreDocs[i].doc);
       assertEquals(text, hitDoc.get("fieldname"));
     }
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/TestSearch.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/TestSearch.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/TestSearch.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/TestSearch.java Fri Sep 21 17:21:34 2012
@@ -34,6 +34,49 @@ import org.apache.lucene.search.*;
 /** JUnit adaptation of an older test case SearchTest. */
 public class TestSearch extends LuceneTestCase {
 
+  public void testNegativeQueryBoost() throws Exception {
+    Query q = new TermQuery(new Term("foo", "bar"));
+    q.setBoost(-42f);
+    assertEquals(-42f, q.getBoost(), 0.0f);
+
+    Directory directory = newDirectory();
+    try {
+      Analyzer analyzer = new MockAnalyzer(random());
+      IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
+      
+      IndexWriter writer = new IndexWriter(directory, conf);
+      try {
+        Document d = new Document();
+        d.add(newTextField("foo", "bar", Field.Store.YES));
+        writer.addDocument(d);
+      } finally {
+        writer.close();
+      }
+      
+      IndexReader reader = DirectoryReader.open(directory);
+      try {
+        IndexSearcher searcher = new IndexSearcher(reader);
+        
+        ScoreDoc[] hits = searcher.search(q, null, 1000).scoreDocs;
+        assertEquals(1, hits.length);
+        assertTrue("score is not negative: " + hits[0].score,
+                   hits[0].score < 0);
+
+        Explanation explain = searcher.explain(q, hits[0].doc);
+        assertEquals("score doesn't match explanation",
+                     hits[0].score, explain.getValue(), 0.001f);
+        assertTrue("explain doesn't think doc is a match",
+                   explain.isMatch());
+
+      } finally {
+        reader.close();
+      }
+    } finally {
+      directory.close();
+    }
+
+  }
+
     /** This test performs a number of searches. It also compares output
      *  of searches using multi-file index segments with single-file
      *  index segments.
@@ -110,7 +153,7 @@ public class TestSearch extends LuceneTe
 
         out.println(hits.length + " total results");
         for (int i = 0 ; i < hits.length && i < 10; i++) {
-          Document d = searcher.doc(hits[i].doc);
+          StoredDocument d = searcher.doc(hits[i].doc);
           out.println(i + " " + hits[i].score + " " + d.get("contents"));
         }
       }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/TestSearchForDuplicates.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/TestSearchForDuplicates.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/TestSearchForDuplicates.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/TestSearchForDuplicates.java Fri Sep 21 17:21:34 2012
@@ -127,7 +127,7 @@ public class TestSearchForDuplicates ext
     out.println(hits.length + " total results\n");
     for (int i = 0 ; i < hits.length; i++) {
       if ( i < 10 || (i > 94 && i < 105) ) {
-        Document d = searcher.doc(hits[i].doc);
+        StoredDocument d = searcher.doc(hits[i].doc);
         out.println(i + " " + d.get(ID_FIELD));
       }
     }
@@ -137,7 +137,7 @@ public class TestSearchForDuplicates ext
     assertEquals("total results", expectedCount, hits.length);
     for (int i = 0 ; i < hits.length; i++) {
       if (i < 10 || (i > 94 && i < 105) ) {
-        Document d = searcher.doc(hits[i].doc);
+        StoredDocument d = searcher.doc(hits[i].doc);
         assertEquals("check " + i, String.valueOf(i), d.get(ID_FIELD));
       }
     }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/analysis/TestCharFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/analysis/TestCharFilter.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/analysis/TestCharFilter.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/analysis/TestCharFilter.java Fri Sep 21 17:21:34 2012
@@ -17,6 +17,7 @@ package org.apache.lucene.analysis;
  * limitations under the License.
  */
 
+import java.io.IOException;
 import java.io.Reader;
 import java.io.StringReader;
 
@@ -51,6 +52,11 @@ public class TestCharFilter extends Luce
     }
 
     @Override
+    public int read(char[] cbuf, int off, int len) throws IOException {
+      return input.read(cbuf, off, len);
+    }
+
+    @Override
     protected int correct(int currentOff) {
       return currentOff + 1;
     }
@@ -61,6 +67,11 @@ public class TestCharFilter extends Luce
     protected CharFilter2(Reader in) {
       super(in);
     }
+    
+    @Override
+    public int read(char[] cbuf, int off, int len) throws IOException {
+      return input.read(cbuf, off, len);
+    }
 
     @Override
     protected int correct(int currentOff) {

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestReuseDocsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestReuseDocsEnum.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestReuseDocsEnum.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestReuseDocsEnum.java Fri Sep 21 17:21:34 2012
@@ -23,9 +23,9 @@ import java.util.Random;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.index.AtomicReader;
+import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.DocsEnum;
-import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
@@ -50,7 +50,8 @@ public class TestReuseDocsEnum extends L
     writer.commit();
 
     DirectoryReader open = DirectoryReader.open(dir);
-    for (AtomicReader indexReader : open.getSequentialSubReaders()) {
+    for (AtomicReaderContext ctx : open.leaves()) {
+      AtomicReader indexReader = ctx.reader();
       Terms terms = indexReader.terms("body");
       TermsEnum iterator = terms.iterator(null);
       IdentityHashMap<DocsEnum, Boolean> enums = new IdentityHashMap<DocsEnum, Boolean>();
@@ -76,8 +77,8 @@ public class TestReuseDocsEnum extends L
     writer.commit();
 
     DirectoryReader open = DirectoryReader.open(dir);
-    for (AtomicReader indexReader : open.getSequentialSubReaders()) {
-      Terms terms = indexReader.terms("body");
+    for (AtomicReaderContext ctx : open.leaves()) {
+      Terms terms = ctx.reader().terms("body");
       TermsEnum iterator = terms.iterator(null);
       IdentityHashMap<DocsEnum, Boolean> enums = new IdentityHashMap<DocsEnum, Boolean>();
       MatchNoBits bits = new Bits.MatchNoBits(open.maxDoc());
@@ -121,11 +122,11 @@ public class TestReuseDocsEnum extends L
 
     DirectoryReader firstReader = DirectoryReader.open(dir);
     DirectoryReader secondReader = DirectoryReader.open(dir);
-    List<? extends AtomicReader> sequentialSubReaders = firstReader.getSequentialSubReaders();
-    List<? extends AtomicReader> sequentialSubReaders2 = secondReader.getSequentialSubReaders();
+    List<AtomicReaderContext> leaves = firstReader.leaves();
+    List<AtomicReaderContext> leaves2 = secondReader.leaves();
     
-    for (IndexReader indexReader : sequentialSubReaders) {
-      Terms terms = ((AtomicReader) indexReader).terms("body");
+    for (AtomicReaderContext ctx : leaves) {
+      Terms terms = ctx.reader().terms("body");
       TermsEnum iterator = terms.iterator(null);
       IdentityHashMap<DocsEnum, Boolean> enums = new IdentityHashMap<DocsEnum, Boolean>();
       MatchNoBits bits = new Bits.MatchNoBits(firstReader.maxDoc());
@@ -133,7 +134,7 @@ public class TestReuseDocsEnum extends L
       DocsEnum docs = null;
       BytesRef term = null;
       while ((term = iterator.next()) != null) {
-        docs = iterator.docs(null, randomDocsEnum("body", term, sequentialSubReaders2, bits), random().nextBoolean() ? DocsEnum.FLAG_FREQS : 0);
+        docs = iterator.docs(null, randomDocsEnum("body", term, leaves2, bits), random().nextBoolean() ? DocsEnum.FLAG_FREQS : 0);
         enums.put(docs, true);
       }
       assertEquals(terms.size(), enums.size());
@@ -142,7 +143,7 @@ public class TestReuseDocsEnum extends L
       enums.clear();
       docs = null;
       while ((term = iterator.next()) != null) {
-        docs = iterator.docs(bits, randomDocsEnum("body", term, sequentialSubReaders2, bits), random().nextBoolean() ? DocsEnum.FLAG_FREQS : 0);
+        docs = iterator.docs(bits, randomDocsEnum("body", term, leaves2, bits), random().nextBoolean() ? DocsEnum.FLAG_FREQS : 0);
         enums.put(docs, true);
       }
       assertEquals(terms.size(), enums.size());
@@ -150,12 +151,20 @@ public class TestReuseDocsEnum extends L
     IOUtils.close(writer, firstReader, secondReader, dir);
   }
   
-  public DocsEnum randomDocsEnum(String field, BytesRef term, List<? extends AtomicReader> readers, Bits bits) throws IOException {
+  public DocsEnum randomDocsEnum(String field, BytesRef term, List<AtomicReaderContext> readers, Bits bits) throws IOException {
     if (random().nextInt(10) == 0) {
       return null;
     }
-    AtomicReader indexReader = (AtomicReader) readers.get(random().nextInt(readers.size()));
-    return indexReader.termDocsEnum(bits, field, term, random().nextBoolean() ? DocsEnum.FLAG_FREQS : 0);
+    AtomicReader indexReader = readers.get(random().nextInt(readers.size())).reader();
+    Terms terms = indexReader.terms(field);
+    if (terms == null) {
+      return null;
+    }
+    TermsEnum iterator = terms.iterator(null);
+    if (iterator.seekExact(term, true)) {
+      return iterator.docs(bits, null, random().nextBoolean() ? DocsEnum.FLAG_FREQS : 0);
+    }
+    return null;
   }
 
   /**

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/codecs/lucene40/values/TestDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/codecs/lucene40/values/TestDocValues.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/codecs/lucene40/values/TestDocValues.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/codecs/lucene40/values/TestDocValues.java Fri Sep 21 17:21:34 2012
@@ -28,12 +28,14 @@ import org.apache.lucene.codecs.DocValue
 import org.apache.lucene.codecs.lucene40.values.Bytes;
 import org.apache.lucene.codecs.lucene40.values.Floats;
 import org.apache.lucene.codecs.lucene40.values.Ints;
+import org.apache.lucene.document.FieldType;
 import org.apache.lucene.index.DocValues.SortedSource;
 import org.apache.lucene.index.DocValues.Source;
 import org.apache.lucene.index.DocValues.Type;
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.IndexableFieldType;
+import org.apache.lucene.index.StorableField;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.Counter;
@@ -438,22 +440,12 @@ public class TestDocValues extends Lucen
     return getSource(values).asSortedSource();
   }
   
-  public static class DocValueHolder implements IndexableField {
+  public static class DocValueHolder implements StorableField {
     BytesRef bytes;
     Number numberValue;
     Comparator<BytesRef> comp;
 
     @Override
-    public TokenStream tokenStream(Analyzer a) {
-      return null;
-    }
-
-    @Override
-    public float boost() {
-      return 0.0f;
-    }
-
-    @Override
     public String name() {
       return "test";
     }
@@ -479,7 +471,7 @@ public class TestDocValues extends Lucen
     }
 
     @Override
-    public IndexableFieldType fieldType() {
+    public FieldType fieldType() {
       return null;
     }
   }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/document/TestBinaryDocument.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/document/TestBinaryDocument.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/document/TestBinaryDocument.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/document/TestBinaryDocument.java Fri Sep 21 17:21:34 2012
@@ -3,6 +3,7 @@ package org.apache.lucene.document;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.StoredDocument;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
@@ -37,8 +38,8 @@ public class TestBinaryDocument extends 
   {
     FieldType ft = new FieldType();
     ft.setStored(true);
-    IndexableField binaryFldStored = new StoredField("binaryStored", binaryValStored.getBytes("UTF-8"));
-    IndexableField stringFldStored = new Field("stringStored", binaryValStored, ft);
+    StoredField binaryFldStored = new StoredField("binaryStored", binaryValStored.getBytes("UTF-8"));
+    Field stringFldStored = new Field("stringStored", binaryValStored, ft);
 
     Document doc = new Document();
     
@@ -56,7 +57,7 @@ public class TestBinaryDocument extends 
     
     /** open a reader and fetch the document */ 
     IndexReader reader = writer.getReader();
-    Document docFromReader = reader.document(0);
+    StoredDocument docFromReader = reader.document(0);
     assertTrue(docFromReader != null);
     
     /** fetch the binary stored field and compare it's content with the original one */
@@ -75,8 +76,8 @@ public class TestBinaryDocument extends 
   }
   
   public void testCompressionTools() throws Exception {
-    IndexableField binaryFldCompressed = new StoredField("binaryCompressed", CompressionTools.compress(binaryValCompressed.getBytes("UTF-8")));
-    IndexableField stringFldCompressed = new StoredField("stringCompressed", CompressionTools.compressString(binaryValCompressed));
+    StoredField binaryFldCompressed = new StoredField("binaryCompressed", CompressionTools.compress(binaryValCompressed.getBytes("UTF-8")));
+    StoredField stringFldCompressed = new StoredField("stringCompressed", CompressionTools.compressString(binaryValCompressed));
     
     Document doc = new Document();
     
@@ -90,7 +91,7 @@ public class TestBinaryDocument extends 
     
     /** open a reader and fetch the document */ 
     IndexReader reader = writer.getReader();
-    Document docFromReader = reader.document(0);
+    StoredDocument docFromReader = reader.document(0);
     assertTrue(docFromReader != null);
     
     /** fetch the binary compressed field and compare it's content with the original one */

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/document/TestDocument.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/document/TestDocument.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/document/TestDocument.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/document/TestDocument.java Fri Sep 21 17:21:34 2012
@@ -18,6 +18,7 @@ package org.apache.lucene.document;
  */
 
 import java.io.StringReader;
+import java.util.List;
 
 import org.apache.lucene.analysis.EmptyTokenizer;
 import org.apache.lucene.analysis.MockAnalyzer;
@@ -27,6 +28,8 @@ import org.apache.lucene.index.IndexWrit
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.StorableField;
+import org.apache.lucene.index.StoredDocument;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
@@ -50,9 +53,9 @@ public class TestDocument extends Lucene
     
     FieldType ft = new FieldType();
     ft.setStored(true);
-    IndexableField stringFld = new Field("string", binaryVal, ft);
-    IndexableField binaryFld = new StoredField("binary", binaryVal.getBytes("UTF-8"));
-    IndexableField binaryFld2 = new StoredField("binary", binaryVal2.getBytes("UTF-8"));
+    Field stringFld = new Field("string", binaryVal, ft);
+    StoredField binaryFld = new StoredField("binary", binaryVal.getBytes("UTF-8"));
+    StoredField binaryFld2 = new StoredField("binary", binaryVal2.getBytes("UTF-8"));
     
     doc.add(stringFld);
     doc.add(binaryFld);
@@ -124,7 +127,7 @@ public class TestDocument extends Lucene
                                        // siltenlty ignored
     assertEquals(0, doc.getFields().size());
   }
-  
+
   public void testConstructorExceptions() {
     FieldType ft = new FieldType();
     ft.setStored(true);
@@ -147,6 +150,34 @@ public class TestDocument extends Lucene
       // expected exception
     }
   }
+
+  public void testClearDocument() {
+    Document doc = makeDocumentWithFields();
+    assertEquals(8, doc.getFields().size());
+    doc.clear();
+    assertEquals(0, doc.getFields().size());
+  }
+
+  public void testGetFieldsImmutable() {
+    Document doc = makeDocumentWithFields();
+    assertEquals(8, doc.getFields().size());
+    List<Field> fields = doc.getFields();
+    try {
+      fields.add( new StringField("name", "value", Field.Store.NO) );
+      fail("Document.getFields() should return immutable List");
+    }
+    catch (UnsupportedOperationException e) {
+      // OK
+    }
+
+    try {
+      fields.clear();
+      fail("Document.getFields() should return immutable List");
+    }
+    catch (UnsupportedOperationException e) {
+      // OK
+    }
+  }
   
   /**
    * Tests {@link Document#getValues(String)} method for a brand new Document
@@ -179,7 +210,7 @@ public class TestDocument extends Lucene
     ScoreDoc[] hits = searcher.search(query, null, 1000).scoreDocs;
     assertEquals(1, hits.length);
     
-    doAssert(searcher.doc(hits[0].doc), true);
+    doAssert(searcher.doc(hits[0].doc));
     writer.close();
     reader.close();
     dir.close();
@@ -214,11 +245,14 @@ public class TestDocument extends Lucene
     return doc;
   }
   
+  private void doAssert(StoredDocument doc) {
+    doAssert(new Document(doc), true);
+  }
   private void doAssert(Document doc, boolean fromIndex) {
-    IndexableField[] keywordFieldValues = doc.getFields("keyword");
-    IndexableField[] textFieldValues = doc.getFields("text");
-    IndexableField[] unindexedFieldValues = doc.getFields("unindexed");
-    IndexableField[] unstoredFieldValues = doc.getFields("unstored");
+    StorableField[] keywordFieldValues = doc.getFields("keyword");
+    StorableField[] textFieldValues = doc.getFields("text");
+    StorableField[] unindexedFieldValues = doc.getFields("unindexed");
+    StorableField[] unstoredFieldValues = doc.getFields("unstored");
     
     assertTrue(keywordFieldValues.length == 2);
     assertTrue(textFieldValues.length == 2);
@@ -268,7 +302,7 @@ public class TestDocument extends Lucene
     assertEquals(3, hits.length);
     int result = 0;
     for (int i = 0; i < 3; i++) {
-      Document doc2 = searcher.doc(hits[i].doc);
+      StoredDocument doc2 = searcher.doc(hits[i].doc);
       Field f = (Field) doc2.getField("id");
       if (f.stringValue().equals("id1")) result |= 1;
       else if (f.stringValue().equals("id2")) result |= 2;
@@ -290,4 +324,25 @@ public class TestDocument extends Lucene
       // expected
     }
   }
+  
+  public void testNumericFieldAsString() throws Exception {
+    Document doc = new Document();
+    doc.add(new IntField("int", 5, Field.Store.YES));
+    assertEquals("5", doc.get("int"));
+    assertNull(doc.get("somethingElse"));
+    doc.add(new IntField("int", 4, Field.Store.YES));
+    assertArrayEquals(new String[] { "5", "4" }, doc.getValues("int"));
+    
+    Directory dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    iw.addDocument(doc);
+    DirectoryReader ir = iw.getReader();
+    StoredDocument sdoc = ir.document(0);
+    assertEquals("5", sdoc.get("int"));
+    assertNull(sdoc.get("somethingElse"));
+    assertArrayEquals(new String[] { "5", "4" }, sdoc.getValues("int"));
+    ir.close();
+    iw.close();
+    dir.close();
+  }
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/document/TestField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/document/TestField.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/document/TestField.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/document/TestField.java Fri Sep 21 17:21:34 2012
@@ -18,9 +18,11 @@ 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;
 
@@ -610,4 +612,40 @@ public class TestField extends LuceneTes
       // expected
     }
   }
+  
+  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/LUCENE-2878/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java Fri Sep 21 17:21:34 2012
@@ -25,22 +25,9 @@ import java.util.List;
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.DocValuesFormat;
-import org.apache.lucene.codecs.FieldInfosFormat;
-import org.apache.lucene.codecs.LiveDocsFormat;
-import org.apache.lucene.codecs.NormsFormat;
+import org.apache.lucene.codecs.FilterCodec;
 import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.SegmentInfoFormat;
-import org.apache.lucene.codecs.StoredFieldsFormat;
-import org.apache.lucene.codecs.TermVectorsFormat;
 import org.apache.lucene.codecs.lucene40.Lucene40Codec;
-import org.apache.lucene.codecs.lucene40.Lucene40DocValuesFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40FieldInfosFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40NormsFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40SegmentInfoFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat;
 import org.apache.lucene.codecs.pulsing.Pulsing40PostingsFormat;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
@@ -1106,10 +1093,11 @@ public class TestAddIndexes extends Luce
     IndexReader[] readers = new IndexReader[] { DirectoryReader.open(dirs[0]), DirectoryReader.open(dirs[1]) };
     
     Directory dir = new MockDirectoryWrapper(random(), new RAMDirectory());
-    IndexWriterConfig conf = new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())).setMergePolicy(newLogMergePolicy());
+    IndexWriterConfig conf = new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())).setMergePolicy(newLogMergePolicy(true));
     LogMergePolicy lmp = (LogMergePolicy) conf.getMergePolicy();
-    lmp.setUseCompoundFile(true);
-    lmp.setNoCFSRatio(1.0); // Force creation of CFS
+    // Force creation of CFS:
+    lmp.setNoCFSRatio(1.0);
+    lmp.setMaxCFSSegmentSizeMB(Double.POSITIVE_INFINITY);
     IndexWriter w3 = new IndexWriter(dir, conf);
     w3.addIndexes(readers);
     w3.close();
@@ -1119,49 +1107,14 @@ public class TestAddIndexes extends Luce
     dir.close();
   }
   
-  private static class UnRegisteredCodec extends Codec {
+  private static final class UnRegisteredCodec extends FilterCodec {
     public UnRegisteredCodec() {
       super("NotRegistered");
     }
 
     @Override
-    public PostingsFormat postingsFormat() {
-      return PostingsFormat.forName("Lucene40");
-    }
-
-    @Override
-    public DocValuesFormat docValuesFormat() {
-      return new Lucene40DocValuesFormat();
-    }
-
-    @Override
-    public StoredFieldsFormat storedFieldsFormat() {
-      return new Lucene40StoredFieldsFormat();
-    }
-    
-    @Override
-    public TermVectorsFormat termVectorsFormat() {
-      return new Lucene40TermVectorsFormat();
-    }
-    
-    @Override
-    public FieldInfosFormat fieldInfosFormat() {
-      return new Lucene40FieldInfosFormat();
-    }
-
-    @Override
-    public SegmentInfoFormat segmentInfoFormat() {
-      return new Lucene40SegmentInfoFormat();
-    }
-
-    @Override
-    public NormsFormat normsFormat() {
-      return new Lucene40NormsFormat();
-    }
-    
-    @Override
-    public LiveDocsFormat liveDocsFormat() {
-      return new Lucene40LiveDocsFormat();
+    protected Codec delegate() {
+      return Codec.forName("Lucene40");
     }
   }
   
@@ -1246,7 +1199,7 @@ public class TestAddIndexes extends Luce
     w.close();
     assertEquals(2, r3.numDocs());
     for(int docID=0;docID<2;docID++) {
-      Document d = r3.document(docID);
+      StoredDocument d = r3.document(docID);
       if (d.get("id").equals("1")) {
         assertEquals("doc1 field1", d.get("f1"));
       } else {

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java Fri Sep 21 17:21:34 2012
@@ -329,13 +329,13 @@ public class TestBackwardsCompatibility 
 
     for(int i=0;i<35;i++) {
       if (liveDocs.get(i)) {
-        Document d = reader.document(i);
-        List<IndexableField> fields = d.getFields();
+        StoredDocument d = reader.document(i);
+        List<StorableField> fields = d.getFields();
         boolean isProxDoc = d.getField("content3") == null;
         if (isProxDoc) {
           final int numFields = is40Index ? 7 : 5;
           assertEquals(numFields, fields.size());
-          IndexableField f =  d.getField("id");
+          StorableField f =  d.getField("id");
           assertEquals(""+i, f.stringValue());
 
           f = d.getField("utf8");
@@ -406,7 +406,7 @@ public class TestBackwardsCompatibility 
     ScoreDoc[] hits = searcher.search(new TermQuery(new Term(new String("content"), "aaa")), null, 1000).scoreDocs;
 
     // First document should be #0
-    Document d = searcher.getIndexReader().document(hits[0].doc);
+    StoredDocument d = searcher.getIndexReader().document(hits[0].doc);
     assertEquals("didn't get the right document first", "0", d.get("id"));
 
     doTestHits(hits, 34, searcher.getIndexReader());
@@ -459,7 +459,7 @@ public class TestBackwardsCompatibility 
     IndexReader reader = DirectoryReader.open(dir);
     IndexSearcher searcher = new IndexSearcher(reader);
     ScoreDoc[] hits = searcher.search(new TermQuery(new Term("content", "aaa")), null, 1000).scoreDocs;
-    Document d = searcher.getIndexReader().document(hits[0].doc);
+    StoredDocument d = searcher.getIndexReader().document(hits[0].doc);
     assertEquals("wrong first document", "0", d.get("id"));
     doTestHits(hits, 44, searcher.getIndexReader());
     reader.close();
@@ -485,7 +485,7 @@ public class TestBackwardsCompatibility 
     IndexSearcher searcher = new IndexSearcher(reader);
     ScoreDoc[] hits = searcher.search(new TermQuery(new Term("content", "aaa")), null, 1000).scoreDocs;
     assertEquals("wrong number of hits", 34, hits.length);
-    Document d = searcher.doc(hits[0].doc);
+    StoredDocument d = searcher.doc(hits[0].doc);
     assertEquals("wrong first document", "0", d.get("id"));
     reader.close();
 
@@ -510,6 +510,7 @@ public class TestBackwardsCompatibility 
     LogByteSizeMergePolicy mp = new LogByteSizeMergePolicy();
     mp.setUseCompoundFile(doCFS);
     mp.setNoCFSRatio(1.0);
+    mp.setMaxCFSSegmentSizeMB(Double.POSITIVE_INFINITY);
     // TODO: remove randomness
     IndexWriterConfig conf = new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()))
       .setMaxBufferedDocs(10).setMergePolicy(mp);
@@ -561,7 +562,10 @@ public class TestBackwardsCompatibility 
       Directory dir = newFSDirectory(outputDir);
 
       LogMergePolicy mergePolicy = newLogMergePolicy(true, 10);
-      mergePolicy.setNoCFSRatio(1); // This test expects all of its segments to be in CFS
+      
+      // This test expects all of its segments to be in CFS:
+      mergePolicy.setNoCFSRatio(1.0); 
+      mergePolicy.setMaxCFSSegmentSizeMB(Double.POSITIVE_INFINITY);
 
       IndexWriter writer = new IndexWriter(
           dir,
@@ -753,7 +757,7 @@ public class TestBackwardsCompatibility 
       for (int id=10; id<15; id++) {
         ScoreDoc[] hits = searcher.search(NumericRangeQuery.newIntRange("trieInt", 4, Integer.valueOf(id), Integer.valueOf(id), true, true), 100).scoreDocs;
         assertEquals("wrong number of hits", 1, hits.length);
-        Document d = searcher.doc(hits[0].doc);
+        StoredDocument d = searcher.doc(hits[0].doc);
         assertEquals(String.valueOf(id), d.get("id"));
         
         hits = searcher.search(NumericRangeQuery.newLongRange("trieLong", 4, Long.valueOf(id), Long.valueOf(id), true, true), 100).scoreDocs;

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/index/TestCompoundFile.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/index/TestCompoundFile.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/index/TestCompoundFile.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/index/TestCompoundFile.java Fri Sep 21 17:21:34 2012
@@ -350,7 +350,7 @@ public class TestCompoundFile extends Lu
 
         IndexInput one = cr.openInput("f11", newIOContext(random()));
 
-        IndexInput two = (IndexInput) one.clone();
+        IndexInput two = one.clone();
 
         assertSameStreams("basic clone one", expected, one);
         expected.seek(0);
@@ -476,8 +476,8 @@ public class TestCompoundFile extends Lu
         IndexInput e1 = cr.openInput("f11", newIOContext(random()));
         IndexInput e2 = cr.openInput("f3", newIOContext(random()));
 
-        IndexInput a1 = (IndexInput) e1.clone();
-        IndexInput a2 = (IndexInput) e2.clone();
+        IndexInput a1 = e1.clone();
+        IndexInput a2 = e2.clone();
 
         // Seek the first pair
         e1.seek(100);

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/index/TestCrashCausesCorruptIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/index/TestCrashCausesCorruptIndex.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/index/TestCrashCausesCorruptIndex.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/index/TestCrashCausesCorruptIndex.java Fri Sep 21 17:21:34 2012
@@ -168,17 +168,11 @@ public class TestCrashCausesCorruptIndex
       this.crashAfterCreateOutput = name;
     }
         
-    /**
-     * {@inheritDoc}
-     */
     @Override
     public void close() throws IOException {
       realDirectory.close();
     }
 
-    /**
-     * {@inheritDoc}
-     */
     @Override
     public IndexOutput createOutput(String name, IOContext cxt) throws IOException {
       IndexOutput indexOutput = realDirectory.createOutput(name, cxt);
@@ -194,49 +188,31 @@ public class TestCrashCausesCorruptIndex
       return indexOutput;
     }
 
-    /**
-     * {@inheritDoc}
-     */
     @Override
     public void deleteFile(String name) throws IOException {
       realDirectory.deleteFile(name);
     }
 
-    /**
-     * {@inheritDoc}
-     */
     @Override
     public boolean fileExists(String name) throws IOException {
       return realDirectory.fileExists(name);
     }
 
-    /**
-     * {@inheritDoc}
-     */
     @Override
     public long fileLength(String name) throws IOException {
       return realDirectory.fileLength(name);
     }
 
-    /**
-     * {@inheritDoc}
-     */
     @Override
     public String[] listAll() throws IOException {
       return realDirectory.listAll();
     }
 
-    /**
-     * {@inheritDoc}
-     */
     @Override
     public IndexInput openInput(String name, IOContext cxt) throws IOException {
       return realDirectory.openInput(name, cxt);
     }
 
-    /**
-     * {@inheritDoc}
-     */
     @Override
     public void sync(Collection<String> names) throws IOException {
       realDirectory.sync(names);

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/index/TestCustomNorms.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/index/TestCustomNorms.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/index/TestCustomNorms.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/index/TestCustomNorms.java Fri Sep 21 17:21:34 2012
@@ -74,7 +74,7 @@ public class TestCustomNorms extends Luc
     assertEquals(Type.FLOAT_32, normValues.getType());
     float[] norms = (float[]) source.getArray();
     for (int i = 0; i < open.maxDoc(); i++) {
-      Document document = open.document(i);
+      StoredDocument document = open.document(i);
       float expected = Float.parseFloat(document.get(floatTestField));
       assertEquals(expected, norms[i], 0.0f);
     }
@@ -143,7 +143,7 @@ public class TestCustomNorms extends Luc
     IndexReader reader = writer.getReader();
     writer.close();
     assertEquals(numAdded, reader.numDocs());
-    IndexReaderContext topReaderContext = reader.getTopReaderContext();
+    IndexReaderContext topReaderContext = reader.getContext();
     for (final AtomicReaderContext ctx : topReaderContext.leaves()) {
       AtomicReader atomicReader = ctx.reader();
       Source source = random().nextBoolean() ? atomicReader.normValues("foo").getSource() : atomicReader.normValues("foo").getDirectSource();