You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jp...@apache.org on 2013/07/30 14:42:40 UTC

svn commit: r1508423 - in /lucene/dev/trunk/lucene: ./ core/src/java/org/apache/lucene/index/ core/src/java/org/apache/lucene/util/ core/src/java/org/apache/lucene/util/packed/ core/src/test/org/apache/lucene/util/packed/

Author: jpountz
Date: Tue Jul 30 12:42:39 2013
New Revision: 1508423

URL: http://svn.apache.org/r1508423
Log:
LUCENE-5145: AppendingPackedLongBuffer and added suport for bulk get operations to the Appending*Buffers.

Introduced bulk retrieval to AbstractAppendingLongBuffer
classes, for faster retrieval. Introduced a new variant,
AppendingPackedLongBuffer which solely relies on PackedInts as a backend.
This new class is useful where people have non-negative numbers with a
uniform distribution over a fixed (limited) range. Ex. facets ordinals. To
distinguish it from AppendingPackedLongBuffer, delta based
AppendingLongBuffer was renamed to AppendingDeltaPackedLongBuffer Fixed an
Issue with NullReader where it didn't respect it's valueCount in bulk gets.

Added:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/AppendingDeltaPackedLongBuffer.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/AppendingPackedLongBuffer.java   (with props)
Removed:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/AppendingLongBuffer.java
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesWriter.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesWriter.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/WAH8DocIdSet.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/AbstractAppendingLongBuffer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/MonotonicAppendingLongBuffer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/PagedGrowableWriter.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/package.html
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/packed/TestPackedInts.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1508423&r1=1508422&r2=1508423&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Tue Jul 30 12:42:39 2013
@@ -136,6 +136,13 @@ Optimizations
 * LUCENE-5119: DiskDV keeps the document-to-ordinal mapping on disk for 
   SortedDocValues.  (Robert Muir)
 
+* LUCENE-5145: New AppendingPackedLongBuffer, a new variant of the former
+  AppendingLongBuffer which assumes values are 0-based.
+  (Boaz Leskes via Adrien Grand)
+
+* LUCENE-5145: All Appending*Buffer now support bulk get.
+  (Boaz Leskes via Adrien Grand)
+
 Documentation
 
 * LUCENE-4894: remove facet userguide as it was outdated. Partially absorbed into

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

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java?rev=1508423&r1=1508422&r2=1508423&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java Tue Jul 30 12:42:39 2013
@@ -23,8 +23,9 @@ import java.util.List;
 import org.apache.lucene.index.MultiTermsEnum.TermsEnumIndex;
 import org.apache.lucene.index.MultiTermsEnum.TermsEnumWithSlice;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.packed.AppendingLongBuffer;
+import org.apache.lucene.util.packed.AppendingPackedLongBuffer;
 import org.apache.lucene.util.packed.MonotonicAppendingLongBuffer;
+import org.apache.lucene.util.packed.PackedInts;
 
 /**
  * A wrapper for CompositeIndexReader providing access to DocValues.
@@ -277,7 +278,7 @@ public class MultiDocValues {
     // globalOrd -> (globalOrd - segmentOrd)
     final MonotonicAppendingLongBuffer globalOrdDeltas;
     // globalOrd -> sub index
-    final AppendingLongBuffer subIndexes;
+    final AppendingPackedLongBuffer subIndexes;
     // segmentOrd -> (globalOrd - segmentOrd)
     final MonotonicAppendingLongBuffer ordDeltas[];
     
@@ -293,8 +294,8 @@ public class MultiDocValues {
       // create the ordinal mappings by pulling a termsenum over each sub's 
       // unique terms, and walking a multitermsenum over those
       this.owner = owner;
-      globalOrdDeltas = new MonotonicAppendingLongBuffer();
-      subIndexes = new AppendingLongBuffer();
+      globalOrdDeltas = new MonotonicAppendingLongBuffer(PackedInts.COMPACT);
+      subIndexes = new AppendingPackedLongBuffer(PackedInts.COMPACT);
       ordDeltas = new MonotonicAppendingLongBuffer[subs.length];
       for (int i = 0; i < ordDeltas.length; i++) {
         ordDeltas[i] = new MonotonicAppendingLongBuffer();

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java?rev=1508423&r1=1508422&r2=1508423&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java Tue Jul 30 12:42:39 2013
@@ -23,7 +23,8 @@ import java.util.NoSuchElementException;
 
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.util.Counter;
-import org.apache.lucene.util.packed.AppendingLongBuffer;
+import org.apache.lucene.util.packed.AppendingDeltaPackedLongBuffer;
+import org.apache.lucene.util.packed.PackedInts;
 
 /** Buffers up pending long per doc, then flushes when
  *  segment flushes. */
@@ -31,13 +32,13 @@ class NumericDocValuesWriter extends Doc
 
   private final static long MISSING = 0L;
 
-  private AppendingLongBuffer pending;
+  private AppendingDeltaPackedLongBuffer pending;
   private final Counter iwBytesUsed;
   private long bytesUsed;
   private final FieldInfo fieldInfo;
 
   public NumericDocValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
-    pending = new AppendingLongBuffer();
+    pending = new AppendingDeltaPackedLongBuffer(PackedInts.COMPACT);
     bytesUsed = pending.ramBytesUsed();
     this.fieldInfo = fieldInfo;
     this.iwBytesUsed = iwBytesUsed;
@@ -89,7 +90,7 @@ class NumericDocValuesWriter extends Doc
   
   // iterates over the values we have in ram
   private class NumericIterator implements Iterator<Number> {
-    final AppendingLongBuffer.Iterator iter = pending.iterator();
+    final AppendingDeltaPackedLongBuffer.Iterator iter = pending.iterator();
     final int size = (int)pending.size();
     final int maxDoc;
     int upto;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesWriter.java?rev=1508423&r1=1508422&r2=1508423&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesWriter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesWriter.java Tue Jul 30 12:42:39 2013
@@ -30,13 +30,14 @@ import org.apache.lucene.util.BytesRefHa
 import org.apache.lucene.util.BytesRefHash;
 import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.RamUsageEstimator;
-import org.apache.lucene.util.packed.AppendingLongBuffer;
+import org.apache.lucene.util.packed.AppendingPackedLongBuffer;
+import org.apache.lucene.util.packed.PackedInts;
 
 /** Buffers up pending byte[] per doc, deref and sorting via
  *  int ord, then flushes when segment flushes. */
 class SortedDocValuesWriter extends DocValuesWriter {
   final BytesRefHash hash;
-  private AppendingLongBuffer pending;
+  private AppendingPackedLongBuffer pending;
   private final Counter iwBytesUsed;
   private long bytesUsed; // this currently only tracks differences in 'pending'
   private final FieldInfo fieldInfo;
@@ -51,7 +52,7 @@ class SortedDocValuesWriter extends DocV
             new ByteBlockPool.DirectTrackingAllocator(iwBytesUsed)),
             BytesRefHash.DEFAULT_CAPACITY,
             new DirectBytesStartArray(BytesRefHash.DEFAULT_CAPACITY, iwBytesUsed));
-    pending = new AppendingLongBuffer();
+    pending = new AppendingPackedLongBuffer(PackedInts.COMPACT);
     bytesUsed = pending.ramBytesUsed();
     iwBytesUsed.addAndGet(bytesUsed);
   }
@@ -176,7 +177,7 @@ class SortedDocValuesWriter extends DocV
   
   // iterates over the ords for each doc we have in ram
   private class OrdsIterator implements Iterator<Number> {
-    final AppendingLongBuffer.Iterator iter = pending.iterator();
+    final AppendingPackedLongBuffer.Iterator iter = pending.iterator();
     final int ordMap[];
     final int maxDoc;
     int docUpto;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesWriter.java?rev=1508423&r1=1508422&r2=1508423&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesWriter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesWriter.java Tue Jul 30 12:42:39 2013
@@ -32,14 +32,16 @@ import org.apache.lucene.util.BytesRefHa
 import org.apache.lucene.util.BytesRefHash;
 import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.RamUsageEstimator;
-import org.apache.lucene.util.packed.AppendingLongBuffer;
+import org.apache.lucene.util.packed.AppendingDeltaPackedLongBuffer;
+import org.apache.lucene.util.packed.AppendingPackedLongBuffer;
+import org.apache.lucene.util.packed.PackedInts;
 
 /** Buffers up pending byte[]s per doc, deref and sorting via
  *  int ord, then flushes when segment flushes. */
 class SortedSetDocValuesWriter extends DocValuesWriter {
   final BytesRefHash hash;
-  private AppendingLongBuffer pending; // stream of all termIDs
-  private AppendingLongBuffer pendingCounts; // termIDs per doc
+  private AppendingPackedLongBuffer pending; // stream of all termIDs
+  private AppendingDeltaPackedLongBuffer pendingCounts; // termIDs per doc
   private final Counter iwBytesUsed;
   private long bytesUsed; // this only tracks differences in 'pending' and 'pendingCounts'
   private final FieldInfo fieldInfo;
@@ -56,8 +58,8 @@ class SortedSetDocValuesWriter extends D
             new ByteBlockPool.DirectTrackingAllocator(iwBytesUsed)),
             BytesRefHash.DEFAULT_CAPACITY,
             new DirectBytesStartArray(BytesRefHash.DEFAULT_CAPACITY, iwBytesUsed));
-    pending = new AppendingLongBuffer();
-    pendingCounts = new AppendingLongBuffer();
+    pending = new AppendingPackedLongBuffer(PackedInts.COMPACT);
+    pendingCounts = new AppendingDeltaPackedLongBuffer(PackedInts.COMPACT);
     bytesUsed = pending.ramBytesUsed() + pendingCounts.ramBytesUsed();
     iwBytesUsed.addAndGet(bytesUsed);
   }
@@ -224,8 +226,8 @@ class SortedSetDocValuesWriter extends D
   
   // iterates over the ords for each doc we have in ram
   private class OrdsIterator implements Iterator<Number> {
-    final AppendingLongBuffer.Iterator iter = pending.iterator();
-    final AppendingLongBuffer.Iterator counts = pendingCounts.iterator();
+    final AppendingPackedLongBuffer.Iterator iter = pending.iterator();
+    final AppendingDeltaPackedLongBuffer.Iterator counts = pendingCounts.iterator();
     final int ordMap[];
     final long numOrds;
     long ordUpto;
@@ -273,7 +275,7 @@ class SortedSetDocValuesWriter extends D
   }
   
   private class OrdCountIterator implements Iterator<Number> {
-    final AppendingLongBuffer.Iterator iter = pendingCounts.iterator();
+    final AppendingDeltaPackedLongBuffer.Iterator iter = pendingCounts.iterator();
     final int maxDoc;
     int docUpto;
     

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/WAH8DocIdSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/WAH8DocIdSet.java?rev=1508423&r1=1508422&r2=1508423&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/WAH8DocIdSet.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/WAH8DocIdSet.java Tue Jul 30 12:42:39 2013
@@ -27,6 +27,7 @@ import org.apache.lucene.search.DocIdSet
 import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.util.packed.MonotonicAppendingLongBuffer;
+import org.apache.lucene.util.packed.PackedInts;
 
 /**
  * {@link DocIdSet} implementation based on word-aligned hybrid encoding on
@@ -330,9 +331,9 @@ public final class WAH8DocIdSet extends 
       } else {
         final int pageSize = 128;
         final int initialPageCount = (valueCount + pageSize - 1) / pageSize;
-        final MonotonicAppendingLongBuffer positions = new MonotonicAppendingLongBuffer(initialPageCount, pageSize);
-        final MonotonicAppendingLongBuffer wordNums = new MonotonicAppendingLongBuffer(initialPageCount, pageSize);
- 
+        final MonotonicAppendingLongBuffer positions = new MonotonicAppendingLongBuffer(initialPageCount, pageSize, PackedInts.COMPACT);
+        final MonotonicAppendingLongBuffer wordNums = new MonotonicAppendingLongBuffer(initialPageCount, pageSize, PackedInts.COMPACT);
+
         positions.add(0L);
         wordNums.add(0L);
         final Iterator it = new Iterator(data, cardinality, Integer.MAX_VALUE, SINGLE_ZERO_BUFFER, SINGLE_ZERO_BUFFER);

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/AbstractAppendingLongBuffer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/AbstractAppendingLongBuffer.java?rev=1508423&r1=1508422&r2=1508423&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/AbstractAppendingLongBuffer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/AbstractAppendingLongBuffer.java Tue Jul 30 12:42:39 2013
@@ -17,14 +17,14 @@ package org.apache.lucene.util.packed;
  * limitations under the License.
  */
 
-import static org.apache.lucene.util.packed.PackedInts.checkBlockSize;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.RamUsageEstimator;
 
 import java.util.Arrays;
 
-import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.RamUsageEstimator;
+import static org.apache.lucene.util.packed.PackedInts.checkBlockSize;
 
-/** Common functionality shared by {@link AppendingLongBuffer} and {@link MonotonicAppendingLongBuffer}. */
+/** Common functionality shared by {@link AppendingDeltaPackedLongBuffer} and {@link MonotonicAppendingLongBuffer}. */
 abstract class AbstractAppendingLongBuffer {
 
   static final int MIN_PAGE_SIZE = 64;
@@ -33,21 +33,21 @@ abstract class AbstractAppendingLongBuff
   static final int MAX_PAGE_SIZE = 1 << 20;
 
   final int pageShift, pageMask;
-  long[] minValues;
-  PackedInts.Reader[] deltas;
-  private long deltasBytes;
+  PackedInts.Reader[] values;
+  private long valuesBytes;
   int valuesOff;
   long[] pending;
   int pendingOff;
+  float acceptableOverheadRatio;
 
-  AbstractAppendingLongBuffer(int initialBlockCount, int pageSize) {
-    minValues = new long[initialBlockCount];
-    deltas = new PackedInts.Reader[initialBlockCount];
+  AbstractAppendingLongBuffer(int initialBlockCount, int pageSize, float acceptableOverheadRatio) {
+    values = new PackedInts.Reader[initialBlockCount];
     pending = new long[pageSize];
     pageShift = checkBlockSize(pageSize, MIN_PAGE_SIZE, MAX_PAGE_SIZE);
     pageMask = pageSize - 1;
     valuesOff = 0;
     pendingOff = 0;
+    this.acceptableOverheadRatio = acceptableOverheadRatio;
   }
 
   final int pageSize() {
@@ -58,7 +58,7 @@ abstract class AbstractAppendingLongBuff
   public final long size() {
     long size = pendingOff;
     if (valuesOff > 0) {
-      size += deltas[valuesOff - 1].size();
+      size += values[valuesOff - 1].size();
     }
     if (valuesOff > 1) {
       size += (long) (valuesOff - 1) * pageSize();
@@ -73,12 +73,12 @@ abstract class AbstractAppendingLongBuff
     }
     if (pendingOff == pending.length) {
       // check size
-      if (deltas.length == valuesOff) {
+      if (values.length == valuesOff) {
         final int newLength = ArrayUtil.oversize(valuesOff + 1, 8);
         grow(newLength);
       }
       packPendingValues();
-      deltasBytes += deltas[valuesOff].ramBytesUsed();
+      valuesBytes += values[valuesOff].ramBytesUsed();
       ++valuesOff;
       // reset pending buffer
       pendingOff = 0;
@@ -87,8 +87,7 @@ abstract class AbstractAppendingLongBuff
   }
 
   void grow(int newBlockCount) {
-    minValues = Arrays.copyOf(minValues, newBlockCount);
-    deltas = Arrays.copyOf(deltas, newBlockCount);
+    values = Arrays.copyOf(values, newBlockCount);
   }
 
   abstract void packPendingValues();
@@ -101,11 +100,33 @@ abstract class AbstractAppendingLongBuff
     return get(block, element);
   }
 
+  /**
+   * Bulk get: read at least one and at most <code>len</code> longs starting
+   * from <code>index</code> into <code>arr[off:off+len]</code> and return
+   * the actual number of values that have been read.
+   */
+  public final int get(long index, long[] arr, int off, int len) {
+    assert len > 0 : "len must be > 0 (got " + len + ")";
+    assert index >= 0 && index < size();
+    assert off + len <= arr.length;
+
+    int block = (int) (index >> pageShift);
+    int element = (int) (index & pageMask);
+    return get(block, element, arr, off, len);
+  }
+
+
   abstract long get(int block, int element);
 
-  abstract Iterator iterator();
+  abstract int get(int block, int element, long[] arr, int off, int len);
+
+
+  /** Return an iterator over the values of this buffer. */
+  public Iterator iterator() {
+    return new Iterator();
+  }
 
-  abstract class Iterator {
+  final public class Iterator {
 
     long[] currentValues;
     int vOff, pOff;
@@ -117,12 +138,22 @@ abstract class AbstractAppendingLongBuff
         currentValues = pending;
         currentCount = pendingOff;
       } else {
-        currentValues = new long[deltas[0].size()];
+        currentValues = new long[values[0].size()];
         fillValues();
       }
     }
 
-    abstract void fillValues();
+    void fillValues() {
+      if (vOff == valuesOff) {
+        currentValues = pending;
+        currentCount = pendingOff;
+      } else {
+        currentCount = values[vOff].size();
+        for (int k = 0; k < currentCount; ) {
+          k += get(vOff, k, currentValues, k, currentCount - k);
+        }
+      }
+    }
 
     /** Whether or not there are remaining values. */
     public final boolean hasNext() {
@@ -149,33 +180,31 @@ abstract class AbstractAppendingLongBuff
 
   long baseRamBytesUsed() {
     return RamUsageEstimator.NUM_BYTES_OBJECT_HEADER
-        + 3 * RamUsageEstimator.NUM_BYTES_OBJECT_REF // the 3 arrays
+        + 2 * RamUsageEstimator.NUM_BYTES_OBJECT_REF // the 2 arrays
         + 2 * RamUsageEstimator.NUM_BYTES_INT // the 2 offsets
         + 2 * RamUsageEstimator.NUM_BYTES_INT // pageShift, pageMask
-        + RamUsageEstimator.NUM_BYTES_LONG; // deltasBytes
+        + RamUsageEstimator.NUM_BYTES_FLOAT   // acceptable overhead
+        + RamUsageEstimator.NUM_BYTES_LONG; // valuesBytes
   }
 
-  /**
-   * Return the number of bytes used by this instance.
-   */
+  /** Return the number of bytes used by this instance. */
   public long ramBytesUsed() {
     // TODO: this is called per-doc-per-norms/dv-field, can we optimize this?
     long bytesUsed = RamUsageEstimator.alignObjectSize(baseRamBytesUsed())
         + (pending != null ? RamUsageEstimator.sizeOf(pending) : 0L)
-        + RamUsageEstimator.sizeOf(minValues)
-        + RamUsageEstimator.alignObjectSize(RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + (long) RamUsageEstimator.NUM_BYTES_OBJECT_REF * deltas.length); // values
+        + RamUsageEstimator.alignObjectSize(RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + (long) RamUsageEstimator.NUM_BYTES_OBJECT_REF * values.length); // values
 
-    return bytesUsed + deltasBytes;
+    return bytesUsed + valuesBytes;
   }
 
   /** Pack all pending values in this buffer. Subsequent calls to {@link #add(long)} will fail. */
   public void freeze() {
     if (pendingOff > 0) {
-      if (deltas.length == valuesOff) {
+      if (values.length == valuesOff) {
         grow(valuesOff + 1); // don't oversize!
       }
       packPendingValues();
-      deltasBytes += deltas[valuesOff].ramBytesUsed();
+      valuesBytes += values[valuesOff].ramBytesUsed();
       ++valuesOff;
       pendingOff = 0;
     }

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/AppendingDeltaPackedLongBuffer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/AppendingDeltaPackedLongBuffer.java?rev=1508423&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/AppendingDeltaPackedLongBuffer.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/AppendingDeltaPackedLongBuffer.java Tue Jul 30 12:42:39 2013
@@ -0,0 +1,136 @@
+package org.apache.lucene.util.packed;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+import org.apache.lucene.util.RamUsageEstimator;
+
+import java.util.Arrays;
+
+/**
+ * Utility class to buffer a list of signed longs in memory. This class only
+ * supports appending and is optimized for the case where values are close to
+ * each other.
+ *
+ * @lucene.internal
+ */
+public final class AppendingDeltaPackedLongBuffer extends AbstractAppendingLongBuffer {
+
+  long[] minValues;
+
+
+  /** Create {@link AppendingDeltaPackedLongBuffer}
+   * @param initialPageCount        the initial number of pages
+   * @param pageSize                the size of a single page
+   * @param acceptableOverheadRatio an acceptable overhead ratio per value
+   */
+  public AppendingDeltaPackedLongBuffer(int initialPageCount, int pageSize, float acceptableOverheadRatio) {
+    super(initialPageCount, pageSize, acceptableOverheadRatio);
+    minValues = new long[values.length];
+  }
+
+  /**
+   * Create an {@link AppendingDeltaPackedLongBuffer} with initialPageCount=16,
+   * pageSize=1024 and acceptableOverheadRatio={@link PackedInts#DEFAULT}
+   */
+  public AppendingDeltaPackedLongBuffer() {
+    this(16, 1024, PackedInts.DEFAULT);
+  }
+
+  /**
+   * Create an {@link AppendingDeltaPackedLongBuffer} with initialPageCount=16,
+   * pageSize=1024
+   */
+  public AppendingDeltaPackedLongBuffer(float acceptableOverheadRatio) {
+    this(16, 1024, acceptableOverheadRatio);
+  }
+
+  @Override
+  long get(int block, int element) {
+    if (block == valuesOff) {
+      return pending[element];
+    } else if (values[block] == null) {
+      return minValues[block];
+    } else {
+      return minValues[block] + values[block].get(element);
+    }
+  }
+
+  @Override
+  int get(int block, int element, long[] arr, int off, int len) {
+    if (block == valuesOff) {
+      int sysCopyToRead = Math.min(len, pendingOff - element);
+      System.arraycopy(pending, element, arr, off, sysCopyToRead);
+      return sysCopyToRead;
+    } else {
+      /* packed block */
+      int read = values[block].get(element, arr, off, len);
+      long d = minValues[block];
+      for (int r = 0; r < read; r++, off++) {
+        arr[off] += d;
+      }
+      return read;
+    }
+  }
+
+  @Override
+  void packPendingValues() {
+    // compute max delta
+    long minValue = pending[0];
+    long maxValue = pending[0];
+    for (int i = 1; i < pendingOff; ++i) {
+      minValue = Math.min(minValue, pending[i]);
+      maxValue = Math.max(maxValue, pending[i]);
+    }
+    final long delta = maxValue - minValue;
+
+    minValues[valuesOff] = minValue;
+    if (delta == 0) {
+      values[valuesOff] = new PackedInts.NullReader(pendingOff);
+    } else {
+      // build a new packed reader
+      final int bitsRequired = delta < 0 ? 64 : PackedInts.bitsRequired(delta);
+      for (int i = 0; i < pendingOff; ++i) {
+        pending[i] -= minValue;
+      }
+      final PackedInts.Mutable mutable = PackedInts.getMutable(pendingOff, bitsRequired, acceptableOverheadRatio);
+      for (int i = 0; i < pendingOff; ) {
+        i += mutable.set(i, pending, i, pendingOff - i);
+      }
+      values[valuesOff] = mutable;
+    }
+  }
+
+  @Override
+  void grow(int newBlockCount) {
+    super.grow(newBlockCount);
+    this.minValues = Arrays.copyOf(minValues, newBlockCount);
+  }
+
+  @Override
+  long baseRamBytesUsed() {
+    return super.baseRamBytesUsed()
+        + RamUsageEstimator.NUM_BYTES_OBJECT_REF; // additional array
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return super.ramBytesUsed() + RamUsageEstimator.sizeOf(minValues);
+  }
+
+}

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/AppendingPackedLongBuffer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/AppendingPackedLongBuffer.java?rev=1508423&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/AppendingPackedLongBuffer.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/AppendingPackedLongBuffer.java Tue Jul 30 12:42:39 2013
@@ -0,0 +1,96 @@
+package org.apache.lucene.util.packed;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+/**
+ * Utility class to buffer a list of signed longs in memory. This class only
+ * supports appending and is optimized for non-negative numbers with a uniform distribution over a fixed (limited) range
+ *
+ * @lucene.internal
+ */
+public final class AppendingPackedLongBuffer extends AbstractAppendingLongBuffer {
+
+  /**{@link AppendingPackedLongBuffer}
+   * @param initialPageCount        the initial number of pages
+   * @param pageSize                the size of a single page
+   * @param acceptableOverheadRatio an acceptable overhead ratio per value
+   */
+  public AppendingPackedLongBuffer(int initialPageCount, int pageSize, float acceptableOverheadRatio) {
+    super(initialPageCount, pageSize, acceptableOverheadRatio);
+  }
+
+  /**
+   * Create an {@link AppendingPackedLongBuffer} with initialPageCount=16,
+   * pageSize=1024 and acceptableOverheadRatio={@link PackedInts#DEFAULT}
+   */
+  public AppendingPackedLongBuffer() {
+    this(16, 1024, PackedInts.DEFAULT);
+  }
+
+  /**
+   * Create an {@link AppendingPackedLongBuffer} with initialPageCount=16,
+   * pageSize=1024
+   */
+  public AppendingPackedLongBuffer(float acceptableOverheadRatio) {
+    this(16, 1024, acceptableOverheadRatio);
+  }
+
+  @Override
+  long get(int block, int element) {
+    if (block == valuesOff) {
+      return pending[element];
+    } else {
+      return values[block].get(element);
+    }
+  }
+
+  @Override
+  int get(int block, int element, long[] arr, int off, int len) {
+    if (block == valuesOff) {
+      int sysCopyToRead = Math.min(len, pendingOff - element);
+      System.arraycopy(pending, element, arr, off, sysCopyToRead);
+      return sysCopyToRead;
+    } else {
+      /* packed block */
+      return values[block].get(element, arr, off, len);
+    }
+  }
+
+  @Override
+  void packPendingValues() {
+    // compute max delta
+    long minValue = pending[0];
+    long maxValue = pending[0];
+    for (int i = 1; i < pendingOff; ++i) {
+      minValue = Math.min(minValue, pending[i]);
+      maxValue = Math.max(maxValue, pending[i]);
+    }
+
+
+    // build a new packed reader
+    final int bitsRequired = minValue < 0 ? 64 : PackedInts.bitsRequired(maxValue);
+    final PackedInts.Mutable mutable = PackedInts.getMutable(pendingOff, bitsRequired, acceptableOverheadRatio);
+    for (int i = 0; i < pendingOff; ) {
+      i += mutable.set(i, pending, i, pendingOff - i);
+    }
+    values[valuesOff] = mutable;
+
+  }
+
+}

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

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java?rev=1508423&r1=1508422&r2=1508423&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java Tue Jul 30 12:42:39 2013
@@ -705,6 +705,9 @@ public class PackedInts {
 
     @Override
     public int get(int index, long[] arr, int off, int len) {
+      assert len > 0 : "len must be > 0 (got " + len + ")";
+      assert index >= 0 && index < valueCount;
+      len = Math.min(len, valueCount - index);
       Arrays.fill(arr, off, off + len, 0);
       return len;
     }

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

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/package.html?rev=1508423&r1=1508422&r2=1508423&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/package.html (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/package.html Tue Jul 30 12:42:39 2013
@@ -50,16 +50,19 @@
     <li><b>{@link org.apache.lucene.util.packed.PagedGrowableWriter}</b><ul>
         <li>Slices data into fixed-size blocks stored in GrowableWriters.</li>
         <li>Supports more than 2B values.</li>
-        <li>You should use AppendingLongBuffer instead if you don't need random write access.</li>
+        <li>You should use Appending(Delta)PackedLongBuffer instead if you don't need random write access.</li>
     </ul></li>
-    <li><b>{@link org.apache.lucene.util.packed.AppendingLongBuffer}</b><ul>
+    <li><b>{@link org.apache.lucene.util.packed.AppendingDeltaPackedLongBuffer}</b><ul>
         <li>Can store any sequence of longs.</li>
         <li>Compression is good when values are close to each other.</li>
         <li>Supports random reads, but only sequential writes.</li>
         <li>Can address up to 2^42 values.</li>
     </ul></li>
+    <li><b>{@link org.apache.lucene.util.packed.AppendingPackedLongBuffer}</b><ul>
+        <li>Same as AppendingDeltaPackedLongBuffer but assumes values are 0-based.</li>
+    </ul></li>
     <li><b>{@link org.apache.lucene.util.packed.MonotonicAppendingLongBuffer}</b><ul>
-        <li>Same as AppendingLongBuffer except that compression is good when the stream is a succession of affine functions.</li>
+        <li>Same as AppendingDeltaPackedLongBuffer except that compression is good when the stream is a succession of affine functions.</li>
     </ul></li>
 </ul>
 

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/packed/TestPackedInts.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/packed/TestPackedInts.java?rev=1508423&r1=1508422&r2=1508423&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/packed/TestPackedInts.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/packed/TestPackedInts.java Tue Jul 30 12:42:39 2013
@@ -540,6 +540,27 @@ public class TestPackedInts extends Luce
     }
   }
 
+  public void testPackedIntsNull() {
+    int size = _TestUtil.nextInt(random(), 0, 256);
+    Reader packedInts = new PackedInts.NullReader(size);
+    assertEquals(0, packedInts.get(_TestUtil.nextInt(random(), 0, size - 1)));
+    long[] arr = new long[size + 10];
+    int r;
+    Arrays.fill(arr, 1);
+    r = packedInts.get(0, arr, 0, size - 1);
+    assertEquals(size - 1, r);
+    for (r--; r >= 0; r--) {
+      assertEquals(0, arr[r]);
+    }
+    Arrays.fill(arr, 1);
+    r = packedInts.get(10, arr, 0, size + 10);
+    assertEquals(size - 10, r);
+    for (int i = 0; i < size - 10; i++) {
+      assertEquals(0, arr[i]);
+    }
+
+  }
+
   public void testBulkGet() {
     final int valueCount = 1111;
     final int index = random().nextInt(valueCount);
@@ -669,8 +690,8 @@ public class TestPackedInts extends Luce
     PagedGrowableWriter writer = new PagedGrowableWriter(0, pageSize, _TestUtil.nextInt(random(), 1, 64), random().nextFloat());
     assertEquals(0, writer.size());
 
-    // compare against AppendingLongBuffer
-    AppendingLongBuffer buf = new AppendingLongBuffer();
+    // compare against AppendingDeltaPackedLongBuffer
+    AppendingDeltaPackedLongBuffer buf = new AppendingDeltaPackedLongBuffer();
     int size = random().nextInt(1000000);
     long max = 5;
     for (int i = 0; i < size; ++i) {
@@ -720,8 +741,8 @@ public class TestPackedInts extends Luce
     PagedMutable writer = new PagedMutable(0, pageSize, bitsPerValue, random().nextFloat() / 2);
     assertEquals(0, writer.size());
 
-    // compare against AppendingLongBuffer
-    AppendingLongBuffer buf = new AppendingLongBuffer();
+    // compare against AppendingDeltaPackedLongBuffer
+    AppendingDeltaPackedLongBuffer buf = new AppendingDeltaPackedLongBuffer();
     int size = random().nextInt(1000000);
     
     for (int i = 0; i < size; ++i) {
@@ -924,25 +945,46 @@ public class TestPackedInts extends Luce
     return true;
   }
 
+  enum DataType {
+    PACKED,
+    DELTA_PACKED,
+    MONOTONIC
+  }
+
+
   public void testAppendingLongBuffer() {
+
     final long[] arr = new long[RandomInts.randomIntBetween(random(), 1, 1000000)];
-    for (int bpv : new int[] {0, 1, 63, 64, RandomInts.randomIntBetween(random(), 2, 62)}) {
-      for (boolean monotonic : new boolean[] {true, false}) {
+    float[] ratioOptions = new float[]{PackedInts.DEFAULT, PackedInts.COMPACT, PackedInts.FAST};
+    for (int bpv : new int[]{0, 1, 63, 64, RandomInts.randomIntBetween(random(), 2, 62)}) {
+      for (DataType dataType : DataType.values()) {
         final int pageSize = 1 << _TestUtil.nextInt(random(), 6, 20);
         final int initialPageCount = _TestUtil.nextInt(random(), 0, 16);
+        float acceptableOverheadRatio = ratioOptions[_TestUtil.nextInt(random(), 0, ratioOptions.length - 1)];
         AbstractAppendingLongBuffer buf;
         final int inc;
-        if (monotonic) {
-          buf = new MonotonicAppendingLongBuffer(initialPageCount, pageSize);
-          inc = _TestUtil.nextInt(random(), -1000, 1000);
-        } else {
-          buf = new AppendingLongBuffer(initialPageCount, pageSize);
-          inc = 0;
+        switch (dataType) {
+          case PACKED:
+            buf = new AppendingPackedLongBuffer(initialPageCount, pageSize, acceptableOverheadRatio);
+            inc = 0;
+            break;
+          case DELTA_PACKED:
+            buf = new AppendingDeltaPackedLongBuffer(initialPageCount, pageSize, acceptableOverheadRatio);
+            inc = 0;
+            break;
+          case MONOTONIC:
+            buf = new MonotonicAppendingLongBuffer(initialPageCount, pageSize, acceptableOverheadRatio);
+            inc = _TestUtil.nextInt(random(), -1000, 1000);
+            break;
+          default:
+            throw new RuntimeException("added a type and forgot to add it here?");
+
         }
+
         if (bpv == 0) {
           arr[0] = random().nextLong();
           for (int i = 1; i < arr.length; ++i) {
-            arr[i] = arr[i-1] + inc;
+            arr[i] = arr[i - 1] + inc;
           }
         } else if (bpv == 64) {
           for (int i = 0; i < arr.length; ++i) {
@@ -954,6 +996,7 @@ public class TestPackedInts extends Luce
             arr[i] = minValue + inc * i + random().nextLong() & PackedInts.maxValue(bpv); // _TestUtil.nextLong is too slow
           }
         }
+
         for (int i = 0; i < arr.length; ++i) {
           buf.add(arr[i]);
         }
@@ -966,6 +1009,11 @@ public class TestPackedInts extends Luce
           }
         }
         assertEquals(arr.length, buf.size());
+
+        for (int i = 0; i < arr.length; ++i) {
+          assertEquals(arr[i], buf.get(i));
+        }
+
         final AbstractAppendingLongBuffer.Iterator it = buf.iterator();
         for (int i = 0; i < arr.length; ++i) {
           if (random().nextBoolean()) {
@@ -974,11 +1022,27 @@ public class TestPackedInts extends Luce
           assertEquals(arr[i], it.next());
         }
         assertFalse(it.hasNext());
-        
-        for (int i = 0; i < arr.length; ++i) {
-          assertEquals(arr[i], buf.get(i));
+
+
+        long[] target = new long[arr.length + 1024]; // check the request for more is OK.
+        for (int i = 0; i < arr.length; i += _TestUtil.nextInt(random(), 0, 10000)) {
+          int lenToRead = random().nextInt(buf.pageSize() * 2) + 1;
+          lenToRead = Math.min(lenToRead, target.length - i);
+          int lenToCheck = Math.min(lenToRead, arr.length - i);
+          int off = i;
+          while (off < arr.length && lenToRead > 0) {
+            int read = buf.get(off, target, off, lenToRead);
+            assertTrue(read > 0);
+            assertTrue(read <= lenToRead);
+            lenToRead -= read;
+            off += read;
+          }
+
+          for (int j = 0; j < lenToCheck; j++) {
+            assertEquals(arr[j + i], target[j + i]);
+          }
         }
-  
+
         final long expectedBytesUsed = RamUsageEstimator.sizeOf(buf);
         final long computedBytesUsed = buf.ramBytesUsed();
         assertEquals(expectedBytesUsed, computedBytesUsed);



Re: svn commit: r1508423 - in /lucene/dev/trunk/lucene: ./ core/src/java/org/apache/lucene/index/ core/src/java/org/apache/lucene/util/ core/src/java/org/apache/lucene/util/packed/ core/src/test/org/apache/lucene/util/packed/

Posted by Robert Muir <rc...@gmail.com>.
On Tue, Jul 30, 2013 at 8:42 AM, <jp...@apache.org> wrote:

> Author: jpountz
> Date: Tue Jul 30 12:42:39 2013
> New Revision: 1508423
>
> URL: http://svn.apache.org/r1508423
> Log:
> LUCENE-5145: AppendingPackedLongBuffer and added suport for bulk get
> operations to the Appending*Buffers.
>
> Introduced bulk retrieval to AbstractAppendingLongBuffer
> classes, for faster retrieval. Introduced a new variant,
> AppendingPackedLongBuffer which solely relies on PackedInts as a backend.
> This new class is useful where people have non-negative numbers with a
> uniform distribution over a fixed (limited) range. Ex. facets ordinals. To
> distinguish it from AppendingPackedLongBuffer, delta based
> AppendingLongBuffer was renamed to AppendingDeltaPackedLongBuffer Fixed an
> Issue with NullReader where it didn't respect it's valueCount in bulk gets.
>

Do you think we should remove 'Long' from this class to help with the name?
e.g. can it just be AppendingPackedBuffer? (or something shorter)