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/11/15 20:44:16 UTC
svn commit: r1409952 - in /lucene/dev/branches/lucene4547/lucene/core/src:
java/org/apache/lucene/index/ java/org/apache/lucene/util/packed/
test/org/apache/lucene/util/packed/
Author: jpountz
Date: Thu Nov 15 19:44:14 2012
New Revision: 1409952
URL: http://svn.apache.org/viewvc?rev=1409952&view=rev
Log:
More efficient buffering of numeric values.
Added:
lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/AppendingLongBuffer.java (with props)
Modified:
lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/NumberDVWriter.java
lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Direct16.java
lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Direct32.java
lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Direct64.java
lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Direct8.java
lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Packed16ThreeBlocks.java
lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Packed64.java
lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Packed64SingleBlock.java
lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Packed8ThreeBlocks.java
lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/gen_Direct.py
lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/gen_PackedThreeBlocks.py
lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/util/packed/TestPackedInts.java
Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/NumberDVWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/NumberDVWriter.java?rev=1409952&r1=1409951&r2=1409952&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/NumberDVWriter.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/NumberDVWriter.java Thu Nov 15 19:44:14 2012
@@ -18,12 +18,11 @@ package org.apache.lucene.index;
*/
import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
import org.apache.lucene.codecs.NumericDocValuesConsumer;
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.PackedInts;
// nocommit pick numeric or number ... then fix all places ...
@@ -33,12 +32,11 @@ import org.apache.lucene.util.RamUsageEs
// nocommit make this a consumer in the chain?
class NumberDVWriter {
- private final static Long MISSING = new Long(0);
+ private final static long MISSING = 0L;
- // nocommit more ram efficient?
- private final ArrayList<Long> pending = new ArrayList<Long>();
+ private AppendingLongBuffer pending;
private final Counter iwBytesUsed;
- private int bytesUsed;
+ private long bytesUsed;
private final FieldInfo fieldInfo;
long minValue;
@@ -46,29 +44,33 @@ class NumberDVWriter {
private boolean anyValues;
public NumberDVWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
+ pending = new AppendingLongBuffer();
+ bytesUsed = pending.ramBytesUsed();
this.fieldInfo = fieldInfo;
this.iwBytesUsed = iwBytesUsed;
}
public void addValue(int docID, long value) {
- final int oldBytesUsed = bytesUsed;
mergeValue(value);
// Fill in any holes:
- while(pending.size() < docID) {
+ for (int i = pending.size(); i < docID; ++i) {
pending.add(MISSING);
- bytesUsed += RamUsageEstimator.NUM_BYTES_OBJECT_REF;
mergeValue(0);
}
pending.add(value);
- // estimate 25% overhead for ArrayList:
- bytesUsed += (int) (RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + RamUsageEstimator.NUM_BYTES_LONG + (RamUsageEstimator.NUM_BYTES_OBJECT_REF * 1.25));
- iwBytesUsed.addAndGet(bytesUsed - oldBytesUsed);
+ updateBytesUsed();
//System.out.println("ADD: " + value);
}
+ private void updateBytesUsed() {
+ final long newBytesUsed = pending.ramBytesUsed();
+ iwBytesUsed.addAndGet(newBytesUsed - bytesUsed);
+ bytesUsed = newBytesUsed;
+ }
+
private void mergeValue(long value) {
if (!anyValues) {
anyValues = true;
@@ -82,9 +84,12 @@ class NumberDVWriter {
public void flush(FieldInfo fieldInfo, SegmentWriteState state, NumericDocValuesConsumer consumer) throws IOException {
final int bufferedDocCount = pending.size();
+ AppendingLongBuffer.Iterator it = pending.iterator();
for(int docID=0;docID<bufferedDocCount;docID++) {
- consumer.add(pending.get(docID));
+ assert it.hasNext();
+ consumer.add(it.next());
}
+ assert !it.hasNext();
final int maxDoc = state.segmentInfo.getDocCount();
for(int docID=bufferedDocCount;docID<maxDoc;docID++) {
consumer.add(0);
@@ -100,11 +105,10 @@ class NumberDVWriter {
// nocommit do we really need this...? can't parent alloc
// a new instance after flush?
private void reset() {
- pending.clear();
- pending.trimToSize();
- iwBytesUsed.addAndGet(-bytesUsed);
+ pending = new AppendingLongBuffer();
+ updateBytesUsed();
anyValues = false;
minValue = maxValue = 0;
- bytesUsed = 0;
}
+
}
\ No newline at end of file
Added: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/AppendingLongBuffer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/AppendingLongBuffer.java?rev=1409952&view=auto
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/AppendingLongBuffer.java (added)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/AppendingLongBuffer.java Thu Nov 15 19:44:14 2012
@@ -0,0 +1,171 @@
+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 java.util.Arrays;
+
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.RamUsageEstimator;
+
+/**
+ * Utility class to buffer a list of signed longs in memory. This class only
+ * supports appending.
+ */
+public class AppendingLongBuffer {
+
+ private static final int MAX_PENDING_COUNT = 1024;
+
+ private long[] minValues;
+ private PackedInts.Reader[] values;
+ private int valuesOff;
+ private long[] pending;
+ private int pendingOff;
+
+ /** Sole constructor. */
+ public AppendingLongBuffer() {
+ minValues = new long[16];
+ values = new PackedInts.Reader[16];
+ pending = new long[MAX_PENDING_COUNT];
+ valuesOff = 0;
+ pendingOff = 0;
+ }
+
+ /** Append a value to this buffer. */
+ public void add(long l) {
+ if (pendingOff == MAX_PENDING_COUNT) {
+ packPendingValues();
+ }
+ pending[pendingOff++] = l;
+ }
+
+ private void packPendingValues() {
+ assert pendingOff == MAX_PENDING_COUNT;
+
+ // check size
+ if (values.length == valuesOff) {
+ final int newLength = ArrayUtil.oversize(valuesOff + 1, 8);
+ minValues = Arrays.copyOf(minValues, newLength);
+ values = Arrays.copyOf(values, newLength);
+ }
+
+ // 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;
+
+ // 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, PackedInts.COMPACT);
+ for (int i = 0; i < pendingOff; ) {
+ i += mutable.set(i, pending, i, pendingOff - i);
+ }
+
+ // store it
+ minValues[valuesOff] = minValue;
+ values[valuesOff] = mutable;
+ ++valuesOff;
+
+ // reset pending buffer
+ pendingOff = 0;
+ }
+
+ public int size() {
+ return valuesOff * MAX_PENDING_COUNT + pendingOff;
+ }
+
+ /** Return an iterator over the values of this buffer. */
+ public Iterator iterator() {
+ return new Iterator();
+ }
+
+ /** A long iterator. */
+ public class Iterator {
+
+ long[] currentValues;
+ int vOff, pOff;
+
+ private Iterator() {
+ vOff = pOff = 0;
+ if (valuesOff == 0) {
+ currentValues = pending;
+ } else {
+ currentValues = new long[MAX_PENDING_COUNT];
+ fillValues();
+ }
+ }
+
+ private void fillValues() {
+ if (vOff == valuesOff) {
+ currentValues = pending;
+ } else {
+ for (int k = 0; k < MAX_PENDING_COUNT; ++k) {
+ k += values[vOff].get(k, currentValues, k, MAX_PENDING_COUNT - k);
+ }
+ for (int k = 0; k < MAX_PENDING_COUNT; ++k) {
+ currentValues[k] += minValues[vOff];
+ }
+ }
+ }
+
+ /** Whether or not there are remaining values. */
+ public boolean hasNext() {
+ return vOff < valuesOff || pOff < pendingOff;
+ }
+
+ /** Return the next long in the buffer. */
+ public long next() {
+ assert hasNext();
+ long result = currentValues[pOff++];
+ if (pOff == MAX_PENDING_COUNT) {
+ vOff += 1;
+ pOff = 0;
+ if (vOff <= valuesOff) {
+ fillValues();
+ }
+ }
+ return result;
+ }
+
+ }
+
+ /**
+ * Return the number of bytes used by this instance.
+ */
+ public long ramBytesUsed() {
+ long bytesUsed = RamUsageEstimator.alignObjectSize(
+ RamUsageEstimator.NUM_BYTES_OBJECT_HEADER
+ + 3 * RamUsageEstimator.NUM_BYTES_OBJECT_REF // the 3 arrays
+ + 2 * RamUsageEstimator.NUM_BYTES_INT) // the 2 offsets
+ + RamUsageEstimator.sizeOf(pending)
+ + RamUsageEstimator.sizeOf(minValues)
+ + RamUsageEstimator.alignObjectSize(RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + (long) RamUsageEstimator.NUM_BYTES_OBJECT_REF * values.length); // values
+
+ for (int i = 0; i < valuesOff; ++i) {
+ bytesUsed += values[i].ramBytesUsed();
+ }
+ return bytesUsed;
+ }
+
+}
Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Direct16.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Direct16.java?rev=1409952&r1=1409951&r2=1409952&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Direct16.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Direct16.java Thu Nov 15 19:44:14 2012
@@ -58,8 +58,13 @@ final class Direct16 extends PackedInts.
values[index] = (short) (value);
}
+ @Override
public long ramBytesUsed() {
- return RamUsageEstimator.sizeOf(values);
+ return RamUsageEstimator.alignObjectSize(
+ RamUsageEstimator.NUM_BYTES_OBJECT_HEADER
+ + 2 * RamUsageEstimator.NUM_BYTES_INT // valueCount,bitsPerValue
+ + RamUsageEstimator.NUM_BYTES_OBJECT_REF) // values ref
+ + RamUsageEstimator.sizeOf(values);
}
public void clear() {
Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Direct32.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Direct32.java?rev=1409952&r1=1409951&r2=1409952&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Direct32.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Direct32.java Thu Nov 15 19:44:14 2012
@@ -58,8 +58,13 @@ final class Direct32 extends PackedInts.
values[index] = (int) (value);
}
+ @Override
public long ramBytesUsed() {
- return RamUsageEstimator.sizeOf(values);
+ return RamUsageEstimator.alignObjectSize(
+ RamUsageEstimator.NUM_BYTES_OBJECT_HEADER
+ + 2 * RamUsageEstimator.NUM_BYTES_INT // valueCount,bitsPerValue
+ + RamUsageEstimator.NUM_BYTES_OBJECT_REF) // values ref
+ + RamUsageEstimator.sizeOf(values);
}
public void clear() {
Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Direct64.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Direct64.java?rev=1409952&r1=1409951&r2=1409952&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Direct64.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Direct64.java Thu Nov 15 19:44:14 2012
@@ -53,8 +53,13 @@ final class Direct64 extends PackedInts.
values[index] = (value);
}
+ @Override
public long ramBytesUsed() {
- return RamUsageEstimator.sizeOf(values);
+ return RamUsageEstimator.alignObjectSize(
+ RamUsageEstimator.NUM_BYTES_OBJECT_HEADER
+ + 2 * RamUsageEstimator.NUM_BYTES_INT // valueCount,bitsPerValue
+ + RamUsageEstimator.NUM_BYTES_OBJECT_REF) // values ref
+ + RamUsageEstimator.sizeOf(values);
}
public void clear() {
Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Direct8.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Direct8.java?rev=1409952&r1=1409951&r2=1409952&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Direct8.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Direct8.java Thu Nov 15 19:44:14 2012
@@ -56,8 +56,13 @@ final class Direct8 extends PackedInts.M
values[index] = (byte) (value);
}
+ @Override
public long ramBytesUsed() {
- return RamUsageEstimator.sizeOf(values);
+ return RamUsageEstimator.alignObjectSize(
+ RamUsageEstimator.NUM_BYTES_OBJECT_HEADER
+ + 2 * RamUsageEstimator.NUM_BYTES_INT // valueCount,bitsPerValue
+ + RamUsageEstimator.NUM_BYTES_OBJECT_REF) // values ref
+ + RamUsageEstimator.sizeOf(values);
}
public void clear() {
Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Packed16ThreeBlocks.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Packed16ThreeBlocks.java?rev=1409952&r1=1409951&r2=1409952&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Packed16ThreeBlocks.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Packed16ThreeBlocks.java Thu Nov 15 19:44:14 2012
@@ -114,8 +114,13 @@ final class Packed16ThreeBlocks extends
Arrays.fill(blocks, (short) 0);
}
+ @Override
public long ramBytesUsed() {
- return RamUsageEstimator.sizeOf(blocks);
+ return RamUsageEstimator.alignObjectSize(
+ RamUsageEstimator.NUM_BYTES_OBJECT_HEADER
+ + 2 * RamUsageEstimator.NUM_BYTES_INT // valueCount,bitsPerValue
+ + RamUsageEstimator.NUM_BYTES_OBJECT_REF) // blocks ref
+ + RamUsageEstimator.sizeOf(blocks);
}
@Override
Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Packed64.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Packed64.java?rev=1409952&r1=1409951&r2=1409952&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Packed64.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Packed64.java Thu Nov 15 19:44:14 2012
@@ -244,7 +244,12 @@ class Packed64 extends PackedInts.Mutabl
@Override
public long ramBytesUsed() {
- return RamUsageEstimator.sizeOf(blocks);
+ return RamUsageEstimator.alignObjectSize(
+ RamUsageEstimator.NUM_BYTES_OBJECT_HEADER
+ + 3 * RamUsageEstimator.NUM_BYTES_INT // bpvMinusBlockSize,valueCount,bitsPerValue
+ + RamUsageEstimator.NUM_BYTES_LONG // maskRight
+ + RamUsageEstimator.NUM_BYTES_OBJECT_REF) // blocks ref
+ + RamUsageEstimator.sizeOf(blocks);
}
@Override
Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Packed64SingleBlock.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Packed64SingleBlock.java?rev=1409952&r1=1409951&r2=1409952&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Packed64SingleBlock.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Packed64SingleBlock.java Thu Nov 15 19:44:14 2012
@@ -58,8 +58,13 @@ abstract class Packed64SingleBlock exten
Arrays.fill(blocks, 0L);
}
+ @Override
public long ramBytesUsed() {
- return RamUsageEstimator.sizeOf(blocks);
+ return RamUsageEstimator.alignObjectSize(
+ RamUsageEstimator.NUM_BYTES_OBJECT_HEADER
+ + 2 * RamUsageEstimator.NUM_BYTES_INT // valueCount,bitsPerValue
+ + RamUsageEstimator.NUM_BYTES_OBJECT_REF) // blocks ref
+ + RamUsageEstimator.sizeOf(blocks);
}
@Override
Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Packed8ThreeBlocks.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Packed8ThreeBlocks.java?rev=1409952&r1=1409951&r2=1409952&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Packed8ThreeBlocks.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/Packed8ThreeBlocks.java Thu Nov 15 19:44:14 2012
@@ -112,8 +112,13 @@ final class Packed8ThreeBlocks extends P
Arrays.fill(blocks, (byte) 0);
}
+ @Override
public long ramBytesUsed() {
- return RamUsageEstimator.sizeOf(blocks);
+ return RamUsageEstimator.alignObjectSize(
+ RamUsageEstimator.NUM_BYTES_OBJECT_HEADER
+ + 2 * RamUsageEstimator.NUM_BYTES_INT // valueCount,bitsPerValue
+ + RamUsageEstimator.NUM_BYTES_OBJECT_REF) // blocks ref
+ + RamUsageEstimator.sizeOf(blocks);
}
@Override
Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/gen_Direct.py
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/gen_Direct.py?rev=1409952&r1=1409951&r2=1409952&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/gen_Direct.py (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/gen_Direct.py Thu Nov 15 19:44:14 2012
@@ -91,8 +91,13 @@ if __name__ == '__main__':
values[index] = %s(value);
}
+ @Override
public long ramBytesUsed() {
- return RamUsageEstimator.sizeOf(values);
+ return RamUsageEstimator.alignObjectSize(
+ RamUsageEstimator.NUM_BYTES_OBJECT_HEADER
+ + 2 * RamUsageEstimator.NUM_BYTES_INT // valueCount,bitsPerValue
+ + RamUsageEstimator.NUM_BYTES_OBJECT_REF) // values ref
+ + RamUsageEstimator.sizeOf(values);
}
public void clear() {
Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/gen_PackedThreeBlocks.py
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/gen_PackedThreeBlocks.py?rev=1409952&r1=1409951&r2=1409952&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/gen_PackedThreeBlocks.py (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/packed/gen_PackedThreeBlocks.py Thu Nov 15 19:44:14 2012
@@ -146,8 +146,13 @@ if __name__ == '__main__':
Arrays.fill(blocks, %s0);
}
+ @Override
public long ramBytesUsed() {
- return RamUsageEstimator.sizeOf(blocks);
+ return RamUsageEstimator.alignObjectSize(
+ RamUsageEstimator.NUM_BYTES_OBJECT_HEADER
+ + 2 * RamUsageEstimator.NUM_BYTES_INT // valueCount,bitsPerValue
+ + RamUsageEstimator.NUM_BYTES_OBJECT_REF) // blocks ref
+ + RamUsageEstimator.sizeOf(blocks);
}
@Override
Modified: lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/util/packed/TestPackedInts.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/util/packed/TestPackedInts.java?rev=1409952&r1=1409951&r2=1409952&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/util/packed/TestPackedInts.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/util/packed/TestPackedInts.java Thu Nov 15 19:44:14 2012
@@ -34,6 +34,7 @@ import org.apache.lucene.store.IndexOutp
import org.apache.lucene.store.RAMDirectory;
import org.apache.lucene.util.LongsRef;
import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.LuceneTestCase.Slow;
import org.apache.lucene.util._TestUtil;
import org.apache.lucene.util.packed.PackedInts.Reader;
@@ -133,6 +134,11 @@ public class TestPackedInts extends Luce
+ r.getClass().getSimpleName(), values[i], r.get(i));
}
in.close();
+
+ final long expectedBytesUsed = RamUsageEstimator.sizeOf(r);
+ final long computedBytesUsed = r.ramBytesUsed();
+ assertEquals(r.getClass() + "expected " + expectedBytesUsed + ", got: " + computedBytesUsed,
+ expectedBytesUsed, computedBytesUsed);
}
{ // test reader iterator next
@@ -787,4 +793,40 @@ public class TestPackedInts extends Luce
return true;
}
+ public void testAppendingLongBuffer() {
+ final long[] arr = new long[RandomInts.randomIntBetween(random(), 1, 2000000)];
+ for (int bpv : new int[] {0, 1, 63, 64, RandomInts.randomIntBetween(random(), 2, 61)}) {
+ if (bpv == 0) {
+ Arrays.fill(arr, random().nextLong());
+ } else if (bpv == 64) {
+ for (int i = 0; i < arr.length; ++i) {
+ arr[i] = random().nextLong();
+ }
+ } else {
+ final long minValue = _TestUtil.nextLong(random(), Long.MIN_VALUE, Long.MAX_VALUE - PackedInts.maxValue(bpv));
+ for (int i = 0; i < arr.length; ++i) {
+ arr[i] = minValue + random().nextLong() & PackedInts.maxValue(bpv);
+ }
+ }
+ AppendingLongBuffer buf = new AppendingLongBuffer();
+ for (int i = 0; i < arr.length; ++i) {
+ buf.add(arr[i]);
+ }
+ assertEquals(arr.length, buf.size());
+ final AppendingLongBuffer.Iterator it = buf.iterator();
+ for (int i = 0; i < arr.length; ++i) {
+ if (random().nextBoolean()) {
+ assertTrue(it.hasNext());
+ }
+ assertEquals(arr[i], it.next());
+ }
+ assertFalse(it.hasNext());
+
+ final long expectedBytesUsed = RamUsageEstimator.sizeOf(buf);
+ final long computedBytesUsed = buf.ramBytesUsed();
+ assertEquals("got " + computedBytesUsed + ", expected: " + expectedBytesUsed,
+ expectedBytesUsed, computedBytesUsed);
+ }
+ }
+
}