You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2013/02/08 04:26:21 UTC

svn commit: r1443834 [7/16] - in /lucene/dev/branches/branch_4x: ./ dev-tools/ lucene/ lucene/analysis/ lucene/analysis/icu/src/java/org/apache/lucene/collation/ lucene/analysis/icu/src/test/org/apache/lucene/collation/ lucene/backwards/ lucene/benchma...

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java Fri Feb  8 03:26:14 2013
@@ -69,7 +69,7 @@ public final class BytesRefHash {
   public BytesRefHash() { 
     this(new ByteBlockPool(new DirectAllocator()));
   }
-
+  
   /**
    * Creates a new {@link BytesRefHash}
    */
@@ -118,7 +118,8 @@ public final class BytesRefHash {
   public BytesRef get(int ord, BytesRef ref) {
     assert bytesStart != null : "bytesStart is null - not initialized";
     assert ord < bytesStart.length: "ord exceeds byteStart len: " + bytesStart.length;
-    return pool.setBytesRef(ref, bytesStart[ord]);
+    pool.setBytesRef(ref, bytesStart[ord]);
+    return ref;
   }
 
   /**
@@ -129,7 +130,7 @@ public final class BytesRefHash {
    * order to reuse this {@link BytesRefHash} instance.
    * </p>
    */
-  public int[] compact() {
+  int[] compact() {
     assert bytesStart != null : "Bytesstart is null - not initialized";
     int upto = 0;
     for (int i = 0; i < hashSize; i++) {
@@ -171,8 +172,9 @@ public final class BytesRefHash {
       protected int compare(int i, int j) {
         final int ord1 = compact[i], ord2 = compact[j];
         assert bytesStart.length > ord1 && bytesStart.length > ord2;
-        return comp.compare(pool.setBytesRef(scratch1, bytesStart[ord1]),
-          pool.setBytesRef(scratch2, bytesStart[ord2]));
+        pool.setBytesRef(scratch1, bytesStart[ord1]);
+        pool.setBytesRef(scratch2, bytesStart[ord2]);
+        return comp.compare(scratch1, scratch2);
       }
 
       @Override
@@ -186,8 +188,8 @@ public final class BytesRefHash {
       protected int comparePivot(int j) {
         final int ord = compact[j];
         assert bytesStart.length > ord;
-        return comp.compare(pivot,
-          pool.setBytesRef(scratch2, bytesStart[ord]));
+        pool.setBytesRef(scratch2, bytesStart[ord]);
+        return comp.compare(pivot, scratch2);
       }
       
       private final BytesRef pivot = new BytesRef(),
@@ -197,7 +199,8 @@ public final class BytesRefHash {
   }
 
   private boolean equals(int ord, BytesRef b) {
-    return pool.setBytesRef(scratch1, bytesStart[ord]).bytesEquals(b);
+    pool.setBytesRef(scratch1, bytesStart[ord]);
+    return scratch1.bytesEquals(b);
   }
 
   private boolean shrink(int targetSize) {
@@ -208,8 +211,7 @@ public final class BytesRefHash {
       newSize /= 2;
     }
     if (newSize != hashSize) {
-      bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT
-          * -(hashSize - newSize));
+      bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT * -(hashSize - newSize));
       hashSize = newSize;
       ords = new int[hashSize];
       Arrays.fill(ords, -1);
@@ -248,8 +250,7 @@ public final class BytesRefHash {
   public void close() {
     clear(true);
     ords = null;
-    bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT
-        * -hashSize);
+    bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT * -hashSize);
   }
 
   /**
@@ -533,50 +534,6 @@ public final class BytesRefHash {
      */
     public abstract Counter bytesUsed();
   }
-  
-  /** A simple {@link BytesStartArray} that tracks all
-   *  memory allocation using a shared {@link Counter}
-   *  instance.  */
-  public static class TrackingDirectBytesStartArray extends BytesStartArray {
-    protected final int initSize;
-    private int[] bytesStart;
-    protected final Counter bytesUsed;
-    
-    public TrackingDirectBytesStartArray(int initSize, Counter bytesUsed) {
-      this.initSize = initSize;
-      this.bytesUsed = bytesUsed;
-    }
-
-    @Override
-    public int[] clear() {
-      if (bytesStart != null) {
-        bytesUsed.addAndGet(-bytesStart.length * RamUsageEstimator.NUM_BYTES_INT);
-      }
-      return bytesStart = null;
-    }
-
-    @Override
-    public int[] grow() {
-      assert bytesStart != null;
-      final int oldSize = bytesStart.length;
-      bytesStart = ArrayUtil.grow(bytesStart, bytesStart.length + 1);
-      bytesUsed.addAndGet((bytesStart.length - oldSize) * RamUsageEstimator.NUM_BYTES_INT);
-      return bytesStart;
-    }
-
-    @Override
-    public int[] init() {
-      bytesStart = new int[ArrayUtil.oversize(initSize,
-          RamUsageEstimator.NUM_BYTES_INT)];
-      bytesUsed.addAndGet((bytesStart.length) * RamUsageEstimator.NUM_BYTES_INT);
-      return bytesStart;
-    }
-
-    @Override
-    public Counter bytesUsed() {
-      return bytesUsed;
-    }
-  }
 
   /** A simple {@link BytesStartArray} that tracks
    *  memory allocation using a private {@link AtomicLong}
@@ -590,9 +547,13 @@ public final class BytesRefHash {
     private int[] bytesStart;
     private final Counter bytesUsed;
     
+    public DirectBytesStartArray(int initSize, Counter counter) {
+      this.bytesUsed = counter;
+      this.initSize = initSize;      
+    }
+    
     public DirectBytesStartArray(int initSize) {
-      this.bytesUsed = Counter.newCounter();
-      this.initSize = initSize;
+      this(initSize, Counter.newCounter());
     }
 
     @Override

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/CharsRef.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/CharsRef.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/CharsRef.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/CharsRef.java Fri Feb  8 03:26:14 2013
@@ -55,13 +55,10 @@ public final class CharsRef implements C
    * length
    */
   public CharsRef(char[] chars, int offset, int length) {
-    assert chars != null;
-    assert offset >= 0;
-    assert length >= 0;
-    assert chars.length >= offset + length;
     this.chars = chars;
     this.offset = offset;
     this.length = length;
+    assert isValid();
   }
 
   /**
@@ -292,4 +289,33 @@ public final class CharsRef implements C
     clone.copyChars(other);
     return clone;
   }
+  
+  /** 
+   * Performs internal consistency checks.
+   * Always returns true (or throws IllegalStateException) 
+   */
+  public boolean isValid() {
+    if (chars == null) {
+      throw new IllegalStateException("chars is null");
+    }
+    if (length < 0) {
+      throw new IllegalStateException("length is negative: " + length);
+    }
+    if (length > chars.length) {
+      throw new IllegalStateException("length is out of bounds: " + length + ",chars.length=" + chars.length);
+    }
+    if (offset < 0) {
+      throw new IllegalStateException("offset is negative: " + offset);
+    }
+    if (offset > chars.length) {
+      throw new IllegalStateException("offset out of bounds: " + offset + ",chars.length=" + chars.length);
+    }
+    if (offset + length < 0) {
+      throw new IllegalStateException("offset+length is negative: offset=" + offset + ",length=" + length);
+    }
+    if (offset + length > chars.length) {
+      throw new IllegalStateException("offset+length out of bounds: offset=" + offset + ",length=" + length + ",chars.length=" + chars.length);
+    }
+    return true;
+  }
 }
\ No newline at end of file

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/IntsRef.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/IntsRef.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/IntsRef.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/IntsRef.java Fri Feb  8 03:26:14 2013
@@ -50,13 +50,10 @@ public final class IntsRef implements Co
    * ints should not be null.
    */
   public IntsRef(int[] ints, int offset, int length) {
-    assert ints != null;
-    assert offset >= 0;
-    assert length >= 0;
-    assert ints.length >= offset + length;
     this.ints = ints;
     this.offset = offset;
     this.length = length;
+    assert isValid();
   }
 
   @Override
@@ -176,4 +173,33 @@ public final class IntsRef implements Co
     clone.copyInts(other);
     return clone;
   }
+  
+  /** 
+   * Performs internal consistency checks.
+   * Always returns true (or throws IllegalStateException) 
+   */
+  public boolean isValid() {
+    if (ints == null) {
+      throw new IllegalStateException("ints is null");
+    }
+    if (length < 0) {
+      throw new IllegalStateException("length is negative: " + length);
+    }
+    if (length > ints.length) {
+      throw new IllegalStateException("length is out of bounds: " + length + ",ints.length=" + ints.length);
+    }
+    if (offset < 0) {
+      throw new IllegalStateException("offset is negative: " + offset);
+    }
+    if (offset > ints.length) {
+      throw new IllegalStateException("offset out of bounds: " + offset + ",ints.length=" + ints.length);
+    }
+    if (offset + length < 0) {
+      throw new IllegalStateException("offset+length is negative: offset=" + offset + ",length=" + length);
+    }
+    if (offset + length > ints.length) {
+      throw new IllegalStateException("offset+length out of bounds: offset=" + offset + ",length=" + length + ",ints.length=" + ints.length);
+    }
+    return true;
+  }
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/LongsRef.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/LongsRef.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/LongsRef.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/LongsRef.java Fri Feb  8 03:26:14 2013
@@ -49,13 +49,10 @@ public final class LongsRef implements C
   /** This instance will directly reference longs w/o making a copy.
    * longs should not be null */
   public LongsRef(long[] longs, int offset, int length) {
-    assert longs != null;
-    assert offset >= 0;
-    assert length >= 0;
-    assert longs.length >= offset + length;
     this.longs = longs;
     this.offset = offset;
     this.length = length;
+    assert isValid();
   }
 
   @Override
@@ -175,4 +172,33 @@ public final class LongsRef implements C
     clone.copyLongs(other);
     return clone;
   }
+  
+  /** 
+   * Performs internal consistency checks.
+   * Always returns true (or throws IllegalStateException) 
+   */
+  public boolean isValid() {
+    if (longs == null) {
+      throw new IllegalStateException("longs is null");
+    }
+    if (length < 0) {
+      throw new IllegalStateException("length is negative: " + length);
+    }
+    if (length > longs.length) {
+      throw new IllegalStateException("length is out of bounds: " + length + ",longs.length=" + longs.length);
+    }
+    if (offset < 0) {
+      throw new IllegalStateException("offset is negative: " + offset);
+    }
+    if (offset > longs.length) {
+      throw new IllegalStateException("offset out of bounds: " + offset + ",longs.length=" + longs.length);
+    }
+    if (offset + length < 0) {
+      throw new IllegalStateException("offset+length is negative: offset=" + offset + ",length=" + length);
+    }
+    if (offset + length > longs.length) {
+      throw new IllegalStateException("offset+length out of bounds: offset=" + offset + ",length=" + length + ",longs.length=" + longs.length);
+    }
+    return true;
+  }
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/PagedBytes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/PagedBytes.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/PagedBytes.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/PagedBytes.java Fri Feb  8 03:26:14 2013
@@ -32,6 +32,8 @@ import org.apache.lucene.store.IndexInpu
  *
  * @lucene.internal
  **/
+// TODO: refactor this, byteblockpool, fst.bytestore, and any
+// other "shift/mask big arrays". there are too many of these classes!
 public final class PagedBytes {
   private final List<byte[]> blocks = new ArrayList<byte[]>();
   private final List<Integer> blockEnd = new ArrayList<Integer>();
@@ -56,7 +58,7 @@ public final class PagedBytes {
     private final int blockMask;
     private final int blockSize;
 
-    public Reader(PagedBytes pagedBytes) {
+    private Reader(PagedBytes pagedBytes) {
       blocks = new byte[pagedBytes.blocks.size()][];
       for(int i=0;i<blocks.length;i++) {
         blocks[i] = pagedBytes.blocks.get(i);
@@ -79,7 +81,7 @@ public final class PagedBytes {
      * </p>
      * @lucene.internal 
      **/
-    public BytesRef fillSlice(BytesRef b, long start, int length) {
+    public void fillSlice(BytesRef b, long start, int length) {
       assert length >= 0: "length=" + length;
       assert length <= blockSize+1;
       final int index = (int) (start >> blockBits);
@@ -96,7 +98,6 @@ public final class PagedBytes {
         System.arraycopy(blocks[index], offset, b.bytes, 0, blockSize-offset);
         System.arraycopy(blocks[1+index], 0, b.bytes, blockSize-offset, length-(blockSize-offset));
       }
-      return b;
     }
     
     /**
@@ -106,67 +107,10 @@ public final class PagedBytes {
      * borders.
      * </p>
      * 
-     * @return the given {@link BytesRef}
-     * 
-     * @lucene.internal
-     **/
-    public BytesRef fill(BytesRef b, long start) {
-      final int index = (int) (start >> blockBits);
-      final int offset = (int) (start & blockMask);
-      final byte[] block = b.bytes = blocks[index];
-
-      if ((block[offset] & 128) == 0) {
-        b.length = block[offset];
-        b.offset = offset+1;
-      } else {
-        b.length = ((block[offset] & 0x7f) << 8) | (block[1+offset] & 0xff);
-        b.offset = offset+2;
-        assert b.length > 0;
-      }
-      return b;
-    }
-
-    /**
-     * Reads length as 1 or 2 byte vInt prefix, starting at <i>start</i>. *
-     * <p>
-     * <b>Note:</b> this method does not support slices spanning across block
-     * borders.
-     * </p>
-     * 
-     * @return the internal block number of the slice.
-     * @lucene.internal
-     **/
-    public int fillAndGetIndex(BytesRef b, long start) {
-      final int index = (int) (start >> blockBits);
-      final int offset = (int) (start & blockMask);
-      final byte[] block = b.bytes = blocks[index];
-
-      if ((block[offset] & 128) == 0) {
-        b.length = block[offset];
-        b.offset = offset+1;
-      } else {
-        b.length = ((block[offset] & 0x7f) << 8) | (block[1+offset] & 0xff);
-        b.offset = offset+2;
-        assert b.length > 0;
-      }
-      return index;
-    }
-
-    /**
-     * Reads length as 1 or 2 byte vInt prefix, starting at <i>start</i> and
-     * returns the start offset of the next part, suitable as start parameter on
-     * next call to sequentially read all {@link BytesRef}.
-     * 
-     * <p>
-     * <b>Note:</b> this method does not support slices spanning across block
-     * borders.
-     * </p>
-     * 
-     * @return the start offset of the next part, suitable as start parameter on
-     *         next call to sequentially read all {@link BytesRef}.
      * @lucene.internal
      **/
-    public long fillAndGetStart(BytesRef b, long start) {
+    // TODO: this really needs to be refactored into fieldcacheimpl
+    public void fill(BytesRef b, long start) {
       final int index = (int) (start >> blockBits);
       final int offset = (int) (start & blockMask);
       final byte[] block = b.bytes = blocks[index];
@@ -174,82 +118,11 @@ public final class PagedBytes {
       if ((block[offset] & 128) == 0) {
         b.length = block[offset];
         b.offset = offset+1;
-        start += 1L + b.length;
       } else {
         b.length = ((block[offset] & 0x7f) << 8) | (block[1+offset] & 0xff);
         b.offset = offset+2;
-        start += 2L + b.length;
         assert b.length > 0;
       }
-      return start;
-    }
-    
-  
-    /**
-     * Gets a slice out of {@link PagedBytes} starting at <i>start</i>, the
-     * length is read as 1 or 2 byte vInt prefix. Iff the slice spans across a
-     * block border this method will allocate sufficient resources and copy the
-     * paged data.
-     * <p>
-     * Slices spanning more than one block are not supported.
-     * </p>
-     * 
-     * @lucene.internal
-     **/
-    public BytesRef fillSliceWithPrefix(BytesRef b, long start) {
-      int index = (int) (start >> blockBits);
-      int offset = (int) (start & blockMask);
-      byte[] block = blocks[index];
-      final int length;
-      assert offset <= block.length-1;
-      if ((block[offset] & 128) == 0) {
-        length = block[offset];
-        offset = offset+1;
-      } else {
-        if (offset==block.length-1) {
-          final byte[] nextBlock = blocks[++index];
-          length = ((block[offset] & 0x7f) << 8) | (nextBlock[0] & 0xff);
-          offset = 1;
-          block = nextBlock;
-          assert length > 0; 
-        } else {
-          assert offset < block.length-1;
-          length = ((block[offset] & 0x7f) << 8) | (block[1+offset] & 0xff);
-          offset = offset+2;
-          assert length > 0;
-        }
-      }
-      assert length >= 0: "length=" + length;
-      b.length = length;
-
-      // NOTE: even though copyUsingLengthPrefix always
-      // allocs a new block if the byte[] to be added won't
-      // fit in current block,
-      // VarDerefBytesImpl.finishInternal does its own
-      // prefix + byte[] writing which can span two blocks,
-      // so we support that here on decode:
-      if (blockSize - offset >= length) {
-        // Within block
-        b.offset = offset;
-        b.bytes = blocks[index];
-      } else {
-        // Split
-        b.bytes = new byte[length];
-        b.offset = 0;
-        System.arraycopy(blocks[index], offset, b.bytes, 0, blockSize-offset);
-        System.arraycopy(blocks[1+index], 0, b.bytes, blockSize-offset, length-(blockSize-offset));
-      }
-      return b;
-    }
-
-    /** @lucene.internal */
-    public byte[][] getBlocks() {
-      return blocks;
-    }
-
-    /** @lucene.internal */
-    public int[] getBlockEnds() {
-      return blockEnds;
     }
   }
 
@@ -288,34 +161,6 @@ public final class PagedBytes {
     }
   }
 
-  /** Copy BytesRef in */
-  public void copy(BytesRef bytes) {
-    int byteCount = bytes.length;
-    int bytesUpto = bytes.offset;
-    while (byteCount > 0) {
-      int left = blockSize - upto;
-      if (left == 0) {
-        if (currentBlock != null) {
-          blocks.add(currentBlock);
-          blockEnd.add(upto);          
-        }
-        currentBlock = new byte[blockSize];
-        upto = 0;
-        left = blockSize;
-      }
-      if (left < byteCount) {
-        System.arraycopy(bytes.bytes, bytesUpto, currentBlock, upto, left);
-        upto = blockSize;
-        byteCount -= left;
-        bytesUpto += left;
-      } else {
-        System.arraycopy(bytes.bytes, bytesUpto, currentBlock, upto, byteCount);
-        upto += byteCount;
-        break;
-      }
-    }
-  }
-
   /** Copy BytesRef in, setting BytesRef out to the result.
    * Do not use this if you will use freeze(true).
    * This only supports bytes.length <= blockSize */
@@ -362,7 +207,7 @@ public final class PagedBytes {
     blockEnd.add(upto); 
     frozen = true;
     currentBlock = null;
-    return new Reader(this);
+    return new PagedBytes.Reader(this);
   }
 
   public long getPointer() {
@@ -375,6 +220,7 @@ public final class PagedBytes {
 
   /** Copy bytes in, writing the length as a 1 or 2 byte
    *  vInt prefix. */
+  // TODO: this really needs to be refactored into fieldcacheimpl
   public long copyUsingLengthPrefix(BytesRef bytes) {
     if (bytes.length >= 32768) {
       throw new IllegalArgumentException("max length is 32767 (got " + bytes.length + ")");

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/Util.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/Util.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/Util.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/Util.java Fri Feb  8 03:26:14 2013
@@ -109,7 +109,15 @@ public final class Util {
     FST.Arc<Long> scratchArc = new FST.Arc<Long>();
 
     final IntsRef result = new IntsRef();
-
+    
+    return getByOutput(fst, targetOutput, in, arc, scratchArc, result);
+  }
+    
+  /** 
+   * Expert: like {@link Util#getByOutput(FST, long)} except reusing 
+   * BytesReader, initial and scratch Arc, and result.
+   */
+  public static IntsRef getByOutput(FST<Long> fst, long targetOutput, BytesReader in, Arc<Long> arc, Arc<Long> scratchArc, IntsRef result) throws IOException {
     long output = arc.output;
     int upto = 0;
 

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/BlockPackedReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/BlockPackedReader.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/BlockPackedReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/BlockPackedReader.java Fri Feb  8 03:26:14 2013
@@ -17,225 +17,73 @@ package org.apache.lucene.util.packed;
  * limitations under the License.
  */
 
+import static org.apache.lucene.util.packed.BlockPackedReaderIterator.readVLong;
+import static org.apache.lucene.util.packed.BlockPackedReaderIterator.zigZagDecode;
 import static org.apache.lucene.util.packed.BlockPackedWriter.BPV_SHIFT;
 import static org.apache.lucene.util.packed.BlockPackedWriter.MIN_VALUE_EQUALS_0;
 import static org.apache.lucene.util.packed.BlockPackedWriter.checkBlockSize;
 
-import java.io.EOFException;
 import java.io.IOException;
-import java.util.Arrays;
 
-import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.util.LongsRef;
 
 /**
- * Reader for sequences of longs written with {@link BlockPackedWriter}.
- * @see BlockPackedWriter
+ * Provides random access to a stream written with {@link BlockPackedWriter}.
  * @lucene.internal
  */
 public final class BlockPackedReader {
 
-  static long zigZagDecode(long n) {
-    return ((n >>> 1) ^ -(n & 1));
-  }
-
-  // same as DataInput.readVLong but supports negative values
-  static long readVLong(DataInput in) throws IOException {
-    byte b = in.readByte();
-    if (b >= 0) return b;
-    long i = b & 0x7FL;
-    b = in.readByte();
-    i |= (b & 0x7FL) << 7;
-    if (b >= 0) return i;
-    b = in.readByte();
-    i |= (b & 0x7FL) << 14;
-    if (b >= 0) return i;
-    b = in.readByte();
-    i |= (b & 0x7FL) << 21;
-    if (b >= 0) return i;
-    b = in.readByte();
-    i |= (b & 0x7FL) << 28;
-    if (b >= 0) return i;
-    b = in.readByte();
-    i |= (b & 0x7FL) << 35;
-    if (b >= 0) return i;
-    b = in.readByte();
-    i |= (b & 0x7FL) << 42;
-    if (b >= 0) return i;
-    b = in.readByte();
-    i |= (b & 0x7FL) << 49;
-    if (b >= 0) return i;
-    b = in.readByte();
-    i |= (b & 0xFFL) << 56;
-    return i;
-  }
+  private final int blockShift, blockMask;
+  private final long valueCount;
+  private final long[] minValues;
+  private final PackedInts.Reader[] subReaders;
 
-  DataInput in;
-  final int packedIntsVersion;
-  long valueCount;
-  final int blockSize;
-  final long[] values;
-  final LongsRef valuesRef;
-  byte[] blocks;
-  int off;
-  long ord;
-
-  /** Sole constructor.
-   * @param blockSize the number of values of a block, must be equal to the
-   *                  block size of the {@link BlockPackedWriter} which has
-   *                  been used to write the stream
-   */
-  public BlockPackedReader(DataInput in, int packedIntsVersion, int blockSize, long valueCount) {
+  /** Sole constructor. */
+  public BlockPackedReader(IndexInput in, int packedIntsVersion, int blockSize, long valueCount, boolean direct) throws IOException {
     checkBlockSize(blockSize);
-    this.packedIntsVersion = packedIntsVersion;
-    this.blockSize = blockSize;
-    this.values = new long[blockSize];
-    this.valuesRef = new LongsRef(this.values, 0, 0);
-    reset(in, valueCount);
-  }
-
-  /** Reset the current reader to wrap a stream of <code>valueCount</code>
-   * values contained in <code>in</code>. The block size remains unchanged. */
-  public void reset(DataInput in, long valueCount) {
-    this.in = in;
-    assert valueCount >= 0;
     this.valueCount = valueCount;
-    off = blockSize;
-    ord = 0;
-  }
-
-  /** Skip exactly <code>count</code> values. */
-  public void skip(long count) throws IOException {
-    assert count >= 0;
-    if (ord + count > valueCount || ord + count < 0) {
-      throw new EOFException();
-    }
-
-    // 1. skip buffered values
-    final int skipBuffer = (int) Math.min(count, blockSize - off);
-    off += skipBuffer;
-    ord += skipBuffer;
-    count -= skipBuffer;
-    if (count == 0L) {
-      return;
-    }
-
-    // 2. skip as many blocks as necessary
-    assert off == blockSize;
-    while (count >= blockSize) {
+    blockShift = Integer.numberOfTrailingZeros(blockSize);
+    blockMask = blockSize - 1;
+    final int numBlocks = (int) (valueCount / blockSize) + (valueCount % blockSize == 0 ? 0 : 1);
+    if ((long) numBlocks * blockSize < valueCount) {
+      throw new IllegalArgumentException("valueCount is too large for this block size");
+    }
+    long[] minValues = null;
+    subReaders = new PackedInts.Reader[numBlocks];
+    for (int i = 0; i < numBlocks; ++i) {
       final int token = in.readByte() & 0xFF;
       final int bitsPerValue = token >>> BPV_SHIFT;
       if (bitsPerValue > 64) {
         throw new IOException("Corrupted");
       }
       if ((token & MIN_VALUE_EQUALS_0) == 0) {
-        readVLong(in);
-      }
-      final long blockBytes = PackedInts.Format.PACKED.byteCount(packedIntsVersion, blockSize, bitsPerValue);
-      skipBytes(blockBytes);
-      ord += blockSize;
-      count -= blockSize;
-    }
-    if (count == 0L) {
-      return;
-    }
-
-    // 3. skip last values
-    assert count < blockSize;
-    refill();
-    ord += count;
-    off += count;
-  }
-
-  private void skipBytes(long count) throws IOException {
-    if (in instanceof IndexInput) {
-      final IndexInput iin = (IndexInput) in;
-      iin.seek(iin.getFilePointer() + count);
-    } else {
-      if (blocks == null) {
-        blocks = new byte[blockSize];
-      }
-      long skipped = 0;
-      while (skipped < count) {
-        final int toSkip = (int) Math.min(blocks.length, count - skipped);
-        in.readBytes(blocks, 0, toSkip);
-        skipped += toSkip;
-      }
-    }
-  }
-
-  /** Read the next value. */
-  public long next() throws IOException {
-    if (ord == valueCount) {
-      throw new EOFException();
-    }
-    if (off == blockSize) {
-      refill();
-    }
-    final long value = values[off++];
-    ++ord;
-    return value;
-  }
-
-  /** Read between <tt>1</tt> and <code>count</code> values. */
-  public LongsRef next(int count) throws IOException {
-    assert count > 0;
-    if (ord == valueCount) {
-      throw new EOFException();
-    }
-    if (off == blockSize) {
-      refill();
-    }
-
-    count = Math.min(count, blockSize - off);
-    count = (int) Math.min(count, valueCount - ord);
-
-    valuesRef.offset = off;
-    valuesRef.length = count;
-    off += count;
-    ord += count;
-    return valuesRef;
-  }
-
-  private void refill() throws IOException {
-    final int token = in.readByte() & 0xFF;
-    final boolean minEquals0 = (token & MIN_VALUE_EQUALS_0) != 0;
-    final int bitsPerValue = token >>> BPV_SHIFT;
-    if (bitsPerValue > 64) {
-      throw new IOException("Corrupted");
-    }
-    final long minValue = minEquals0 ? 0L : zigZagDecode(1L + readVLong(in));
-    assert minEquals0 || minValue != 0;
-
-    if (bitsPerValue == 0) {
-      Arrays.fill(values, minValue);
-    } else {
-      final PackedInts.Decoder decoder = PackedInts.getDecoder(PackedInts.Format.PACKED, packedIntsVersion, bitsPerValue);
-      final int iterations = blockSize / decoder.byteValueCount();
-      final int blocksSize = iterations * decoder.byteBlockCount();
-      if (blocks == null || blocks.length < blocksSize) {
-        blocks = new byte[blocksSize];
+        if (minValues == null) {
+          minValues = new long[numBlocks];
+        }
+        minValues[i] = zigZagDecode(1L + readVLong(in));
       }
-
-      final int valueCount = (int) Math.min(this.valueCount - ord, blockSize);
-      final int blocksCount = (int) PackedInts.Format.PACKED.byteCount(packedIntsVersion, valueCount, bitsPerValue);
-      in.readBytes(blocks, 0, blocksCount);
-
-      decoder.decode(blocks, 0, values, 0, iterations);
-
-      if (minValue != 0) {
-        for (int i = 0; i < valueCount; ++i) {
-          values[i] += minValue;
+      if (bitsPerValue == 0) {
+        subReaders[i] = new PackedInts.NullReader(blockSize);
+      } else {
+        final int size = (int) Math.min(blockSize, valueCount - (long) i * blockSize);
+        if (direct) {
+          final long pointer = in.getFilePointer();
+          subReaders[i] = PackedInts.getDirectReaderNoHeader(in, PackedInts.Format.PACKED, packedIntsVersion, size, bitsPerValue);
+          in.seek(pointer + PackedInts.Format.PACKED.byteCount(packedIntsVersion, size, bitsPerValue));
+        } else {
+          subReaders[i] = PackedInts.getReaderNoHeader(in, PackedInts.Format.PACKED, packedIntsVersion, size, bitsPerValue);
         }
       }
     }
-    off = 0;
+    this.minValues = minValues;
   }
 
-  /** Return the offset of the next value to read. */
-  public long ord() {
-    return ord;
+  /** Get value at <code>index</code>. */
+  public long get(long index) {
+    assert index >= 0 && index < valueCount;
+    final int block = (int) (index >>> blockShift);
+    final int idx = (int) (index & blockMask);
+    return (minValues == null ? 0 : minValues[block]) + subReaders[block].get(idx);
   }
 
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/BlockPackedWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/BlockPackedWriter.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/BlockPackedWriter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/BlockPackedWriter.java Fri Feb  8 03:26:14 2013
@@ -18,7 +18,6 @@ package org.apache.lucene.util.packed;
  */
 
 import java.io.IOException;
-import java.util.Arrays;
 
 import org.apache.lucene.store.DataOutput;
 
@@ -30,92 +29,43 @@ import org.apache.lucene.store.DataOutpu
  * using as few bits as possible. Memory usage of this class is proportional to
  * the block size. Each block has an overhead between 1 and 10 bytes to store
  * the minimum value and the number of bits per value of the block.
+ * <p>
+ * Format:
+ * <ul>
+ * <li>&lt;BLock&gt;<sup>BlockCount</sup>
+ * <li>BlockCount: &lceil; ValueCount / BlockSize &rceil;
+ * <li>Block: &lt;Header, (Ints)&gt;
+ * <li>Header: &lt;Token, (MinValue)&gt;
+ * <li>Token: a {@link DataOutput#writeByte(byte) byte}, first 7 bits are the
+ *     number of bits per value (<tt>bitsPerValue</tt>). If the 8th bit is 1,
+ *     then MinValue (see next) is <tt>0</tt>, otherwise MinValue and needs to
+ *     be decoded
+ * <li>MinValue: a
+ *     <a href="https://developers.google.com/protocol-buffers/docs/encoding#types">zigzag-encoded</a>
+ *     {@link DataOutput#writeVLong(long) variable-length long} whose value
+ *     should be added to every int from the block to restore the original
+ *     values
+ * <li>Ints: If the number of bits per value is <tt>0</tt>, then there is
+ *     nothing to decode and all ints are equal to MinValue. Otherwise: BlockSize
+ *     {@link PackedInts packed ints} encoded on exactly <tt>bitsPerValue</tt>
+ *     bits per value. They are the subtraction of the original values and
+ *     MinValue
+ * </ul>
+ * @see BlockPackedReaderIterator
  * @see BlockPackedReader
  * @lucene.internal
  */
-public final class BlockPackedWriter {
-
-  static final int MAX_BLOCK_SIZE = 1 << (30 - 3);
-  static final int MIN_VALUE_EQUALS_0 = 1 << 0;
-  static final int BPV_SHIFT = 1;
-
-  static void checkBlockSize(int blockSize) {
-    if (blockSize <= 0 || blockSize > MAX_BLOCK_SIZE) {
-      throw new IllegalArgumentException("blockSize must be > 0 and < " + MAX_BLOCK_SIZE + ", got " + blockSize);
-    }
-    if (blockSize % 64 != 0) {
-      throw new IllegalArgumentException("blockSize must be a multiple of 64, got " + blockSize);
-    }
-  }
-
-  static long zigZagEncode(long n) {
-    return (n >> 63) ^ (n << 1);
-  }
-
-  // same as DataOutput.writeVLong but accepts negative values
-  static void writeVLong(DataOutput out, long i) throws IOException {
-    int k = 0;
-    while ((i & ~0x7FL) != 0L && k++ < 8) {
-      out.writeByte((byte)((i & 0x7FL) | 0x80L));
-      i >>>= 7;
-    }
-    out.writeByte((byte) i);
-  }
-
-  DataOutput out;
-  final long[] values;
-  byte[] blocks;
-  int off;
-  long ord;
-  boolean finished;
+public final class BlockPackedWriter extends AbstractBlockPackedWriter {
 
   /**
    * Sole constructor.
-   * @param blockSize the number of values of a single block, must be a multiple of <tt>64</tt>
+   * @param blockSize the number of values of a single block, must be a power of 2
    */
   public BlockPackedWriter(DataOutput out, int blockSize) {
-    checkBlockSize(blockSize);
-    reset(out);
-    values = new long[blockSize];
-  }
-
-  /** Reset this writer to wrap <code>out</code>. The block size remains unchanged. */
-  public void reset(DataOutput out) {
-    assert out != null;
-    this.out = out;
-    off = 0;
-    ord = 0L;
-    finished = false;
+    super(out, blockSize);
   }
 
-  private void checkNotFinished() {
-    if (finished) {
-      throw new IllegalStateException("Already finished");
-    }
-  }
-
-  /** Append a new long. */
-  public void add(long l) throws IOException {
-    checkNotFinished();
-    if (off == values.length) {
-      flush();
-    }
-    values[off++] = l;
-    ++ord;
-  }
-
-  /** Flush all buffered data to disk. This instance is not usable anymore
-   *  after this method has been called until {@link #reset(DataOutput)} has
-   *  been called. */
-  public void finish() throws IOException {
-    checkNotFinished();
-    if (off > 0) {
-      flush();
-    }
-    finished = true;
-  }
-
-  private void flush() throws IOException {
+  protected void flush() throws IOException {
     assert off > 0;
     long min = Long.MAX_VALUE, max = Long.MIN_VALUE;
     for (int i = 0; i < off; ++i) {
@@ -146,26 +96,10 @@ public final class BlockPackedWriter {
           values[i] -= min;
         }
       }
-      final PackedInts.Encoder encoder = PackedInts.getEncoder(PackedInts.Format.PACKED, PackedInts.VERSION_CURRENT, bitsRequired);
-      final int iterations = values.length / encoder.byteValueCount();
-      final int blockSize = encoder.byteBlockCount() * iterations;
-      if (blocks == null || blocks.length < blockSize) {
-        blocks = new byte[blockSize];
-      }
-      if (off < values.length) {
-        Arrays.fill(values, off, values.length, 0L);
-      }
-      encoder.encode(values, 0, blocks, 0, iterations);
-      final int blockCount = (int) PackedInts.Format.PACKED.byteCount(PackedInts.VERSION_CURRENT, off, bitsRequired);
-      out.writeBytes(blocks, blockCount);
+      writeValues(bitsRequired);
     }
 
     off = 0;
   }
 
-  /** Return the number of values which have been added. */
-  public long ord() {
-    return ord;
-  }
-
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Direct16.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Direct16.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Direct16.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Direct16.java Fri Feb  8 03:26:14 2013
@@ -61,7 +61,11 @@ final class Direct16 extends PackedInts.
 
   @Override
   public long ramBytesUsed() {
-    return RamUsageEstimator.sizeOf(values);
+    return RamUsageEstimator.alignObjectSize(
+        RamUsageEstimator.NUM_BYTES_OBJECT_HEADER
+        + 2 * RamUsageEstimator.NUM_BYTES_INT     // valueCount,bitsPerValue
+        + RamUsageEstimator.NUM_BYTES_OBJECT_REF) // values ref
+        + RamUsageEstimator.sizeOf(values);
   }
 
   @Override

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Direct32.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Direct32.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Direct32.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Direct32.java Fri Feb  8 03:26:14 2013
@@ -61,7 +61,11 @@ final class Direct32 extends PackedInts.
 
   @Override
   public long ramBytesUsed() {
-    return RamUsageEstimator.sizeOf(values);
+    return RamUsageEstimator.alignObjectSize(
+        RamUsageEstimator.NUM_BYTES_OBJECT_HEADER
+        + 2 * RamUsageEstimator.NUM_BYTES_INT     // valueCount,bitsPerValue
+        + RamUsageEstimator.NUM_BYTES_OBJECT_REF) // values ref
+        + RamUsageEstimator.sizeOf(values);
   }
 
   @Override

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Direct64.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Direct64.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Direct64.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Direct64.java Fri Feb  8 03:26:14 2013
@@ -56,7 +56,11 @@ final class Direct64 extends PackedInts.
 
   @Override
   public long ramBytesUsed() {
-    return RamUsageEstimator.sizeOf(values);
+    return RamUsageEstimator.alignObjectSize(
+        RamUsageEstimator.NUM_BYTES_OBJECT_HEADER
+        + 2 * RamUsageEstimator.NUM_BYTES_INT     // valueCount,bitsPerValue
+        + RamUsageEstimator.NUM_BYTES_OBJECT_REF) // values ref
+        + RamUsageEstimator.sizeOf(values);
   }
 
   @Override

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Direct8.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Direct8.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Direct8.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Direct8.java Fri Feb  8 03:26:14 2013
@@ -59,7 +59,11 @@ final class Direct8 extends PackedInts.M
 
   @Override
   public long ramBytesUsed() {
-    return RamUsageEstimator.sizeOf(values);
+    return RamUsageEstimator.alignObjectSize(
+        RamUsageEstimator.NUM_BYTES_OBJECT_HEADER
+        + 2 * RamUsageEstimator.NUM_BYTES_INT     // valueCount,bitsPerValue
+        + RamUsageEstimator.NUM_BYTES_OBJECT_REF) // values ref
+        + RamUsageEstimator.sizeOf(values);
   }
 
   @Override

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/DirectPackedReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/DirectPackedReader.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/DirectPackedReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/DirectPackedReader.java Fri Feb  8 03:26:14 2013
@@ -26,20 +26,10 @@ class DirectPackedReader extends PackedI
   private final IndexInput in;
   private final long startPointer;
 
-  // masks[n-1] masks for bottom n bits
-  private final long[] masks;
-
   public DirectPackedReader(int bitsPerValue, int valueCount, IndexInput in) {
     super(valueCount, bitsPerValue);
     this.in = in;
 
-    long v = 1;
-    masks = new long[bitsPerValue];
-    for (int i = 0; i < bitsPerValue; i++) {
-      v *= 2;
-      masks[i] = v - 1;
-    }
-
     startPointer = in.getFilePointer();
   }
 

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Packed16ThreeBlocks.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Packed16ThreeBlocks.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Packed16ThreeBlocks.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Packed16ThreeBlocks.java Fri Feb  8 03:26:14 2013
@@ -116,7 +116,11 @@ final class Packed16ThreeBlocks extends 
 
   @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

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Packed64.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Packed64.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Packed64.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Packed64.java Fri Feb  8 03:26:14 2013
@@ -244,7 +244,12 @@ class Packed64 extends PackedInts.Mutabl
 
   @Override
   public long ramBytesUsed() {
-    return RamUsageEstimator.sizeOf(blocks);
+    return RamUsageEstimator.alignObjectSize(
+        RamUsageEstimator.NUM_BYTES_OBJECT_HEADER
+        + 3 * RamUsageEstimator.NUM_BYTES_INT     // bpvMinusBlockSize,valueCount,bitsPerValue
+        + RamUsageEstimator.NUM_BYTES_LONG        // maskRight
+        + RamUsageEstimator.NUM_BYTES_OBJECT_REF) // blocks ref
+        + RamUsageEstimator.sizeOf(blocks);
   }
 
   @Override

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Packed64SingleBlock.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Packed64SingleBlock.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Packed64SingleBlock.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Packed64SingleBlock.java Fri Feb  8 03:26:14 2013
@@ -60,7 +60,11 @@ abstract class Packed64SingleBlock exten
 
   @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

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Packed8ThreeBlocks.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Packed8ThreeBlocks.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Packed8ThreeBlocks.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Packed8ThreeBlocks.java Fri Feb  8 03:26:14 2013
@@ -114,7 +114,11 @@ final class Packed8ThreeBlocks extends P
 
   @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

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java Fri Feb  8 03:26:14 2013
@@ -681,6 +681,53 @@ public class PackedInts {
     }
   }
 
+  /** A {@link Reader} which has all its values equal to 0 (bitsPerValue = 0). */
+  public static final class NullReader implements Reader {
+
+    private final int valueCount;
+
+    /** Sole constructor. */
+    public NullReader(int valueCount) {
+      this.valueCount = valueCount;
+    }
+
+    @Override
+    public long get(int index) {
+      return 0;
+    }
+
+    @Override
+    public int get(int index, long[] arr, int off, int len) {
+      return 0;
+    }
+
+    @Override
+    public int getBitsPerValue() {
+      return 0;
+    }
+
+    @Override
+    public int size() {
+      return valueCount;
+    }
+
+    @Override
+    public long ramBytesUsed() {
+      return 0;
+    }
+
+    @Override
+    public Object getArray() {
+      return null;
+    }
+
+    @Override
+    public boolean hasArray() {
+      return false;
+    }
+
+  }
+
   /** A write-once Writer.
    * @lucene.internal
    */
@@ -800,6 +847,22 @@ public class PackedInts {
         throw new AssertionError("Unknown Writer format: " + format);
     }
   }
+  
+  /**
+   * Expert: Restore a {@link Reader} from a stream without reading metadata at
+   * the beginning of the stream. This method is useful to restore data when
+   * metadata has been previously read using {@link #readHeader(DataInput)}.
+   *
+   * @param in           the stream to read data from, positioned at the beginning of the packed values
+   * @param header       metadata result from <code>readHeader()</code>
+   * @return             a Reader
+   * @throws IOException If there is a low-level I/O error
+   * @see #readHeader(DataInput)
+   * @lucene.internal
+   */
+  public static Reader getReaderNoHeader(DataInput in, Header header) throws IOException {
+    return getReaderNoHeader(in, header.format, header.version, header.valueCount, header.bitsPerValue);
+  }
 
   /**
    * Restore a {@link Reader} from a stream.
@@ -912,6 +975,23 @@ public class PackedInts {
         throw new AssertionError("Unknwown format: " + format);
     }
   }
+  
+  /**
+   * Expert: Construct a direct {@link Reader} from an {@link IndexInput} 
+   * without reading metadata at the beginning of the stream. This method is 
+   * useful to restore data when metadata has been previously read using 
+   * {@link #readHeader(DataInput)}.
+   *
+   * @param in           the stream to read data from, positioned at the beginning of the packed values
+   * @param header       metadata result from <code>readHeader()</code>
+   * @return             a Reader
+   * @throws IOException If there is a low-level I/O error
+   * @see #readHeader(DataInput)
+   * @lucene.internal
+   */
+  public static Reader getDirectReaderNoHeader(IndexInput in, Header header) throws IOException {
+    return getDirectReaderNoHeader(in, header.format, header.version, header.valueCount, header.bitsPerValue);
+  }
 
   /**
    * Construct a direct {@link Reader} from an {@link IndexInput}. This method
@@ -1144,5 +1224,41 @@ public class PackedInts {
       }
     }
   }
+  
+  /**
+   * Expert: reads only the metadata from a stream. This is useful to later
+   * restore a stream or open a direct reader via 
+   * {@link #getReaderNoHeader(DataInput, Header)}
+   * or {@link #getDirectReaderNoHeader(IndexInput, Header)}.
+   * @param    in the stream to read data
+   * @return   packed integer metadata.
+   * @throws   IOException If there is a low-level I/O error
+   * @see #getReaderNoHeader(DataInput, Header)
+   * @see #getDirectReaderNoHeader(IndexInput, Header)
+   */
+  public static Header readHeader(DataInput in) throws IOException {
+    final int version = CodecUtil.checkHeader(in, CODEC_NAME, VERSION_START, VERSION_CURRENT);
+    final int bitsPerValue = in.readVInt();
+    assert bitsPerValue > 0 && bitsPerValue <= 64: "bitsPerValue=" + bitsPerValue;
+    final int valueCount = in.readVInt();
+    final Format format = Format.byId(in.readVInt());
+    return new Header(format, valueCount, bitsPerValue, version);
+  }
+  
+  /** Header identifying the structure of a packed integer array. */
+  public static class Header {
+
+    private final Format format;
+    private final int valueCount;
+    private final int bitsPerValue;
+    private final int version;
+
+    public Header(Format format, int valueCount, int bitsPerValue, int version) {
+      this.format = format;
+      this.valueCount = valueCount;
+      this.bitsPerValue = bitsPerValue;
+      this.version = version;
+    }    
+  }
 
 }
\ No newline at end of file

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/PackedWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/PackedWriter.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/PackedWriter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/PackedWriter.java Fri Feb  8 03:26:14 2013
@@ -56,7 +56,7 @@ final class PackedWriter extends PackedI
 
   @Override
   public void add(long v) throws IOException {
-    assert v >= 0 && v <= PackedInts.maxValue(bitsPerValue);
+    assert bitsPerValue == 64 || (v >= 0 && v <= PackedInts.maxValue(bitsPerValue)) : bitsPerValue;
     assert !finished;
     if (valueCount != -1 && written >= valueCount) {
       throw new EOFException("Writing past end of stream");

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/gen_Direct.py
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/gen_Direct.py?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/gen_Direct.py (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/gen_Direct.py Fri Feb  8 03:26:14 2013
@@ -91,8 +91,13 @@ if __name__ == '__main__':
     values[index] = %s(value);
   }
 
+  @Override
   public long ramBytesUsed() {
-    return RamUsageEstimator.sizeOf(values);
+    return RamUsageEstimator.alignObjectSize(
+        RamUsageEstimator.NUM_BYTES_OBJECT_HEADER
+        + 2 * RamUsageEstimator.NUM_BYTES_INT     // valueCount,bitsPerValue
+        + RamUsageEstimator.NUM_BYTES_OBJECT_REF) // values ref
+        + RamUsageEstimator.sizeOf(values);
   }
 
   public void clear() {

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/gen_PackedThreeBlocks.py
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/gen_PackedThreeBlocks.py?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/gen_PackedThreeBlocks.py (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/gen_PackedThreeBlocks.py Fri Feb  8 03:26:14 2013
@@ -146,8 +146,13 @@ if __name__ == '__main__':
     Arrays.fill(blocks, %s0);
   }
 
+  @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

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestLucene40TermVectorsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestLucene40TermVectorsFormat.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestLucene40TermVectorsFormat.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestLucene40TermVectorsFormat.java Fri Feb  8 03:26:14 2013
@@ -24,7 +24,7 @@ public class TestLucene40TermVectorsForm
 
   @Override
   protected Codec getCodec() {
-    return new Lucene40Codec();
+    return new Lucene40RWCodec();
   }
   
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat2.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat2.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat2.java Fri Feb  8 03:26:14 2013
@@ -48,7 +48,6 @@ public class TestBlockPostingsFormat2 ex
     iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
     iwc.setCodec(_TestUtil.alwaysPostingsFormat(new Lucene41PostingsFormat()));
     iw = new RandomIndexWriter(random(), dir, iwc);
-    iw.setAddDocValuesFields(false);
     iw.setDoRandomForceMerge(false); // we will ourselves
   }
   

Copied: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldDocValuesFormat.java (from r1443717, lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldDocValuesFormat.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldDocValuesFormat.java?p2=lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldDocValuesFormat.java&p1=lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldDocValuesFormat.java&r1=1443717&r2=1443834&rev=1443834&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldDocValuesFormat.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldDocValuesFormat.java Fri Feb  8 03:26:14 2013
@@ -38,7 +38,6 @@ import org.apache.lucene.index.IndexWrit
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.RandomCodec;
-import org.apache.lucene.index.StoredDocument;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
@@ -106,7 +105,7 @@ public class TestPerFieldDocValuesFormat
     BytesRef scratch = new BytesRef();
     // Iterate through the results:
     for (int i = 0; i < hits.scoreDocs.length; i++) {
-      StoredDocument hitDoc = isearcher.doc(hits.scoreDocs[i].doc);
+      Document hitDoc = isearcher.doc(hits.scoreDocs[i].doc);
       assertEquals(text, hitDoc.get("fieldname"));
       assert ireader.leaves().size() == 1;
       NumericDocValues dv = ireader.leaves().get(0).reader().getNumericDocValues("dv1");

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/document/TestField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/document/TestField.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/document/TestField.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/document/TestField.java Fri Feb  8 03:26:14 2013
@@ -29,44 +29,6 @@ import org.apache.lucene.util.LuceneTest
 // sanity check some basics of fields
 public class TestField extends LuceneTestCase {
   
-  public void testByteDocValuesField() throws Exception {
-    ByteDocValuesField field = new ByteDocValuesField("foo", (byte) 5);
-
-    trySetBoost(field);
-    field.setByteValue((byte) 6); // ok
-    trySetBytesValue(field);
-    trySetBytesRefValue(field);
-    trySetDoubleValue(field);
-    trySetIntValue(field);
-    trySetFloatValue(field);
-    trySetLongValue(field);
-    trySetReaderValue(field);
-    trySetShortValue(field);
-    trySetStringValue(field);
-    trySetTokenStreamValue(field);
-    
-    assertEquals(6, field.numericValue().byteValue());
-  }
-  
-  public void testDerefBytesDocValuesField() throws Exception {
-    DerefBytesDocValuesField field = new DerefBytesDocValuesField("foo", new BytesRef("bar"));
-
-    trySetBoost(field);
-    trySetByteValue(field);
-    field.setBytesValue("fubar".getBytes("UTF-8"));
-    field.setBytesValue(new BytesRef("baz"));
-    trySetDoubleValue(field);
-    trySetIntValue(field);
-    trySetFloatValue(field);
-    trySetLongValue(field);
-    trySetReaderValue(field);
-    trySetShortValue(field);
-    trySetStringValue(field);
-    trySetTokenStreamValue(field);
-    
-    assertEquals(new BytesRef("baz"), field.binaryValue());
-  }
-  
   public void testDoubleField() throws Exception {
     Field fields[] = new Field[] {
         new DoubleField("foo", 5d, Field.Store.NO),
@@ -107,7 +69,7 @@ public class TestField extends LuceneTes
     trySetStringValue(field);
     trySetTokenStreamValue(field);
     
-    assertEquals(6d, field.numericValue().doubleValue(), 0.0d);
+    assertEquals(6d, Double.longBitsToDouble(field.numericValue().longValue()), 0.0d);
   }
   
   public void testFloatDocValuesField() throws Exception {
@@ -126,7 +88,7 @@ public class TestField extends LuceneTes
     trySetStringValue(field);
     trySetTokenStreamValue(field);
     
-    assertEquals(6f, field.numericValue().floatValue(), 0.0f);
+    assertEquals(6f, Float.intBitsToFloat(field.numericValue().intValue()), 0.0f);
   }
   
   public void testFloatField() throws Exception {
@@ -153,25 +115,6 @@ public class TestField extends LuceneTes
     }
   }
   
-  public void testIntDocValuesField() throws Exception {
-    IntDocValuesField field = new IntDocValuesField("foo", 5);
-
-    trySetBoost(field);
-    trySetByteValue(field);
-    trySetBytesValue(field);
-    trySetBytesRefValue(field);
-    trySetDoubleValue(field);
-    field.setIntValue(6); // ok
-    trySetFloatValue(field);
-    trySetLongValue(field);
-    trySetReaderValue(field);
-    trySetShortValue(field);
-    trySetStringValue(field);
-    trySetTokenStreamValue(field);
-    
-    assertEquals(6, field.numericValue().intValue());
-  }
-  
   public void testIntField() throws Exception {
     Field fields[] = new Field[] {
         new IntField("foo", 5, Field.Store.NO),
@@ -196,8 +139,8 @@ public class TestField extends LuceneTes
     }
   }
   
-  public void testLongDocValuesField() throws Exception {
-    LongDocValuesField field = new LongDocValuesField("foo", 5L);
+  public void testNumericDocValuesField() throws Exception {
+    NumericDocValuesField field = new NumericDocValuesField("foo", 5L);
 
     trySetBoost(field);
     trySetByteValue(field);
@@ -239,46 +182,8 @@ public class TestField extends LuceneTes
     }
   }
   
-  public void testPackedLongDocValuesField() throws Exception {
-    PackedLongDocValuesField field = new PackedLongDocValuesField("foo", 5L);
-
-    trySetBoost(field);
-    trySetByteValue(field);
-    trySetBytesValue(field);
-    trySetBytesRefValue(field);
-    trySetDoubleValue(field);
-    trySetIntValue(field);
-    trySetFloatValue(field);
-    field.setLongValue(6); // ok
-    trySetReaderValue(field);
-    trySetShortValue(field);
-    trySetStringValue(field);
-    trySetTokenStreamValue(field);
-    
-    assertEquals(6L, field.numericValue().longValue());
-  }
-  
-  public void testShortDocValuesField() throws Exception {
-    ShortDocValuesField field = new ShortDocValuesField("foo", (short)5);
-
-    trySetBoost(field);
-    trySetByteValue(field);
-    trySetBytesValue(field);
-    trySetBytesRefValue(field);
-    trySetDoubleValue(field);
-    trySetIntValue(field);
-    trySetFloatValue(field);
-    trySetLongValue(field);
-    trySetReaderValue(field);
-    field.setShortValue((short) 6); // ok
-    trySetStringValue(field);
-    trySetTokenStreamValue(field);
-    
-    assertEquals((short)6, field.numericValue().shortValue());
-  }
-  
   public void testSortedBytesDocValuesField() throws Exception {
-    SortedBytesDocValuesField field = new SortedBytesDocValuesField("foo", new BytesRef("bar"));
+    SortedDocValuesField field = new SortedDocValuesField("foo", new BytesRef("bar"));
 
     trySetBoost(field);
     trySetByteValue(field);
@@ -296,8 +201,8 @@ public class TestField extends LuceneTes
     assertEquals(new BytesRef("baz"), field.binaryValue());
   }
   
-  public void testStraightBytesDocValuesField() throws Exception {
-    StraightBytesDocValuesField field = new StraightBytesDocValuesField("foo", new BytesRef("bar"));
+  public void testBinaryDocValuesField() throws Exception {
+    BinaryDocValuesField field = new BinaryDocValuesField("foo", new BytesRef("bar"));
 
     trySetBoost(field);
     trySetByteValue(field);

Copied: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/Test2BBinaryDocValues.java (from r1443717, lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/Test2BBinaryDocValues.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/Test2BBinaryDocValues.java?p2=lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/Test2BBinaryDocValues.java&p1=lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/Test2BBinaryDocValues.java&r1=1443717&r2=1443834&rev=1443834&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/Test2BBinaryDocValues.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/Test2BBinaryDocValues.java Fri Feb  8 03:26:14 2013
@@ -26,6 +26,7 @@ import org.apache.lucene.store.ByteArray
 import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
 import org.apache.lucene.util.TimeUnits;
 import org.apache.lucene.util._TestUtil;
 import org.junit.Ignore;
@@ -34,6 +35,7 @@ import com.carrotsearch.randomizedtestin
 
 @TimeoutSuite(millis = 80 * TimeUnits.HOUR)
 @Ignore("takes ~ 45 minutes")
+@SuppressCodecs("Lucene3x")
 public class Test2BBinaryDocValues extends LuceneTestCase {
   
   // indexes Integer.MAX_VALUE docs with a fixed binary field

Copied: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/Test2BNumericDocValues.java (from r1443717, lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/Test2BNumericDocValues.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/Test2BNumericDocValues.java?p2=lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/Test2BNumericDocValues.java&p1=lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/Test2BNumericDocValues.java&r1=1443717&r2=1443834&rev=1443834&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/Test2BNumericDocValues.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/Test2BNumericDocValues.java Fri Feb  8 03:26:14 2013
@@ -25,12 +25,14 @@ import org.apache.lucene.store.MockDirec
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TimeUnits;
 import org.apache.lucene.util._TestUtil;
+import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
 import org.junit.Ignore;
 
 import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
 
 @TimeoutSuite(millis = 80 * TimeUnits.HOUR)
 @Ignore("takes ~ 30 minutes")
+@SuppressCodecs("Lucene3x")
 public class Test2BNumericDocValues extends LuceneTestCase {
   
   // indexes Integer.MAX_VALUE docs with an increasing dv field

Copied: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/Test2BSortedDocValues.java (from r1443717, lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/Test2BSortedDocValues.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/Test2BSortedDocValues.java?p2=lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/Test2BSortedDocValues.java&p1=lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/Test2BSortedDocValues.java&r1=1443717&r2=1443834&rev=1443834&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/Test2BSortedDocValues.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/Test2BSortedDocValues.java Fri Feb  8 03:26:14 2013
@@ -28,12 +28,14 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TimeUnits;
 import org.apache.lucene.util._TestUtil;
+import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
 import org.junit.Ignore;
 
 import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
 
 @TimeoutSuite(millis = 80 * TimeUnits.HOUR)
 @Ignore("very slow")
+@SuppressCodecs("Lucene3x")
 public class Test2BSortedDocValues extends LuceneTestCase {
   
   // indexes Integer.MAX_VALUE docs with a fixed binary field

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java Fri Feb  8 03:26:14 2013
@@ -30,24 +30,18 @@ import java.util.Map;
 import java.util.Random;
 
 import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.document.ByteDocValuesField;
-import org.apache.lucene.document.DerefBytesDocValuesField;
+import org.apache.lucene.document.BinaryDocValuesField;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.DoubleDocValuesField;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.document.FloatDocValuesField;
-import org.apache.lucene.document.IntDocValuesField;
 import org.apache.lucene.document.IntField;
-import org.apache.lucene.document.LongDocValuesField;
+import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.document.LongField;
-import org.apache.lucene.document.PackedLongDocValuesField;
-import org.apache.lucene.document.ShortDocValuesField;
-import org.apache.lucene.document.SortedBytesDocValuesField;
-import org.apache.lucene.document.StraightBytesDocValuesField;
+import org.apache.lucene.document.SortedDocValuesField;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.document.TextField;
-import org.apache.lucene.index.DocValues.Source;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 import org.apache.lucene.search.DocIdSetIterator;
@@ -62,12 +56,13 @@ import org.apache.lucene.store.RAMDirect
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.Constants;
+import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util._TestUtil;
-import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 
 /*
   Verify we can read the pre-5.0 file format, do searches
@@ -393,26 +388,26 @@ public class TestBackwardsCompatibility 
         assertEquals(7, i);
       }
     }
-    
+
     if (is40Index) {
       // check docvalues fields
-      Source dvByte = MultiDocValues.getDocValues(reader, "dvByte").getSource();
-      Source dvBytesDerefFixed = MultiDocValues.getDocValues(reader, "dvBytesDerefFixed").getSource();
-      Source dvBytesDerefVar = MultiDocValues.getDocValues(reader, "dvBytesDerefVar").getSource();
-      Source dvBytesSortedFixed = MultiDocValues.getDocValues(reader, "dvBytesSortedFixed").getSource();
-      Source dvBytesSortedVar = MultiDocValues.getDocValues(reader, "dvBytesSortedVar").getSource();
-      Source dvBytesStraightFixed = MultiDocValues.getDocValues(reader, "dvBytesStraightFixed").getSource();
-      Source dvBytesStraightVar = MultiDocValues.getDocValues(reader, "dvBytesStraightVar").getSource();
-      Source dvDouble = MultiDocValues.getDocValues(reader, "dvDouble").getSource();
-      Source dvFloat = MultiDocValues.getDocValues(reader, "dvFloat").getSource();
-      Source dvInt = MultiDocValues.getDocValues(reader, "dvInt").getSource();
-      Source dvLong = MultiDocValues.getDocValues(reader, "dvLong").getSource();
-      Source dvPacked = MultiDocValues.getDocValues(reader, "dvPacked").getSource();
-      Source dvShort = MultiDocValues.getDocValues(reader, "dvShort").getSource();
+      NumericDocValues dvByte = MultiDocValues.getNumericValues(reader, "dvByte");
+      BinaryDocValues dvBytesDerefFixed = MultiDocValues.getBinaryValues(reader, "dvBytesDerefFixed");
+      BinaryDocValues dvBytesDerefVar = MultiDocValues.getBinaryValues(reader, "dvBytesDerefVar");
+      SortedDocValues dvBytesSortedFixed = MultiDocValues.getSortedValues(reader, "dvBytesSortedFixed");
+      SortedDocValues dvBytesSortedVar = MultiDocValues.getSortedValues(reader, "dvBytesSortedVar");
+      BinaryDocValues dvBytesStraightFixed = MultiDocValues.getBinaryValues(reader, "dvBytesStraightFixed");
+      BinaryDocValues dvBytesStraightVar = MultiDocValues.getBinaryValues(reader, "dvBytesStraightVar");
+      NumericDocValues dvDouble = MultiDocValues.getNumericValues(reader, "dvDouble");
+      NumericDocValues dvFloat = MultiDocValues.getNumericValues(reader, "dvFloat");
+      NumericDocValues dvInt = MultiDocValues.getNumericValues(reader, "dvInt");
+      NumericDocValues dvLong = MultiDocValues.getNumericValues(reader, "dvLong");
+      NumericDocValues dvPacked = MultiDocValues.getNumericValues(reader, "dvPacked");
+      NumericDocValues dvShort = MultiDocValues.getNumericValues(reader, "dvShort");
       
       for (int i=0;i<35;i++) {
         int id = Integer.parseInt(reader.document(i).get("id"));
-        assertEquals((byte)id, dvByte.getInt(i));
+        assertEquals(id, dvByte.get(i));
         
         byte bytes[] = new byte[] {
             (byte)(id >>> 24), (byte)(id >>> 16),(byte)(id >>> 8),(byte)id
@@ -420,19 +415,25 @@ public class TestBackwardsCompatibility 
         BytesRef expectedRef = new BytesRef(bytes);
         BytesRef scratch = new BytesRef();
         
-        assertEquals(expectedRef, dvBytesDerefFixed.getBytes(i, scratch));
-        assertEquals(expectedRef, dvBytesDerefVar.getBytes(i, scratch));
-        assertEquals(expectedRef, dvBytesSortedFixed.getBytes(i, scratch));
-        assertEquals(expectedRef, dvBytesSortedVar.getBytes(i, scratch));
-        assertEquals(expectedRef, dvBytesStraightFixed.getBytes(i, scratch));
-        assertEquals(expectedRef, dvBytesStraightVar.getBytes(i, scratch));
+        dvBytesDerefFixed.get(i, scratch);
+        assertEquals(expectedRef, scratch);
+        dvBytesDerefVar.get(i, scratch);
+        assertEquals(expectedRef, scratch);
+        dvBytesSortedFixed.get(i, scratch);
+        assertEquals(expectedRef, scratch);
+        dvBytesSortedVar.get(i, scratch);
+        assertEquals(expectedRef, scratch);
+        dvBytesStraightFixed.get(i, scratch);
+        assertEquals(expectedRef, scratch);
+        dvBytesStraightVar.get(i, scratch);
+        assertEquals(expectedRef, scratch);
         
-        assertEquals((double)id, dvDouble.getFloat(i), 0D);
-        assertEquals((float)id, dvFloat.getFloat(i), 0F);
-        assertEquals(id, dvInt.getInt(i));
-        assertEquals(id, dvLong.getInt(i));
-        assertEquals(id, dvPacked.getInt(i));
-        assertEquals(id, dvShort.getInt(i));
+        assertEquals((double)id, Double.longBitsToDouble(dvDouble.get(i)), 0D);
+        assertEquals((float)id, Float.intBitsToFloat((int)dvFloat.get(i)), 0F);
+        assertEquals(id, dvInt.get(i));
+        assertEquals(id, dvLong.get(i));
+        assertEquals(id, dvPacked.get(i));
+        assertEquals(id, dvShort.get(i));
       }
     }
     
@@ -679,23 +680,23 @@ public class TestBackwardsCompatibility 
     doc.add(new IntField("trieInt", id, Field.Store.NO));
     doc.add(new LongField("trieLong", (long) id, Field.Store.NO));
     // add docvalues fields
-    doc.add(new ByteDocValuesField("dvByte", (byte) id));
+    doc.add(new NumericDocValuesField("dvByte", (byte) id));
     byte bytes[] = new byte[] {
       (byte)(id >>> 24), (byte)(id >>> 16),(byte)(id >>> 8),(byte)id
     };
     BytesRef ref = new BytesRef(bytes);
-    doc.add(new DerefBytesDocValuesField("dvBytesDerefFixed", ref, true));
-    doc.add(new DerefBytesDocValuesField("dvBytesDerefVar", ref, false));
-    doc.add(new SortedBytesDocValuesField("dvBytesSortedFixed", ref, true));
-    doc.add(new SortedBytesDocValuesField("dvBytesSortedVar", ref, false));
-    doc.add(new StraightBytesDocValuesField("dvBytesStraightFixed", ref, true));
-    doc.add(new StraightBytesDocValuesField("dvBytesStraightVar", ref, false));
+    doc.add(new BinaryDocValuesField("dvBytesDerefFixed", ref));
+    doc.add(new BinaryDocValuesField("dvBytesDerefVar", ref));
+    doc.add(new SortedDocValuesField("dvBytesSortedFixed", ref));
+    doc.add(new SortedDocValuesField("dvBytesSortedVar", ref));
+    doc.add(new BinaryDocValuesField("dvBytesStraightFixed", ref));
+    doc.add(new BinaryDocValuesField("dvBytesStraightVar", ref));
     doc.add(new DoubleDocValuesField("dvDouble", (double)id));
     doc.add(new FloatDocValuesField("dvFloat", (float)id));
-    doc.add(new IntDocValuesField("dvInt", id));
-    doc.add(new LongDocValuesField("dvLong", id));
-    doc.add(new PackedLongDocValuesField("dvPacked", id));
-    doc.add(new ShortDocValuesField("dvShort", (short)id));
+    doc.add(new NumericDocValuesField("dvInt", id));
+    doc.add(new NumericDocValuesField("dvLong", id));
+    doc.add(new NumericDocValuesField("dvPacked", id));
+    doc.add(new NumericDocValuesField("dvShort", (short)id));
     // a field with both offsets and term vectors for a cross-check
     FieldType customType3 = new FieldType(TextField.TYPE_STORED);
     customType3.setStoreTermVectors(true);
@@ -841,13 +842,16 @@ public class TestBackwardsCompatibility 
       assertEquals("wrong number of hits", 34, hits.length);
       
       // check decoding into field cache
-      int[] fci = FieldCache.DEFAULT.getInts(SlowCompositeReaderWrapper.wrap(searcher.getIndexReader()), "trieInt", false);
-      for (int val : fci) {
+      FieldCache.Ints fci = FieldCache.DEFAULT.getInts(SlowCompositeReaderWrapper.wrap(searcher.getIndexReader()), "trieInt", false);
+      int maxDoc = searcher.getIndexReader().maxDoc();
+      for(int doc=0;doc<maxDoc;doc++) {
+        int val = fci.get(doc);
         assertTrue("value in id bounds", val >= 0 && val < 35);
       }
       
-      long[] fcl = FieldCache.DEFAULT.getLongs(SlowCompositeReaderWrapper.wrap(searcher.getIndexReader()), "trieLong", false);
-      for (long val : fcl) {
+      FieldCache.Longs fcl = FieldCache.DEFAULT.getLongs(SlowCompositeReaderWrapper.wrap(searcher.getIndexReader()), "trieLong", false);
+      for(int doc=0;doc<maxDoc;doc++) {
+        long val = fcl.get(doc);
         assertTrue("value in id bounds", val >= 0L && val < 35L);
       }