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 2012/07/04 10:17:04 UTC

svn commit: r1357159 [4/4] - in /lucene/dev/trunk/lucene/core/src: java/org/apache/lucene/codecs/ java/org/apache/lucene/codecs/lucene40/values/ java/org/apache/lucene/util/ java/org/apache/lucene/util/packed/ test/org/apache/lucene/util/packed/

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/gen_Direct.py
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/gen_Direct.py?rev=1357159&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/gen_Direct.py (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/gen_Direct.py Wed Jul  4 08:17:02 2012
@@ -0,0 +1,175 @@
+#! /usr/bin/env python
+
+# 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.
+
+HEADER="""// This file has been automatically generated, DO NOT EDIT
+
+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.store.DataInput;
+import org.apache.lucene.util.RamUsageEstimator;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+"""
+
+TYPES = {8: "byte", 16: "short", 32: "int", 64: "long"}
+MASKS = {8: " & 0xFFL", 16: " & 0xFFFFL", 32: " & 0xFFFFFFFFL", 64: ""}
+CASTS = {8: "(byte) ", 16: "(short) ", 32: "(int) ", 64: ""}
+
+if __name__ == '__main__':
+  for bpv in TYPES.keys():
+    type
+    f = open("Direct%d.java" %bpv, 'w')
+    f.write(HEADER)
+    f.write("""/**
+ * Direct wrapping of %d-bits values to a backing array.
+ * @lucene.internal
+ */\n""" %bpv)
+    f.write("final class Direct%d extends PackedInts.MutableImpl {\n" %bpv)
+    f.write("  final %s[] values;\n\n" %TYPES[bpv])
+
+    f.write("  Direct%d(int valueCount) {\n" %bpv)
+    f.write("    super(valueCount, %d);\n" %bpv)
+    f.write("    values = new %s[valueCount];\n" %TYPES[bpv])
+    f.write("  }\n\n")
+
+    f.write("  Direct%d(DataInput in, int valueCount) throws IOException {\n" %bpv)
+    f.write("    this(valueCount);\n")
+    f.write("    for (int i = 0; i < valueCount; ++i) {\n")
+    f.write("      values[i] = in.read%s();\n" %TYPES[bpv].title())
+    f.write("    }\n")
+    if bpv != 64:
+      f.write("    final int mod = valueCount %% %d;\n" %(64 / bpv))
+      f.write("    if (mod != 0) {\n")
+      f.write("      for (int i = mod; i < %d; ++i) {\n" %(64 / bpv))
+      f.write("        in.read%s();\n" %TYPES[bpv].title())
+      f.write("      }\n")
+      f.write("    }\n")
+    f.write("  }\n")
+
+    f.write("""
+  @Override
+  public long get(final int index) {
+    return values[index]%s;
+  }
+
+  public void set(final int index, final long value) {
+    values[index] = %s(value);
+  }
+
+  public long ramBytesUsed() {
+    return RamUsageEstimator.sizeOf(values);
+  }
+
+  public void clear() {
+    Arrays.fill(values, %s0L);
+  }
+
+  @Override
+  public Object getArray() {
+    return values;
+  }
+
+  @Override
+  public boolean hasArray() {
+    return true;
+  }
+""" %(MASKS[bpv], CASTS[bpv], CASTS[bpv]))
+
+    if bpv == 64:
+      f.write("""
+  @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;
+    assert off + len <= arr.length;
+
+    final int gets = Math.min(valueCount - index, len);
+    System.arraycopy(values, index, arr, off, gets);
+    return gets;
+  }
+
+  public int set(int index, long[] arr, int off, int len) {
+    assert len > 0 : "len must be > 0 (got " + len + ")";
+    assert index >= 0 && index < valueCount;
+    assert off + len <= arr.length;
+
+    final int sets = Math.min(valueCount - index, len);
+    System.arraycopy(arr, off, values, index, sets);
+    return sets;
+  }
+
+  @Override
+  public void fill(int fromIndex, int toIndex, long val) {
+    Arrays.fill(values, fromIndex, toIndex, val);
+  }
+""")
+    else:
+      f.write("""
+  @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;
+    assert off + len <= arr.length;
+
+    final int gets = Math.min(valueCount - index, len);
+    for (int i = index, o = off, end = index + gets; i < end; ++i, ++o) {
+      arr[o] = values[i]%s;
+    }
+    return gets;
+  }
+
+  public int set(int index, long[] arr, int off, int len) {
+    assert len > 0 : "len must be > 0 (got " + len + ")";
+    assert index >= 0 && index < valueCount;
+    assert off + len <= arr.length;
+
+    final int sets = Math.min(valueCount - index, len);
+    for (int i = index, o = off, end = index + sets; i < end; ++i, ++o) {
+      values[i] = %sarr[o];
+    }
+    return sets;
+  }
+
+  @Override
+  public void fill(int fromIndex, int toIndex, long val) {
+    assert val == (val%s);
+    Arrays.fill(values, fromIndex, toIndex, %sval);
+  }
+""" %(MASKS[bpv], CASTS[bpv], MASKS[bpv], CASTS[bpv]))
+
+    f.write("}\n")
+
+    f.close()

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/gen_Packed64SingleBlock.py
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/gen_Packed64SingleBlock.py?rev=1357159&r1=1357158&r2=1357159&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/gen_Packed64SingleBlock.py (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/gen_Packed64SingleBlock.py Wed Jul  4 08:17:02 2012
@@ -58,12 +58,6 @@ abstract class Packed64SingleBlock exten
     return Arrays.binarySearch(SUPPORTED_BITS_PER_VALUE, bitsPerValue) >= 0;
   }
 
-  public static float overheadPerValue(int bitsPerValue) {
-    int valuesPerBlock = 64 / bitsPerValue;
-    int overhead = 64 %% bitsPerValue;
-    return (float) overhead / valuesPerBlock;
-  }
-
   private static int requiredCapacity(int valueCount, int valuesPerBlock) {
     return valueCount / valuesPerBlock
         + (valueCount %% valuesPerBlock == 0 ? 0 : 1);
@@ -111,16 +105,14 @@ abstract class Packed64SingleBlock exten
 
     // bulk get
     assert index %% valuesPerBlock == 0;
-    final long readMask = (1L << bitsPerValue) - 1;
-    final int startBlock = index / valuesPerBlock;
-    final int endBlock = (index + len) / valuesPerBlock;
-    final int diff = (endBlock - startBlock) * valuesPerBlock;
+    final BulkOperation op = BulkOperation.of(PackedInts.Format.PACKED_SINGLE_BLOCK, bitsPerValue);
+    assert op.blocks() == 1;
+    assert op.values() == valuesPerBlock;
+    final int blockIndex = index / valuesPerBlock;
+    final int nblocks = (index + len) / valuesPerBlock - blockIndex;
+    op.get(blocks, blockIndex, arr, off, nblocks);
+    final int diff = nblocks * valuesPerBlock;
     index += diff; len -= diff;
-    for (int block = startBlock; block < endBlock; ++block) {
-      for (int i = 0; i < valuesPerBlock; ++i) {
-        arr[off++] = (blocks[block] >>> (i * bitsPerValue)) & readMask;
-      }
-    }
 
     if (index > originalIndex) {
       // stay at the block boundary
@@ -157,17 +149,14 @@ abstract class Packed64SingleBlock exten
 
     // bulk set
     assert index %% valuesPerBlock == 0;
-    final int startBlock = index / valuesPerBlock;
-    final int endBlock = (index + len) / valuesPerBlock;
-    final int diff = (endBlock - startBlock) * valuesPerBlock;
+    final BulkOperation op = BulkOperation.of(PackedInts.Format.PACKED_SINGLE_BLOCK, bitsPerValue);
+    assert op.blocks() == 1;
+    assert op.values() == valuesPerBlock;
+    final int blockIndex = index / valuesPerBlock;
+    final int nblocks = (index + len) / valuesPerBlock - blockIndex;
+    op.set(blocks, blockIndex, arr, off, nblocks);
+    final int diff = nblocks * valuesPerBlock;
     index += diff; len -= diff;
-    for (int block = startBlock; block < endBlock; ++block) {
-      long next = 0L;
-      for (int i = 0; i < valuesPerBlock; ++i) {
-        next |= (arr[off++] << (i * bitsPerValue));
-      }
-      blocks[block] = next;
-    }
 
     if (index > originalIndex) {
       // stay at the block boundary
@@ -221,8 +210,8 @@ abstract class Packed64SingleBlock exten
   }
 
   @Override
-  protected int getFormat() {
-    return PackedInts.PACKED_SINGLE_BLOCK;
+  protected PackedInts.Format getFormat() {
+    return PackedInts.Format.PACKED_SINGLE_BLOCK;
   }
 
   @Override

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/gen_PackedThreeBlocks.py
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/gen_PackedThreeBlocks.py?rev=1357159&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/gen_PackedThreeBlocks.py (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/gen_PackedThreeBlocks.py Wed Jul  4 08:17:02 2012
@@ -0,0 +1,161 @@
+#! /usr/bin/env python
+
+# 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.
+
+HEADER="""// This file has been automatically generated, DO NOT EDIT
+
+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.store.DataInput;
+import org.apache.lucene.util.RamUsageEstimator;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+"""
+
+TYPES = {8: "byte", 16: "short"}
+MASKS = {8: " & 0xFFL", 16: " & 0xFFFFL", 32: " & 0xFFFFFFFFL", 64: ""}
+CASTS = {8: "(byte) ", 16: "(short) ", 32: "(int) ", 64: ""}
+
+if __name__ == '__main__':
+  for bpv in TYPES.keys():
+    type
+    f = open("Packed%dThreeBlocks.java" %bpv, 'w')
+    f.write(HEADER)
+    f.write("""/**
+ * Packs integers into 3 %ss (%d bits per value).
+ * @lucene.internal
+ */\n""" %(TYPES[bpv], bpv*3))
+    f.write("final class Packed%dThreeBlocks extends PackedInts.MutableImpl {\n" %bpv)
+    f.write("  final %s[] blocks;\n\n" %TYPES[bpv])
+
+    f.write("  public static final int MAX_SIZE = Integer.MAX_VALUE / 3;\n\n")
+
+    f.write("  Packed%dThreeBlocks(int valueCount) {\n" %bpv)
+    f.write("    super(valueCount, %d);\n" %(bpv*3))
+    f.write("    if (valueCount > MAX_SIZE) {\n")
+    f.write("      throw new ArrayIndexOutOfBoundsException(\"MAX_SIZE exceeded\");\n")
+    f.write("    }\n")
+    f.write("    blocks = new %s[valueCount * 3];\n" %TYPES[bpv])
+    f.write("  }\n\n")
+
+    f.write("  Packed%dThreeBlocks(DataInput in, int valueCount) throws IOException {\n" %bpv)
+    f.write("    this(valueCount);\n")
+    f.write("    for (int i = 0; i < 3 * valueCount; ++i) {\n")
+    f.write("      blocks[i] = in.read%s();\n" %TYPES[bpv].title())
+    f.write("    }\n")
+    f.write("    final int mod = blocks.length %% %d;\n" %(64 / bpv))
+    f.write("    if (mod != 0) {\n")
+    f.write("      for (int i = mod; i < %d; ++i) {\n" %(64 / bpv))
+    f.write("         in.read%s();\n" %TYPES[bpv].title())
+    f.write("      }\n")
+    f.write("    }\n")
+    f.write("  }\n")
+
+    f.write("""
+  @Override
+  public long get(int index) {
+    final int o = index * 3;
+    return (blocks[o]%s) << %d | (blocks[o+1]%s) << %d | (blocks[o+2]%s);
+  }
+
+  @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;
+    assert off + len <= arr.length;
+
+    final int gets = Math.min(valueCount - index, len);
+    for (int i = index * 3, end = (index + gets) * 3; i < end; i+=3) {
+      arr[off++] = (blocks[i]%s) << %d | (blocks[i+1]%s) << %d | (blocks[i+2]%s);
+    }
+    return gets;
+  }
+
+  @Override
+  public void set(int index, long value) {
+    final int o = index * 3;
+    blocks[o] = %s(value >>> %d);
+    blocks[o+1] = %s(value >>> %d);
+    blocks[o+2] = %svalue;
+  }
+
+  @Override
+  public int set(int index, long[] arr, int off, int len) {
+    assert len > 0 : "len must be > 0 (got " + len + ")";
+    assert index >= 0 && index < valueCount;
+    assert off + len <= arr.length;
+
+    final int sets = Math.min(valueCount - index, len);
+    for (int i = off, o = index * 3, end = off + sets; i < end; ++i) {
+      final long value = arr[i];
+      blocks[o++] = %s(value >>> %d);
+      blocks[o++] = %s(value >>> %d);
+      blocks[o++] = %svalue;
+    }
+    return sets;
+  }
+
+  @Override
+  public void fill(int fromIndex, int toIndex, long val) {
+    final %s block1 = %s(val >>> %d);
+    final %s block2 = %s(val >>> %d);
+    final %s block3 = %sval;
+    for (int i = fromIndex * 3, end = toIndex * 3; i < end; i += 3) {
+      blocks[i] = block1;
+      blocks[i+1] = block2;
+      blocks[i+2] = block3;
+    }
+  }
+
+  @Override
+  public void clear() {
+    Arrays.fill(blocks, %s0);
+  }
+
+  public long ramBytesUsed() {
+    return RamUsageEstimator.sizeOf(blocks);
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(bitsPerValue=" + bitsPerValue
+        + ", size=" + size() + ", elements.length=" + blocks.length + ")";
+  }
+}
+""" %(MASKS[bpv], 2*bpv, MASKS[bpv], bpv, MASKS[bpv], MASKS[bpv], 2*bpv, MASKS[bpv], bpv, MASKS[bpv], CASTS[bpv], 2*bpv, CASTS[bpv], bpv, CASTS[bpv], CASTS[bpv],
+      2*bpv, CASTS[bpv], bpv, CASTS[bpv], TYPES[bpv], CASTS[bpv], 2*bpv, TYPES[bpv],
+      CASTS[bpv], bpv, TYPES[bpv], CASTS[bpv], CASTS[bpv]))
+
+    f.close()

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=1357159&r1=1357158&r2=1357159&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 Wed Jul  4 08:17:02 2012
@@ -19,9 +19,16 @@
 <head></head>
 <body bgcolor="white">
 
+<p>Packed integer arrays and streams.</p>
+
 <p>
-    The packed package provides random access capable arrays of positive longs.
-    The implementations provides different trade offs between memory usage and
+    The packed package provides
+    <ul>
+    	<li>sequential and random access capable arrays of positive longs,</li>
+    	<li>routines for efficient serialization and deserialization of streams of packed integers.</li>
+    </ul>
+
+    The implementations provide different trade-offs between memory usage and
     access speed. The standard usage scenario is replacing large int or long
     arrays in order to reduce the memory footprint.
 </p><p>

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=1357159&r1=1357158&r2=1357159&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 Wed Jul  4 08:17:02 2012
@@ -24,6 +24,7 @@ import java.util.Random;
 
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.store.*;
+import org.apache.lucene.util.LongsRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util._TestUtil;
 import org.apache.lucene.util.LuceneTestCase.Slow;
@@ -57,6 +58,9 @@ public class TestPackedInts extends Luce
       for(int nbits=1;nbits<=64;nbits++) {
         final long maxValue = PackedInts.maxValue(nbits);
         final int valueCount = 100+random().nextInt(500);
+        final int bufferSize = random().nextBoolean()
+            ? _TestUtil.nextInt(random(), 0, 48)
+            : _TestUtil.nextInt(random(), 0, 4096);
         final Directory d = newDirectory();
         
         IndexOutput out = d.createOutput("out.bin", newIOContext(random()));
@@ -79,10 +83,10 @@ public class TestPackedInts extends Luce
         // ensure that finish() added the (valueCount-actualValueCount) missing values
         final long bytes;
         switch (w.getFormat()) {
-          case PackedInts.PACKED:
+          case PACKED:
             bytes = (long) Math.ceil((double) valueCount * w.bitsPerValue / 64) << 3;
             break;
-          case PackedInts.PACKED_SINGLE_BLOCK:
+          case PACKED_SINGLE_BLOCK:
             final int valuesPerBlock = 64 / w.bitsPerValue;
             bytes = (long) Math.ceil((double) valueCount / valuesPerBlock) << 3;
             break;
@@ -97,7 +101,7 @@ public class TestPackedInts extends Luce
           CodecUtil.checkHeader(in, PackedInts.CODEC_NAME, PackedInts.VERSION_START, PackedInts.VERSION_CURRENT); // codec header
           assertEquals(w.bitsPerValue, in.readVInt());
           assertEquals(valueCount, in.readVInt());
-          assertEquals(w.getFormat(), in.readVInt());
+          assertEquals(w.getFormat().getId(), in.readVInt());
           assertEquals(startFp, in.getFilePointer());
           in.close();
         }
@@ -113,37 +117,34 @@ public class TestPackedInts extends Luce
           }
           in.close();
         }
+
         { // test reader iterator next
           IndexInput in = d.openInput("out.bin", newIOContext(random()));
-          PackedInts.ReaderIterator r = PackedInts.getReaderIterator(in);
+          PackedInts.ReaderIterator r = PackedInts.getReaderIterator(in, bufferSize);
           for(int i=0;i<valueCount;i++) {
             assertEquals("index=" + i + " valueCount="
                     + valueCount + " nbits=" + nbits + " for "
                     + r.getClass().getSimpleName(), values[i], r.next());
+            assertEquals(i, r.ord());
           }
           assertEquals(fp, in.getFilePointer());
           in.close();
         }
-        { // test reader iterator next vs. advance
+
+        { // test reader iterator bulk next
           IndexInput in = d.openInput("out.bin", newIOContext(random()));
-          PackedInts.ReaderIterator intsEnum = PackedInts.getReaderIterator(in);
-          for (int i = 0; i < valueCount; i += 
-            1 + ((valueCount - i) <= 20 ? random().nextInt(valueCount - i)
-              : random().nextInt(20))) {
-            final String msg = "index=" + i + " valueCount="
-                + valueCount + " nbits=" + nbits + " for "
-                + intsEnum.getClass().getSimpleName();
-            if (i - intsEnum.ord() == 1 && random().nextBoolean()) {
-              assertEquals(msg, values[i], intsEnum.next());
-            } else {
-              assertEquals(msg, values[i], intsEnum.advance(i));
+          PackedInts.ReaderIterator r = PackedInts.getReaderIterator(in, bufferSize);
+          int i = 0;
+          while (i < valueCount) {
+            final int count = _TestUtil.nextInt(random(), 1, 95);
+            final LongsRef next = r.next(count);
+            for (int k = 0; k < next.length; ++k) {
+              assertEquals("index=" + i + " valueCount="
+                  + valueCount + " nbits=" + nbits + " for "
+                  + r.getClass().getSimpleName(), values[i + k], next.longs[next.offset + k]);
             }
-            assertEquals(msg, i, intsEnum.ord());
+            i += next.length;
           }
-          if (intsEnum.ord() < valueCount - 1)
-            assertEquals(values[valueCount - 1], intsEnum
-                .advance(valueCount - 1));
-          assertEquals(valueCount - 1, intsEnum.ord());
           assertEquals(fp, in.getFilePointer());
           in.close();
         }