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/10/15 17:30:52 UTC

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

Author: jpountz
Date: Tue Oct 15 15:30:51 2013
New Revision: 1532388

URL: http://svn.apache.org/r1532388
Log:
LUCENE-5236: EliasFanoDocIdSet now has an index and uses broadword bit selection to speed-up advance()

Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BroadWord.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/EliasFanoDecoder.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/EliasFanoDocIdSet.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/EliasFanoEncoder.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestBroadWord.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/packed/TestEliasFanoSequence.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1532388&r1=1532387&r2=1532388&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Tue Oct 15 15:30:51 2013
@@ -172,6 +172,9 @@ Optimizations
   doc ids and child scores if the ToParentBlockJoinCollector is used.
   (Martijn van Groningen)
 
+* LUCENE-5236: EliasFanoDocIdSet now has an index and uses broadword bit
+  selection to speed-up advance(). (Paul Elschot via Adrien Grand)
+
 Documentation
 
 * LUCENE-5211: Better javadocs and error checking of 'format' option in 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BroadWord.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BroadWord.java?rev=1532388&r1=1532387&r2=1532388&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BroadWord.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BroadWord.java Tue Oct 15 15:30:51 2013
@@ -21,8 +21,8 @@ package org.apache.lucene.util;
  * Methods and constants inspired by the article
  * "Broadword Implementation of Rank/Select Queries" by Sebastiano Vigna, January 30, 2012:
  * <ul>
- * <li>algorithm 1: {@link #rank9(long)}, count of set bits in a <code>long</code>
- * <li>algorithm 2: {@link #select9(long, int)}, selection of a set bit in a <code>long</code>,
+ * <li>algorithm 1: {@link #bitCount(long)}, count of set bits in a <code>long</code>
+ * <li>algorithm 2: {@link #select(long, int)}, selection of a set bit in a <code>long</code>,
  * <li>bytewise signed smaller &lt;<sub><small>8</small></sub> operator: {@link #smallerUpTo7_8(long,long)}.
  * <li>shortwise signed smaller &lt;<sub><small>16</small></sub> operator: {@link #smallerUpto15_16(long,long)}.
  * <li>some of the Lk and Hk constants that are used by the above:
@@ -36,11 +36,11 @@ public final class BroadWord {
   private BroadWord() {} // no instance
 
   /** Bit count of a long.
-   * Only here to compare the implementation with {@link #select9(long,int)},
+   * Only here to compare the implementation with {@link #select(long,int)},
    * normally {@link Long#bitCount} is preferable.
    * @return The total number of 1 bits in x.
    */
-  static int rank9(long x) {
+  static int bitCount(long x) {
     // Step 0 leaves in each pair of bits the number of ones originally contained in that pair:
     x = x - ((x & 0xAAAAAAAAAAAAAAAAL) >>> 1);
     // Step 1, idem for each nibble:
@@ -54,7 +54,7 @@ public final class BroadWord {
   /** Select a 1-bit from a long.
    * @return The index of the r-th 1 bit in x, or if no such bit exists, 72.
    */
-  public static int select9(long x, int r) {
+  public static int select(long x, int r) {
     long s = x - ((x & 0xAAAAAAAAAAAAAAAAL) >>> 1); // Step 0, pairwise bitsums
 
     // Correct a small mistake in algorithm 2:
@@ -132,7 +132,8 @@ public final class BroadWord {
   public final static long H16_L = L16_L << 15;
 
   /**
-   * Naive implementation of {@link #select9(long,int)}, using {@link Long#numberOfTrailingZeros} repetitively.
+   * Naive implementation of {@link #select(long,int)}, using {@link Long#numberOfTrailingZeros} repetitively.
+   * Works relatively fast for low ranks.
    * @return The index of the r-th 1 bit in x, or if no such bit exists, 72.
    */
   public static int selectNaive(long x, int r) {
@@ -147,4 +148,5 @@ public final class BroadWord {
     int res = (r > 0) ? 72 : s;
     return res;
   }
+
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/EliasFanoDecoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/EliasFanoDecoder.java?rev=1532388&r1=1532387&r2=1532388&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/EliasFanoDecoder.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/EliasFanoDecoder.java Tue Oct 15 15:30:51 2013
@@ -17,6 +17,9 @@
 
 package org.apache.lucene.util.packed;
 
+import org.apache.lucene.util.BroadWord; // bit selection in long
+
+
 /** A decoder for an {@link EliasFanoEncoder}.
  * @lucene.internal
  */
@@ -24,34 +27,48 @@ public class EliasFanoDecoder {
   private static final int LOG2_LONG_SIZE = Long.numberOfTrailingZeros(Long.SIZE);
 
   private final EliasFanoEncoder efEncoder;
-  final long numEncoded;
+  private final long numEncoded;
   private long efIndex = -1; // the decoding index.
   private long setBitForIndex = -1; // the index of the high bit at the decoding index.
 
   public final static long NO_MORE_VALUES = -1L;
 
+  private final long numIndexEntries;
+  private final long indexMask;
+
   /** Construct a decoder for a given {@link EliasFanoEncoder}.
    * The decoding index is set to just before the first encoded value.
    */
   public EliasFanoDecoder(EliasFanoEncoder efEncoder) {
     this.efEncoder = efEncoder;
-    this.numEncoded = efEncoder.numEncoded; // numEncoded is not final in EliasFanoEncoder
+    this.numEncoded = efEncoder.numEncoded; // not final in EliasFanoEncoder
+    this.numIndexEntries = efEncoder.currentEntryIndex;  // not final in EliasFanoEncoder
+    this.indexMask = (1L << efEncoder.nIndexEntryBits) - 1;
   }
 
-  /** Return the Elias-Fano encoder that is decoded. */
+  /** @return The Elias-Fano encoder that is decoded. */
   public EliasFanoEncoder getEliasFanoEncoder() {
     return efEncoder;
   }
+  
+  /** The number of values encoded by the encoder.
+   * @return The number of values encoded by the encoder.
+   */
+  public long numEncoded() { 
+    return numEncoded;
+  }
 
 
-  /** Return the index of the last decoded value.
+  /** The current decoding index.
    * The first value encoded by {@link EliasFanoEncoder#encodeNext} has index 0.
    * Only valid directly after
    * {@link #nextValue}, {@link #advanceToValue},
    * {@link #previousValue}, or {@link #backToValue}
-   * returned another value than {@link #NO_MORE_VALUES}.
+   * returned another value than {@link #NO_MORE_VALUES},
+   * or {@link #advanceToIndex} returned true.
+   * @return The decoding index of the last decoded value, or as last set by {@link #advanceToIndex}.
    */
-  public long index() {
+  public long currentIndex() {
     if (efIndex < 0) {
       throw new IllegalStateException("index before sequence");
     }
@@ -61,30 +78,43 @@ public class EliasFanoDecoder {
     return efIndex;
   }
 
-  /** Return the high value for the current decoding index. */
+  /** The value at the current decoding index.
+   * Only valid when {@link #currentIndex} would return a valid result.
+   * <br>This is only intended for use after {@link #advanceToIndex} returned true.
+   * @return The value encoded at {@link #currentIndex}.
+   */
+  public long currentValue() {
+    return combineHighLowValues(currentHighValue(), currentLowValue());
+  }
+
+  /**  @return The high value for the current decoding index. */
   private long currentHighValue() {
     return setBitForIndex - efIndex; // sequence of unary gaps
   }
 
-  /**  Return the low value for the current decoding index. */
-  private long currentLowValue() {
-    assert efIndex >= 0;
-    assert efIndex < numEncoded;
-    if (efEncoder.numLowBits == 0) {
+  /** See also {@link EliasFanoEncoder#packValue} */
+  private static long unPackValue(long[] longArray, int numBits, long packIndex, long bitsMask) {
+    if (numBits == 0) {
       return 0;
     }
-    long bitPos = efIndex * efEncoder.numLowBits;
-    int lowIndex = (int) (bitPos >>> LOG2_LONG_SIZE);
+    long bitPos = packIndex * numBits;
+    int index = (int) (bitPos >>> LOG2_LONG_SIZE);
     int bitPosAtIndex = (int) (bitPos & (Long.SIZE-1));
-    long lowValue = efEncoder.lowerLongs[lowIndex] >>> bitPosAtIndex;
-    if ((bitPosAtIndex + efEncoder.numLowBits) > Long.SIZE) {
-      lowValue |= (efEncoder.lowerLongs[lowIndex + 1] << (Long.SIZE - bitPosAtIndex));
+    long value = longArray[index] >>> bitPosAtIndex;
+    if ((bitPosAtIndex + numBits) > Long.SIZE) {
+      value |= (longArray[index + 1] << (Long.SIZE - bitPosAtIndex));
     }
-    lowValue &= efEncoder.lowerBitsMask;
-    return lowValue;
+    value &= bitsMask;
+    return value;
+  }
+
+  /**  @return The low value for the current decoding index. */
+  private long currentLowValue() {
+    assert ((efIndex >= 0) && (efIndex < numEncoded)) : "efIndex " + efIndex;
+    return unPackValue(efEncoder.lowerLongs, efEncoder.numLowBits, efIndex, efEncoder.lowerBitsMask);
   }
 
-  /**  Return the given highValue shifted left by the number of low bits from by the EliasFanoSequence,
+  /**  @return The given highValue shifted left by the number of low bits from by the EliasFanoSequence,
    *           logically OR-ed with the given lowValue.
    */
   private long combineHighLowValues(long highValue, long lowValue) {
@@ -116,7 +146,7 @@ public class EliasFanoDecoder {
     setBitForIndex = -1;
   }
 
-  /** Return the number of bits in a long after (setBitForIndex modulo Long.SIZE) */
+  /** @return the number of bits in a long after (setBitForIndex modulo Long.SIZE) */
   private int getCurrentRightShift() {
     int s = (int) (setBitForIndex & (Long.SIZE-1));
     return s;
@@ -179,6 +209,9 @@ public class EliasFanoDecoder {
 
   /** Advance the decoding index to a given index.
    * and return <code>true</code> iff it is available.
+   * <br>See also {@link #currentValue}.
+   * <br>The current implementation does not use the index on the upper bit zero bit positions.
+   * <br>Note: there is currently no implementation of <code>backToIndex</code>.
    */
   public boolean advanceToIndex(long index) {
     assert index > efIndex;
@@ -189,6 +222,7 @@ public class EliasFanoDecoder {
     if (! toAfterCurrentHighBit()) {
       assert false;
     }
+    /* CHECKME: Add a (binary) search in the upperZeroBitPositions here. */
     int curSetBits = Long.bitCount(curHighLong);
     while ((efIndex + curSetBits) < index) { // curHighLong has not enough set bits to reach index
       efIndex += curSetBits;
@@ -209,53 +243,110 @@ public class EliasFanoDecoder {
   }
 
 
-  /** setBitForIndex and efIndex have just been incremented, scan forward to the high set bit
-   *  of at least a given high value
-   *  by incrementing setBitForIndex, and by setting curHighLong accordingly.
-   *  @return the smallest encoded high value that is at least the given one.
+
+  /** Given a target value, advance the decoding index to the first bigger or equal value
+   * and return it if it is available. Otherwise return {@link #NO_MORE_VALUES}.
+   * <br>The current implementation uses the index on the upper zero bit positions.
    */
-  private long advanceToHighValue(long highTarget) {
-    int curSetBits = Long.bitCount(curHighLong); // is shifted by getCurrentRightShift()
-    int curClearBits = Long.SIZE - curSetBits - getCurrentRightShift();
-    while ((currentHighValue() + curClearBits) < highTarget) {
+  public long advanceToValue(long target) {
+    efIndex += 1;
+    if (efIndex >= numEncoded) {
+      return NO_MORE_VALUES;
+    }
+    setBitForIndex += 1; // the high bit at setBitForIndex belongs to the unary code for efIndex
+
+    int highIndex = (int)(setBitForIndex >>> LOG2_LONG_SIZE);
+    long upperLong = efEncoder.upperLongs[highIndex];
+    curHighLong = upperLong >>> ((int) (setBitForIndex & (Long.SIZE-1))); // may contain the unary 1 bit for efIndex
+
+    // determine index entry to advance to
+    long highTarget = target >>> efEncoder.numLowBits;
+
+    long indexEntryIndex = (highTarget / efEncoder.indexInterval) - 1;
+    if (indexEntryIndex >= 0) { // not before first index entry
+      if (indexEntryIndex >= numIndexEntries) {
+        indexEntryIndex = numIndexEntries - 1; // no further than last index entry
+      }
+      long indexHighValue = (indexEntryIndex + 1) * efEncoder.indexInterval;
+      assert indexHighValue <= highTarget;
+      if (indexHighValue > (setBitForIndex - efIndex)) { // advance to just after zero bit position of index entry.
+        setBitForIndex = unPackValue(efEncoder.upperZeroBitPositionIndex, efEncoder.nIndexEntryBits, indexEntryIndex, indexMask);
+        efIndex = setBitForIndex - indexHighValue; // the high bit at setBitForIndex belongs to the unary code for efIndex
+        highIndex = (int)(setBitForIndex >>> LOG2_LONG_SIZE);
+        upperLong = efEncoder.upperLongs[highIndex];
+        curHighLong = upperLong >>> ((int) (setBitForIndex & (Long.SIZE-1))); // may contain the unary 1 bit for efIndex
+      }
+      assert efIndex < numEncoded; // there is a high value to be found.
+    }
+
+    int curSetBits = Long.bitCount(curHighLong); // shifted right.
+    int curClearBits = Long.SIZE - curSetBits - ((int) (setBitForIndex & (Long.SIZE-1))); // subtract right shift, may be more than encoded
+
+    while (((setBitForIndex - efIndex) + curClearBits) < highTarget) {
       // curHighLong has not enough clear bits to reach highTarget
       efIndex += curSetBits;
       if (efIndex >= numEncoded) {
         return NO_MORE_VALUES;
       }
-      toNextHighLong();
-      // assert getCurrentRightShift() == 0;
+      setBitForIndex += Long.SIZE - (setBitForIndex & (Long.SIZE-1));
+      // highIndex = (int)(setBitForIndex >>> LOG2_LONG_SIZE);
+      assert (highIndex + 1) == (int)(setBitForIndex >>> LOG2_LONG_SIZE);
+      highIndex += 1;
+      upperLong = efEncoder.upperLongs[highIndex];
+      curHighLong = upperLong;
       curSetBits = Long.bitCount(curHighLong);
       curClearBits = Long.SIZE - curSetBits;
     }
-    // curHighLong has enough clear bits to reach highTarget, but may not have enough set bits.
-    long highValue = nextHighValue();
-    while (highValue < highTarget) {
-      /* CHECKME: Instead of the linear search here, use (forward) broadword selection from
-       * "Broadword Implementation of Rank/Select Queries", Sebastiano Vigna, January 30, 2012.
-       */
-      if (! toAfterCurrentHighBit()) {
+    // curHighLong has enough clear bits to reach highTarget, and may not have enough set bits.
+    while (curHighLong == 0L) {
+      setBitForIndex += Long.SIZE - (setBitForIndex & (Long.SIZE-1));
+      assert (highIndex + 1) == (int)(setBitForIndex >>> LOG2_LONG_SIZE);
+      highIndex += 1;
+      upperLong = efEncoder.upperLongs[highIndex];
+      curHighLong = upperLong;
+    }
+
+    // curHighLong has enough clear bits to reach highTarget, has at least 1 set bit, and may not have enough set bits.
+    int rank = (int) (highTarget - (setBitForIndex - efIndex)); // the rank of the zero bit for highValue.
+    assert (rank <= Long.SIZE) : ("rank " + rank);
+    if (rank >= 1) {
+      long invCurHighLong = ~curHighLong;
+      int clearBitForValue = (rank <= 8)
+                              ? BroadWord.selectNaive(invCurHighLong, rank)
+                              : BroadWord.select(invCurHighLong, rank);
+      assert clearBitForValue <= (Long.SIZE-1);
+      setBitForIndex += clearBitForValue + 1; // the high bit just before setBitForIndex is zero
+      int oneBitsBeforeClearBit = clearBitForValue - rank + 1;
+      efIndex += oneBitsBeforeClearBit; // the high bit at setBitForIndex and belongs to the unary code for efIndex
+      if (efIndex >= numEncoded) {
         return NO_MORE_VALUES;
       }
-      highValue = nextHighValue();
-    }
-    return highValue;
-  }
 
-  /** Given a target value, advance the decoding index to the first bigger or equal value
-   * and return it if it is available. Otherwise return {@link #NO_MORE_VALUES}.
-   */
-  public long advanceToValue(long target) {
-    if (! toAfterCurrentHighBit()) {
-      return NO_MORE_VALUES;
-    }
-    long highTarget = target >>> efEncoder.numLowBits;
-    long highValue = advanceToHighValue(highTarget);
-    if (highValue == NO_MORE_VALUES) {
-      return NO_MORE_VALUES;
+      if ((setBitForIndex & (Long.SIZE - 1)) == 0L) { // exhausted curHighLong
+        assert (highIndex + 1) == (int)(setBitForIndex >>> LOG2_LONG_SIZE);
+        highIndex += 1;
+        upperLong = efEncoder.upperLongs[highIndex];
+        curHighLong = upperLong;
+      }
+      else {
+        assert highIndex == (int)(setBitForIndex >>> LOG2_LONG_SIZE);
+        curHighLong = upperLong >>> ((int) (setBitForIndex & (Long.SIZE-1)));
+      }
+      // curHighLong has enough clear bits to reach highTarget, and may not have enough set bits.
+ 
+      while (curHighLong == 0L) {
+        setBitForIndex += Long.SIZE - (setBitForIndex & (Long.SIZE-1));
+        assert (highIndex + 1) == (int)(setBitForIndex >>> LOG2_LONG_SIZE);
+        highIndex += 1;
+        upperLong = efEncoder.upperLongs[highIndex];
+        curHighLong = upperLong;
+      }
     }
-    // Linear search with low values:
-    long currentValue = combineHighLowValues(highValue, currentLowValue());
+    setBitForIndex += Long.numberOfTrailingZeros(curHighLong);
+    assert (setBitForIndex - efIndex) >= highTarget; // highTarget reached
+
+    // Linear search also with low values
+    long currentValue = combineHighLowValues((setBitForIndex - efIndex), currentLowValue());
     while (currentValue < target) {
       currentValue = nextValue();
       if (currentValue == NO_MORE_VALUES) {
@@ -275,7 +366,7 @@ public class EliasFanoDecoder {
     setBitForIndex = (efEncoder.lastEncoded >>> efEncoder.numLowBits) + numEncoded;
   }
 
-  /** Return the number of bits in a long before (setBitForIndex modulo Long.SIZE) */
+  /** @return the number of bits in a long before (setBitForIndex modulo Long.SIZE) */
   private int getCurrentLeftShift() {
     int s = Long.SIZE - 1 - (int) (setBitForIndex & (Long.SIZE-1));
     return s;
@@ -318,7 +409,7 @@ public class EliasFanoDecoder {
     return currentHighValue();
   }
 
-  /** If another value is available before the current decoding index, return this value and
+  /** If another value is available before the current decoding index, return this value
    * and decrease the decoding index by 1. Otherwise return {@link #NO_MORE_VALUES}.
    */
   public long previousValue() {
@@ -333,9 +424,11 @@ public class EliasFanoDecoder {
   /** setBitForIndex and efIndex have just been decremented, scan backward to the high set bit
    *  of at most a given high value
    *  by decrementing setBitForIndex and by setting curHighLong accordingly.
+   * <br>The current implementation does not use the index on the upper zero bit positions.
    *  @return the largest encoded high value that is at most the given one.
    */
   private long backToHighValue(long highTarget) {
+    /* CHECKME: Add using the index as in advanceToHighValue */
     int curSetBits = Long.bitCount(curHighLong); // is shifted by getCurrentLeftShift()
     int curClearBits = Long.SIZE - curSetBits - getCurrentLeftShift();
     while ((currentHighValue() - curClearBits) > highTarget) {
@@ -352,7 +445,7 @@ public class EliasFanoDecoder {
     // curHighLong has enough clear bits to reach highTarget, but may not have enough set bits.
     long highValue = previousHighValue();
     while (highValue > highTarget) {
-      /* CHECKME: See at advanceToHighValue. */
+      /* CHECKME: See at advanceToHighValue on using broadword bit selection. */
       if (! toBeforeCurrentHighBit()) {
         return NO_MORE_VALUES;
       }
@@ -363,6 +456,7 @@ public class EliasFanoDecoder {
 
   /** Given a target value, go back to the first smaller or equal value
    * and return it if it is available. Otherwise return {@link #NO_MORE_VALUES}.
+   * <br>The current implementation does not use the index on the upper zero bit positions.
    */
   public long backToValue(long target) {
     if (! toBeforeCurrentHighBit()) {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/EliasFanoDocIdSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/EliasFanoDocIdSet.java?rev=1532388&r1=1532387&r2=1532388&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/EliasFanoDocIdSet.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/EliasFanoDocIdSet.java Tue Oct 15 15:30:51 2013
@@ -22,21 +22,38 @@ import java.io.IOException;
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
 
+import org.apache.lucene.util.FixedBitSet; // for javadocs
+
 
 /** A DocIdSet in Elias-Fano encoding.
  * @lucene.internal
  */
 public class EliasFanoDocIdSet extends DocIdSet {
   final EliasFanoEncoder efEncoder;
-  /*
-   * Construct an EliasFanoDocIdSet.
-   * @param numValues The number of values that can be encoded.
-   * @param upperBound  At least the highest value that will be encoded.
+
+  /**
+   * Construct an EliasFanoDocIdSet. For efficient encoding, the parameters should be chosen as low as possible.
+   * @param numValues At least the number of document ids that will be encoded.
+   * @param upperBound  At least the highest document id that will be encoded.
    */
   public EliasFanoDocIdSet(int numValues, int upperBound) {
     efEncoder = new EliasFanoEncoder(numValues, upperBound);
   }
 
+  /** Provide an indication that is better to use an {@link EliasFanoDocIdSet} than a {@link FixedBitSet}
+   *  to encode document identifiers.
+   *  @param numValues The number of document identifiers that is to be encoded. Should be non negative.
+   *  @param upperBound The maximum possible value for a document identifier. Should be at least <code>numValues</code>.
+   *  @return See {@link EliasFanoEncoder#sufficientlySmallerThanBitSet(long, long)}
+   */
+  public static boolean sufficientlySmallerThanBitSet(long numValues, long upperBound) {
+    return EliasFanoEncoder.sufficientlySmallerThanBitSet(numValues, upperBound);
+  }
+
+  /** Encode the document ids from a DocIdSetIterator.
+   *  @param disi This DocIdSetIterator should provide document ids that are consistent
+   *              with <code>numValues</code> and <code>upperBound</code> as provided to the constructor.  
+   */
   public void encodeFromDisi(DocIdSetIterator disi) throws IOException {
     while (efEncoder.numEncoded < efEncoder.numValues) {
       int x = disi.nextDoc();
@@ -67,10 +84,10 @@ public class EliasFanoDocIdSet extends D
         return curDocId;
       }
 
-      private int setCurDocID(long nextValue) {
-        curDocId = (nextValue == EliasFanoDecoder.NO_MORE_VALUES)
+      private int setCurDocID(long value) {
+        curDocId = (value == EliasFanoDecoder.NO_MORE_VALUES)
             ?  NO_MORE_DOCS
-                : (int) nextValue;
+                : (int) value;
         return curDocId;
       }
 
@@ -86,12 +103,14 @@ public class EliasFanoDocIdSet extends D
 
       @Override
       public long cost() {
-        return efDecoder.numEncoded;
+        return efDecoder.numEncoded();
       }
     };
   }
 
-  /** This DocIdSet implementation is cacheable. @return <code>true</code> */
+  /** This DocIdSet implementation is cacheable.
+   * @return <code>true</code>
+   */
   @Override
   public boolean isCacheable() {
     return true;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/EliasFanoEncoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/EliasFanoEncoder.java?rev=1532388&r1=1532387&r2=1532388&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/EliasFanoEncoder.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/EliasFanoEncoder.java Tue Oct 15 15:30:51 2013
@@ -19,6 +19,7 @@ package org.apache.lucene.util.packed;
 
 import java.util.Arrays;
 
+import org.apache.lucene.util.ToStringUtils;
 import org.apache.lucene.util.FixedBitSet; // for javadocs
 
 
@@ -64,10 +65,12 @@ import org.apache.lucene.util.FixedBitSe
  * In this implementation the values in the sequence can be given as <code>long</code>,
  * <code>numValues = 0</code> and <code>upperBound = 0</code> are allowed,
  * and each of the upper and lower bit arrays should fit in a <code>long[]</code>.
+ * <br>
+ * An index of positions of zero's in the upper bits is also built.
  * <p>
  * This implementation is based on this article:
  * <br>
- * Sebastiano Vigna, "Quasi Succinct Indices", June 19, 2012, sections 3 and 4.
+ * Sebastiano Vigna, "Quasi Succinct Indices", June 19, 2012, sections 3, 4 and 9.
  * Retrieved from http://arxiv.org/pdf/1206.4300 .
  *
  * <p>The articles originally describing the Elias-Fano representation are:
@@ -91,6 +94,19 @@ public class EliasFanoEncoder {
   long numEncoded = 0L;
   long lastEncoded = 0L;
 
+  /** The default index interval for zero upper bits. */
+  public static final long DEFAULT_INDEX_INTERVAL = 256;
+  final long numIndexEntries;
+  final long indexInterval;
+  final int nIndexEntryBits;
+  /** upperZeroBitPositionIndex[i] (filled using packValue) will contain the bit position
+   *  just after the zero bit ((i+1) * indexInterval) in the upper bits.
+   */
+  final long[] upperZeroBitPositionIndex;
+  long currentEntryIndex; // also indicates how many entries in the index are valid.
+
+
+
   /**
    * Construct an Elias-Fano encoder.
    * After construction, call {@link #encodeNext} <code>numValues</code> times to encode
@@ -101,6 +117,10 @@ public class EliasFanoEncoder {
    *                or is the first higher than the actual maximum.
    *                <br>When <code>numValues >= (upperBound/3)</code>
    *                a {@link FixedBitSet} will take less space.
+   * @param indexInterval The number of high zero bits for which a single index entry is built.
+   *                The index will have at most <code>2 * numValues / indexInterval</code> entries
+   *                and each index entry will use at most <code>ceil(log2(3 * numValues))</code> bits,
+   *                see {@link EliasFanoEncoder}.
    * @throws IllegalArgumentException when:
    *         <ul>
    *         <li><code>numValues</code> is negative, or
@@ -108,10 +128,13 @@ public class EliasFanoEncoder {
    *         <li>the low bits do not fit in a <code>long[]</code>:
    *             <code>(L * numValues / 64) > Integer.MAX_VALUE</code>, or
    *         <li>the high bits do not fit in a <code>long[]</code>:
-   *             <code>(2 * numValues / 64) > Integer.MAX_VALUE</code>.
+   *             <code>(2 * numValues / 64) > Integer.MAX_VALUE</code>, or
+   *         <li><code>indexInterval < 2</code>,
+   *         <li>the index bits do not fit in a <code>long[]</code>:
+   *             <code>(numValues / indexInterval * ceil(2log(3 * numValues)) / 64) > Integer.MAX_VALUE</code>.
    *         </ul>
    */
-  public EliasFanoEncoder(long numValues, long upperBound) {
+  public EliasFanoEncoder(long numValues, long upperBound, long indexInterval) {
     if (numValues < 0L) {
       throw new IllegalArgumentException("numValues should not be negative: " + numValues);
     }
@@ -145,18 +168,42 @@ public class EliasFanoEncoder {
       throw new IllegalArgumentException("numLongsForHighBits too large to index a long array: " + numLongsForHighBits);
     }
     this.upperLongs = new long[(int) numLongsForHighBits];
+    if (indexInterval < 2) {
+      throw new IllegalArgumentException("indexInterval should at least 2: " + indexInterval);
+    }
+    // For the index:
+    long maxHighValue = upperBound >>> this.numLowBits;
+    long nIndexEntries = maxHighValue / indexInterval; // no zero value index entry
+    this.numIndexEntries = (nIndexEntries >= 0) ? nIndexEntries : 0;
+    long maxIndexEntry = maxHighValue + numValues - 1; // clear upper bits, set upper bits, start at zero
+    this.nIndexEntryBits = (maxIndexEntry <= 0) ? 0
+                          : (64 - Long.numberOfLeadingZeros(maxIndexEntry - 1));
+    long numLongsForIndexBits = numLongsForBits(numIndexEntries * nIndexEntryBits);
+    if (numLongsForIndexBits > Integer.MAX_VALUE) {
+      throw new IllegalArgumentException("numLongsForIndexBits too large to index a long array: " + numLongsForIndexBits);
+    }
+    this.upperZeroBitPositionIndex = new long[(int) numLongsForIndexBits];
+    this.currentEntryIndex = 0;
+    this.indexInterval = indexInterval;
   }
 
-  private static long numLongsForBits(long numBits) {
+  /**
+  * Construct an Elias-Fano encoder using {@link #DEFAULT_INDEX_INTERVAL}.
+  */
+  public EliasFanoEncoder(long numValues, long upperBound) {
+    this(numValues, upperBound, DEFAULT_INDEX_INTERVAL);
+  }
+
+  private static long numLongsForBits(long numBits) { // Note: int version in FixedBitSet.bits2words()
     assert numBits >= 0 : numBits;
     return (numBits + (Long.SIZE-1)) >>> LOG2_LONG_SIZE;
   }
 
   /** Call at most <code>numValues</code> times to encode a non decreasing sequence of non negative numbers.
    * @param x The next number to be encoded.
+   * @throws IllegalStateException when called more than <code>numValues</code> times.
    * @throws IllegalArgumentException when:
    *         <ul>
-   *         <li>called more than <code>numValues</code> times, or
    *         <li><code>x</code> is smaller than an earlier encoded value, or
    *         <li><code>x</code> is larger than <code>upperBound</code>.
    *         </ul>
@@ -171,10 +218,19 @@ public class EliasFanoEncoder {
     if (x > upperBound) {
       throw new IllegalArgumentException(x + " larger than upperBound " + upperBound);
     }
-    encodeUpperBits(x >>> numLowBits);
+    long highValue = x >>> numLowBits;
+    encodeUpperBits(highValue);
     encodeLowerBits(x & lowerBitsMask);
-    numEncoded++;
     lastEncoded = x;
+    // Add index entries:
+    long indexValue = (currentEntryIndex + 1) * indexInterval;
+    while (indexValue <= highValue) { 
+      long afterZeroBitPosition = indexValue + numEncoded;
+      packValue(afterZeroBitPosition, upperZeroBitPositionIndex, nIndexEntryBits, currentEntryIndex);
+      currentEntryIndex += 1;
+      indexValue += indexInterval;
+    }
+    numEncoded++;
   }
 
   private void encodeUpperBits(long highValue) {
@@ -198,25 +254,28 @@ public class EliasFanoEncoder {
     }
   }
 
-  /** Provide an indication that is better to use an {@link EliasFanoEncoder} than a {@link FixedBitSet}
+  /** Provide an indication that it is better to use an {@link EliasFanoEncoder} than a {@link FixedBitSet}
    *  to encode document identifiers.
    *  This indication is not precise and may change in the future.
    *  <br>An EliasFanoEncoder is favoured when the size of the encoding by the EliasFanoEncoder
-   *  is at most 5/6 of the size of the FixedBitSet.
-   *  <br>This condition is the same as comparing estimates of the number of bits accessed by a pair of FixedBitSets and
+   *  (including some space for its index) is at most about 5/6 of the size of the FixedBitSet,
+   *  this is the same as comparing estimates of the number of bits accessed by a pair of FixedBitSets and
    *  by a pair of non indexed EliasFanoDocIdSets when determining the intersections of the pairs.
+   *  <br>A bit set is preferred when <code>upperbound <= 256</code>.
+   *  <br>It is assumed that {@link #DEFAULT_INDEX_INTERVAL} is used.
    *  @param numValues The number of document identifiers that is to be encoded. Should be non negative.
-   *  @param upperBound The maximum possible value for a document identifier. Should be at least numValues.
+   *  @param upperBound The maximum possible value for a document identifier. Should be at least <code>numValues</code>.
    */
   public static boolean sufficientlySmallerThanBitSet(long numValues, long upperBound) {
     /* When (upperBound / 6) == numValues,
      * the number of bits per entry for the EliasFanoEncoder is 2 + ceil(2log(upperBound/numValues)) == 5.
-     */
-    /* For intersecting two bit sets upperBound bits are accessed, roughly half of one, half of the other.
+     *
+     * For intersecting two bit sets upperBound bits are accessed, roughly half of one, half of the other.
      * For intersecting two EliasFano sequences without index on the upper bits,
      * all (2 * 3 * numValues) upper bits are accessed.
      */
-    return (upperBound / 6) > numValues;
+    return (upperBound > (4 * Long.SIZE)) // prefer a bit set when it takes no more than 4 longs.
+            && (upperBound / 7) > numValues; // 6 + 1 to allow some room for the index.
   }
 
   /**
@@ -237,6 +296,11 @@ public class EliasFanoEncoder {
   public long[] getUpperBits() {
     return upperLongs;
   }
+  
+  /** Expert. The index bits. */
+  public long[] getIndexBits() {
+    return upperZeroBitPositionIndex;
+  }
 
   @Override
   public String toString() {
@@ -248,11 +312,16 @@ public class EliasFanoEncoder {
     s.append(" numLowBits " + numLowBits);
     s.append("\nupperLongs[" + upperLongs.length + "]");
     for (int i = 0; i < upperLongs.length; i++) {
-      s.append(" " + longHex(upperLongs[i]));
+      s.append(" " + ToStringUtils.longHex(upperLongs[i]));
     }
     s.append("\nlowerLongs[" + lowerLongs.length + "]");
     for (int i = 0; i < lowerLongs.length; i++) {
-      s.append(" " + longHex(lowerLongs[i]));
+      s.append(" " + ToStringUtils.longHex(lowerLongs[i]));
+    }
+    s.append("\nindexInterval: " + indexInterval + ", nIndexEntryBits: " + nIndexEntryBits);
+    s.append("\nupperZeroBitPositionIndex[" + upperZeroBitPositionIndex.length + "]");
+    for (int i = 0; i < upperZeroBitPositionIndex.length; i++) { 
+      s.append(" " + ToStringUtils.longHex(upperZeroBitPositionIndex[i]));
     }
     return s.toString();
   }
@@ -267,29 +336,19 @@ public class EliasFanoEncoder {
     return (this.numValues == oefs.numValues)
         && (this.numEncoded == oefs.numEncoded)
         && (this.numLowBits == oefs.numLowBits)
+        && (this.numIndexEntries == oefs.numIndexEntries)
+        && (this.indexInterval == oefs.indexInterval) // no need to check index content
         && Arrays.equals(this.upperLongs, oefs.upperLongs)
         && Arrays.equals(this.lowerLongs, oefs.lowerLongs);
   }
 
   @Override
   public int hashCode() {
-    int h = ((int) (numValues + numEncoded))
-        ^ numLowBits
-        ^ Arrays.hashCode(upperLongs)
-        ^ Arrays.hashCode(lowerLongs);
+    int h = ((int) (31*(numValues + 7*(numEncoded + 5*(numLowBits + 3*(numIndexEntries + 11*indexInterval))))))
+            ^ Arrays.hashCode(upperLongs)
+            ^ Arrays.hashCode(lowerLongs);
     return h;
   }
 
-  public static String longHex(long x) {
-    String hx = Long.toHexString(x);
-    StringBuilder sb = new StringBuilder("0x");
-    int l = 16 - hx.length();
-    while (l > 0) {
-      sb.append('0');
-      l--;
-    }
-    sb.append(hx);
-    return sb.toString();
-  }
 }
 

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestBroadWord.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestBroadWord.java?rev=1532388&r1=1532387&r2=1532388&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestBroadWord.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestBroadWord.java Tue Oct 15 15:30:51 2013
@@ -20,7 +20,7 @@ package org.apache.lucene.util;
 
 public class TestBroadWord extends LuceneTestCase {
   private void tstRank(long x) {
-    assertEquals("rank9(" + x + ")", Long.bitCount(x), BroadWord.rank9(x));
+    assertEquals("rank(" + x + ")", Long.bitCount(x), BroadWord.bitCount(x));
   }
 
   public void testRank1() {
@@ -34,7 +34,7 @@ public class TestBroadWord extends Lucen
 
   private void tstSelect(long x, int r, int exp) {
     assertEquals("selectNaive(" + x + "," + r + ")", exp, BroadWord.selectNaive(x, r));
-    assertEquals("select9(" + x + "," + r + ")", exp, BroadWord.select9(x, r));
+    assertEquals("select(" + x + "," + r + ")", exp, BroadWord.select(x, r));
   }
 
   public void testSelectFromZero() {
@@ -77,7 +77,7 @@ public class TestBroadWord extends Lucen
   public void testPerfSelectAllBitsBroad() {
     for (int j = 0; j < 100000; j++) { // 1000000 for real perf test
       for (int i = 0; i < 64; i++) {
-        assertEquals(i, BroadWord.select9(0xFFFFFFFFFFFFFFFFL, i+1));
+        assertEquals(i, BroadWord.select(0xFFFFFFFFFFFFFFFFL, i+1));
       }
     }
   }

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/packed/TestEliasFanoSequence.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/packed/TestEliasFanoSequence.java?rev=1532388&r1=1532387&r2=1532388&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/packed/TestEliasFanoSequence.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/packed/TestEliasFanoSequence.java Tue Oct 15 15:30:51 2013
@@ -21,13 +21,13 @@ import org.apache.lucene.util.LuceneTest
 
 public class TestEliasFanoSequence extends LuceneTestCase {
 
-  private static EliasFanoEncoder makeEncoder(long[] values) {
+  private static EliasFanoEncoder makeEncoder(long[] values, long indexInterval) {
     long upperBound = -1L;
     for (long value: values) {
       assertTrue(value >= upperBound); // test data ok
       upperBound = value;
     }
-    EliasFanoEncoder efEncoder = new EliasFanoEncoder(values.length, upperBound);
+    EliasFanoEncoder efEncoder = new EliasFanoEncoder(values.length, upperBound, indexInterval);
     for (long value: values) {
       efEncoder.encodeNext(value);
     }
@@ -64,13 +64,13 @@ public class TestEliasFanoSequence exten
         long advanceValue = efd.advanceToValue(expValue);
         assertFalse("advanceValue at end too early", EliasFanoDecoder.NO_MORE_VALUES == advanceValue);
         assertEquals(expValue, advanceValue);
-        assertEquals(index, efd.index());
+        assertEquals(index, efd.currentIndex());
         previousValue = expValue;
       }
       index++;
     }
     long advanceValue = efd.advanceToValue(previousValue+1);
-    assertEquals(EliasFanoDecoder.NO_MORE_VALUES, advanceValue);
+    assertEquals("at end", EliasFanoDecoder.NO_MORE_VALUES, advanceValue);
   }
 
   private static void tstDecodeAdvanceToMultiples(long[] values, EliasFanoDecoder efd, final long m) {
@@ -86,7 +86,7 @@ public class TestEliasFanoSequence exten
         long advanceValue = efd.advanceToValue(mm);
         assertFalse("advanceValue at end too early", EliasFanoDecoder.NO_MORE_VALUES == advanceValue);
         assertEquals(expValue, advanceValue);
-        assertEquals(index, efd.index());
+        assertEquals(index, efd.currentIndex());
         previousValue = expValue;
         do {
           mm += m;
@@ -118,7 +118,7 @@ public class TestEliasFanoSequence exten
         long backValue = efd.backToValue(mm);
         assertFalse("backToValue at end too early", EliasFanoDecoder.NO_MORE_VALUES == backValue);
         assertEquals(expValue, backValue);
-        assertEquals(index, efd.index());
+        assertEquals(index, efd.currentIndex());
         previousValue = expValue;
         do {
           mm -= m;
@@ -146,25 +146,31 @@ public class TestEliasFanoSequence exten
   }
 
   private static void tstEFS(long[] values, long[] expHighLongs, long[] expLowLongs) {
-    EliasFanoEncoder efEncoder = makeEncoder(values);
+    EliasFanoEncoder efEncoder = makeEncoder(values, EliasFanoEncoder.DEFAULT_INDEX_INTERVAL);
     tstEqual("upperBits", expHighLongs, efEncoder.getUpperBits());
     tstEqual("lowerBits", expLowLongs, efEncoder.getLowerBits());
     tstDecodeAll(efEncoder, values);
   }
 
   private static void tstEFS2(long[] values) {
-    EliasFanoEncoder efEncoder = makeEncoder(values);
+    EliasFanoEncoder efEncoder = makeEncoder(values, EliasFanoEncoder.DEFAULT_INDEX_INTERVAL);
     tstDecodeAll(efEncoder, values);
   }
 
   private static void tstEFSadvanceToAndBackToMultiples(long[] values, long maxValue, long minAdvanceMultiple) {
-    EliasFanoEncoder efEncoder = makeEncoder(values);
+    EliasFanoEncoder efEncoder = makeEncoder(values, EliasFanoEncoder.DEFAULT_INDEX_INTERVAL);
     for (long m = minAdvanceMultiple; m <= maxValue; m += 1) {
       tstDecodeAdvanceToMultiples(values, efEncoder.getDecoder(), m);
       tstDecodeBackToMultiples(values, efEncoder.getDecoder(), m);
     }
   }
 
+  private EliasFanoEncoder tstEFVI(long[] values, long indexInterval, long[] expIndexBits) {
+    EliasFanoEncoder efEncVI = makeEncoder(values, indexInterval);
+    tstEqual("upperZeroBitPositionIndex", expIndexBits, efEncVI.getIndexBits());
+    return efEncVI;
+  }
+
   public void testEmpty() {
     long[] values = new long[0];
     long[] expHighBits = new long[0];
@@ -223,29 +229,31 @@ public class TestEliasFanoSequence exten
 
   public void testHashCodeEquals() {
     long[] values = new long[] {5,8,8,15,32};
-    EliasFanoEncoder efEncoder1 = makeEncoder(values);
-    EliasFanoEncoder efEncoder2 = makeEncoder(values);
+    EliasFanoEncoder efEncoder1 = makeEncoder(values, EliasFanoEncoder.DEFAULT_INDEX_INTERVAL);
+    EliasFanoEncoder efEncoder2 = makeEncoder(values, EliasFanoEncoder.DEFAULT_INDEX_INTERVAL);
     assertEquals(efEncoder1, efEncoder2);
     assertEquals(efEncoder1.hashCode(), efEncoder2.hashCode());
 
-    EliasFanoEncoder efEncoder3 = makeEncoder(new long[] {1,2,3});
+    EliasFanoEncoder efEncoder3 = makeEncoder(new long[] {1,2,3}, EliasFanoEncoder.DEFAULT_INDEX_INTERVAL);
     assertFalse(efEncoder1.equals(efEncoder3));
     assertFalse(efEncoder3.equals(efEncoder1));
     assertFalse(efEncoder1.hashCode() == efEncoder3.hashCode()); // implementation ok for these.
   }
 
   public void testMonotoneSequences() {
-    for (int s = 2; s < 1222; s++) {
+    //for (int s = 2; s < 1222; s++) {
+    for (int s = 2; s < 4422; s++) {
       long[] values = new long[s];
       for (int i = 0; i < s; i++) {
-        values[i] = (i/2);
+        values[i] = (i/2); // upperbound smaller than number of values, only upper bits encoded
       }
       tstEFS2(values);
     }
   }
 
   public void testStrictMonotoneSequences() {
-    for (int s = 2; s < 1222; s++) {
+    // for (int s = 2; s < 1222; s++) {
+    for (int s = 2; s < 4422; s++) {
       long[] values = new long[s];
       for (int i = 0; i < s; i++) {
         values[i] = i * ((long) i - 1) / 2; // Add a gap of (s-1) to previous
@@ -277,5 +285,98 @@ public class TestEliasFanoSequence exten
       tstEFSadvanceToAndBackToMultiples(values, values[s-1], 10);
     }
   }
+
+  public void testEmptyIndex() {
+    long indexInterval = 2;
+    long[] emptyLongs = new long[0];
+    tstEFVI(emptyLongs, indexInterval, emptyLongs);
+  }
+  public void testMaxContentEmptyIndex() {
+    long indexInterval = 2;
+    long[] twoLongs = new long[] {0,1};
+    long[] emptyLongs = new long[0];
+    tstEFVI(twoLongs, indexInterval, emptyLongs);
+  }
+
+  public void testMinContentNonEmptyIndex() {
+    long indexInterval = 2;
+    long[] twoLongs = new long[] {0,2};
+    long[] indexLongs = new long[] {3}; // high bits 1001, index position after zero bit.
+    tstEFVI(twoLongs, indexInterval, indexLongs);
+  }
+
+  public void testIndexAdvanceToLast() {
+    long indexInterval = 2;
+    long[] twoLongs = new long[] {0,2};
+    long[] indexLongs = new long[] {3}; // high bits 1001
+    EliasFanoEncoder efEncVI = tstEFVI(twoLongs, indexInterval, indexLongs);
+    assertEquals(2, efEncVI.getDecoder().advanceToValue(2));
+  }
+
+  public void testIndexAdvanceToAfterLast() {
+    long indexInterval = 2;
+    long[] twoLongs = new long[] {0,2};
+    long[] indexLongs = new long[] {3}; // high bits 1001
+    EliasFanoEncoder efEncVI = tstEFVI(twoLongs, indexInterval, indexLongs);
+    assertEquals(EliasFanoDecoder.NO_MORE_VALUES, efEncVI.getDecoder().advanceToValue(3));
+  }
+
+  public void testIndexAdvanceToFirst() {
+    long indexInterval = 2;
+    long[] twoLongs = new long[] {0,2};
+    long[] indexLongs = new long[] {3}; // high bits 1001
+    EliasFanoEncoder efEncVI = tstEFVI(twoLongs, indexInterval, indexLongs);
+    assertEquals(0, efEncVI.getDecoder().advanceToValue(0));
+  }
+  
+  public void testTwoIndexEntries() {
+    long indexInterval = 2;
+    long[] twoLongs = new long[] {0,1,2,3,4,5};
+    long[] indexLongs = new long[] {4 + 8*16}; // high bits 0b10101010101
+    EliasFanoEncoder efEncVI = tstEFVI(twoLongs, indexInterval, indexLongs);
+    EliasFanoDecoder efDecVI = efEncVI.getDecoder();
+    assertEquals("advance 0", 0, efDecVI.advanceToValue(0));
+    assertEquals("advance 5", 5, efDecVI.advanceToValue(5));
+    assertEquals("advance 6", EliasFanoDecoder.NO_MORE_VALUES, efDecVI.advanceToValue(5));
+  }
+
+  public void testExample2a() { // Figure 2 from Vigna 2012 paper
+    long indexInterval = 4;
+    long[] values = new long[] {5,8,8,15,32}; // two low bits, high values 1,2,2,3,8.
+    long[] indexLongs = new long[] {8 + 12*16}; // high bits 0b 0001 0000 0101 1010
+    EliasFanoEncoder efEncVI = tstEFVI(values, indexInterval, indexLongs);
+    EliasFanoDecoder efDecVI = efEncVI.getDecoder();
+    assertEquals("advance 22", 32, efDecVI.advanceToValue(22));
+  }
+
+  public void testExample2b() { // Figure 2 from Vigna 2012 paper
+    long indexInterval = 4;
+    long[] values = new long[] {5,8,8,15,32}; // two low bits, high values 1,2,2,3,8.
+    long[] indexLongs = new long[] {8 + 12*16}; // high bits 0b 0001 0000 0101 1010
+    EliasFanoEncoder efEncVI = tstEFVI(values, indexInterval, indexLongs);
+    EliasFanoDecoder efDecVI = efEncVI.getDecoder();
+    assertEquals("initial next", 5, efDecVI.nextValue());
+    assertEquals("advance 22", 32, efDecVI.advanceToValue(22));
+  }
+
+  public void testExample2NoIndex1() { // Figure 2 from Vigna 2012 paper, no index, test broadword selection.
+    long indexInterval = 16;
+    long[] values = new long[] {5,8,8,15,32}; // two low bits, high values 1,2,2,3,8.
+    long[] indexLongs = new long[0]; // high bits 0b 0001 0000 0101 1010
+    EliasFanoEncoder efEncVI = tstEFVI(values, indexInterval, indexLongs);
+    EliasFanoDecoder efDecVI = efEncVI.getDecoder();
+    assertEquals("advance 22", 32, efDecVI.advanceToValue(22));
+  }
+
+  public void testExample2NoIndex2() { // Figure 2 from Vigna 2012 paper, no index, test broadword selection.
+    long indexInterval = 16;
+    long[] values = new long[] {5,8,8,15,32}; // two low bits, high values 1,2,2,3,8.
+    long[] indexLongs = new long[0]; // high bits 0b 0001 0000 0101 1010
+    EliasFanoEncoder efEncVI = tstEFVI(values, indexInterval, indexLongs);
+    EliasFanoDecoder efDecVI = efEncVI.getDecoder();
+    assertEquals("initial next", 5, efDecVI.nextValue());
+    assertEquals("advance 22", 32, efDecVI.advanceToValue(22));
+  }
+
 }