LUCENE-5792: Simplified *AppendingBuffer APIs.

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1607103 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Adrien Grand 2014-07-01 14:55:46 +00:00
parent 59a18c685b
commit a974c1aab1
23 changed files with 718 additions and 793 deletions

View File

@ -30,8 +30,8 @@ import org.apache.lucene.util.Counter;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.PagedBytes;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.packed.AppendingDeltaPackedLongBuffer;
import org.apache.lucene.util.packed.PackedInts;
import org.apache.lucene.util.packed.PackedLongValues;
/** Buffers up pending byte[] per doc, then flushes when
* segment flushes. */
@ -47,7 +47,7 @@ class BinaryDocValuesWriter extends DocValuesWriter {
private final DataOutput bytesOut;
private final Counter iwBytesUsed;
private final AppendingDeltaPackedLongBuffer lengths;
private final PackedLongValues.Builder lengths;
private FixedBitSet docsWithField;
private final FieldInfo fieldInfo;
private int addedValues;
@ -57,7 +57,7 @@ class BinaryDocValuesWriter extends DocValuesWriter {
this.fieldInfo = fieldInfo;
this.bytes = new PagedBytes(BLOCK_BITS);
this.bytesOut = bytes.getDataOutput();
this.lengths = new AppendingDeltaPackedLongBuffer(PackedInts.COMPACT);
this.lengths = PackedLongValues.deltaPackedBuilder(PackedInts.COMPACT);
this.iwBytesUsed = iwBytesUsed;
this.docsWithField = new FixedBitSet(64);
this.bytesUsed = docsWithFieldBytesUsed();
@ -112,11 +112,12 @@ class BinaryDocValuesWriter extends DocValuesWriter {
public void flush(SegmentWriteState state, DocValuesConsumer dvConsumer) throws IOException {
final int maxDoc = state.segmentInfo.getDocCount();
bytes.freeze(false);
final PackedLongValues lengths = this.lengths.build();
dvConsumer.addBinaryField(fieldInfo,
new Iterable<BytesRef>() {
@Override
public Iterator<BytesRef> iterator() {
return new BytesIterator(maxDoc);
return new BytesIterator(maxDoc, lengths);
}
});
}
@ -124,14 +125,15 @@ class BinaryDocValuesWriter extends DocValuesWriter {
// iterates over the values we have in ram
private class BytesIterator implements Iterator<BytesRef> {
final BytesRef value = new BytesRef();
final AppendingDeltaPackedLongBuffer.Iterator lengthsIterator = lengths.iterator();
final PackedLongValues.Iterator lengthsIterator;
final DataInput bytesIterator = bytes.getDataInput();
final int size = (int) lengths.size();
final int maxDoc;
int upto;
BytesIterator(int maxDoc) {
BytesIterator(int maxDoc, PackedLongValues lengths) {
this.maxDoc = maxDoc;
this.lengthsIterator = lengths.iterator();
}
@Override

View File

@ -22,7 +22,8 @@ import java.util.List;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.InfoStream;
import org.apache.lucene.util.packed.MonotonicAppendingLongBuffer;
import org.apache.lucene.util.packed.PackedInts;
import org.apache.lucene.util.packed.PackedLongValues;
/** Holds common state used during segment merging.
*
@ -69,15 +70,15 @@ public class MergeState {
static DocMap build(final int maxDoc, final Bits liveDocs) {
assert liveDocs != null;
final MonotonicAppendingLongBuffer docMap = new MonotonicAppendingLongBuffer();
final PackedLongValues.Builder docMapBuilder = PackedLongValues.monotonicBuilder(PackedInts.COMPACT);
int del = 0;
for (int i = 0; i < maxDoc; ++i) {
docMap.add(i - del);
docMapBuilder.add(i - del);
if (!liveDocs.get(i)) {
++del;
}
}
docMap.freeze();
final PackedLongValues docMap = docMapBuilder.build();
final int numDeletedDocs = del;
assert docMap.size() == maxDoc;
return new DocMap() {

View File

@ -29,9 +29,8 @@ import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.InPlaceMergeSorter;
import org.apache.lucene.util.LongValues;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.packed.AppendingPackedLongBuffer;
import org.apache.lucene.util.packed.MonotonicAppendingLongBuffer;
import org.apache.lucene.util.packed.PackedInts;
import org.apache.lucene.util.packed.PackedLongValues;
/**
* A wrapper for CompositeIndexReader providing access to DocValues.
@ -488,9 +487,9 @@ public class MultiDocValues {
// cache key of whoever asked for this awful thing
final Object owner;
// globalOrd -> (globalOrd - segmentOrd) where segmentOrd is the the ordinal in the first segment that contains this term
final MonotonicAppendingLongBuffer globalOrdDeltas;
final PackedLongValues globalOrdDeltas;
// globalOrd -> first segment container
final AppendingPackedLongBuffer firstSegments;
final PackedLongValues firstSegments;
// for every segment, segmentOrd -> globalOrd
final LongValues segmentToGlobalOrds[];
// the map from/to segment ids
@ -506,11 +505,11 @@ public class MultiDocValues {
// even though we accept an overhead ratio, we keep these ones with COMPACT
// since they are only used to resolve values given a global ord, which is
// slow anyway
globalOrdDeltas = new MonotonicAppendingLongBuffer(PackedInts.COMPACT);
firstSegments = new AppendingPackedLongBuffer(PackedInts.COMPACT);
final MonotonicAppendingLongBuffer[] ordDeltas = new MonotonicAppendingLongBuffer[subs.length];
PackedLongValues.Builder globalOrdDeltas = PackedLongValues.monotonicBuilder(PackedInts.COMPACT);
PackedLongValues.Builder firstSegments = PackedLongValues.packedBuilder(PackedInts.COMPACT);
final PackedLongValues.Builder[] ordDeltas = new PackedLongValues.Builder[subs.length];
for (int i = 0; i < ordDeltas.length; i++) {
ordDeltas[i] = new MonotonicAppendingLongBuffer(acceptableOverheadRatio);
ordDeltas[i] = PackedLongValues.monotonicBuilder(acceptableOverheadRatio);
}
long[] ordDeltaBits = new long[subs.length];
long segmentOrds[] = new long[subs.length];
@ -551,18 +550,15 @@ public class MultiDocValues {
globalOrdDeltas.add(globalOrdDelta);
globalOrd++;
}
firstSegments.freeze();
globalOrdDeltas.freeze();
for (int i = 0; i < ordDeltas.length; ++i) {
ordDeltas[i].freeze();
}
this.firstSegments = firstSegments.build();
this.globalOrdDeltas = globalOrdDeltas.build();
// ordDeltas is typically the bottleneck, so let's see what we can do to make it faster
segmentToGlobalOrds = new LongValues[subs.length];
long ramBytesUsed = BASE_RAM_BYTES_USED + globalOrdDeltas.ramBytesUsed()
+ firstSegments.ramBytesUsed() + RamUsageEstimator.shallowSizeOf(segmentToGlobalOrds)
long ramBytesUsed = BASE_RAM_BYTES_USED + this.globalOrdDeltas.ramBytesUsed()
+ this.firstSegments.ramBytesUsed() + RamUsageEstimator.shallowSizeOf(segmentToGlobalOrds)
+ segmentMap.ramBytesUsed();
for (int i = 0; i < ordDeltas.length; ++i) {
final MonotonicAppendingLongBuffer deltas = ordDeltas[i];
final PackedLongValues deltas = ordDeltas[i].build();
if (ordDeltaBits[i] == 0L) {
// segment ords perfectly match global ordinals
// likely in case of low cardinalities and large segments
@ -576,7 +572,7 @@ public class MultiDocValues {
// monotonic compression mostly adds overhead, let's keep the mapping in plain packed ints
final int size = (int) deltas.size();
final PackedInts.Mutable newDeltas = PackedInts.getMutable(size, bitsRequired, acceptableOverheadRatio);
final MonotonicAppendingLongBuffer.Iterator it = deltas.iterator();
final PackedLongValues.Iterator it = deltas.iterator();
for (int ord = 0; ord < size; ++ord) {
newDeltas.set(ord, it.next());
}

View File

@ -25,8 +25,8 @@ import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.packed.AppendingDeltaPackedLongBuffer;
import org.apache.lucene.util.packed.PackedInts;
import org.apache.lucene.util.packed.PackedLongValues;
/** Buffers up pending long per doc, then flushes when
* segment flushes. */
@ -34,14 +34,14 @@ class NumericDocValuesWriter extends DocValuesWriter {
private final static long MISSING = 0L;
private AppendingDeltaPackedLongBuffer pending;
private PackedLongValues.Builder pending;
private final Counter iwBytesUsed;
private long bytesUsed;
private FixedBitSet docsWithField;
private final FieldInfo fieldInfo;
public NumericDocValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed, boolean trackDocsWithField) {
pending = new AppendingDeltaPackedLongBuffer(PackedInts.COMPACT);
pending = PackedLongValues.deltaPackedBuilder(PackedInts.COMPACT);
docsWithField = trackDocsWithField ? new FixedBitSet(64) : null;
bytesUsed = pending.ramBytesUsed() + docsWithFieldBytesUsed();
this.fieldInfo = fieldInfo;
@ -87,25 +87,30 @@ class NumericDocValuesWriter extends DocValuesWriter {
public void flush(SegmentWriteState state, DocValuesConsumer dvConsumer) throws IOException {
final int maxDoc = state.segmentInfo.getDocCount();
final PackedLongValues values = pending.build();
dvConsumer.addNumericField(fieldInfo,
new Iterable<Number>() {
@Override
public Iterator<Number> iterator() {
return new NumericIterator(maxDoc);
return new NumericIterator(maxDoc, values, docsWithField);
}
});
}
// iterates over the values we have in ram
private class NumericIterator implements Iterator<Number> {
final AppendingDeltaPackedLongBuffer.Iterator iter = pending.iterator();
final int size = (int)pending.size();
private static class NumericIterator implements Iterator<Number> {
final PackedLongValues.Iterator iter;
final FixedBitSet docsWithField;
final int size;
final int maxDoc;
int upto;
NumericIterator(int maxDoc) {
NumericIterator(int maxDoc, PackedLongValues values, FixedBitSet docsWithFields) {
this.maxDoc = maxDoc;
this.iter = values.iterator();
this.size = (int) values.size();
this.docsWithField = docsWithFields;
}
@Override

View File

@ -30,14 +30,14 @@ import org.apache.lucene.util.BytesRefHash.DirectBytesStartArray;
import org.apache.lucene.util.BytesRefHash;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.packed.AppendingDeltaPackedLongBuffer;
import org.apache.lucene.util.packed.PackedInts;
import org.apache.lucene.util.packed.PackedLongValues;
/** Buffers up pending byte[] per doc, deref and sorting via
* int ord, then flushes when segment flushes. */
class SortedDocValuesWriter extends DocValuesWriter {
final BytesRefHash hash;
private AppendingDeltaPackedLongBuffer pending;
private PackedLongValues.Builder pending;
private final Counter iwBytesUsed;
private long bytesUsed; // this currently only tracks differences in 'pending'
private final FieldInfo fieldInfo;
@ -52,7 +52,7 @@ class SortedDocValuesWriter extends DocValuesWriter {
new ByteBlockPool.DirectTrackingAllocator(iwBytesUsed)),
BytesRefHash.DEFAULT_CAPACITY,
new DirectBytesStartArray(BytesRefHash.DEFAULT_CAPACITY, iwBytesUsed));
pending = new AppendingDeltaPackedLongBuffer(PackedInts.COMPACT);
pending = PackedLongValues.deltaPackedBuilder(PackedInts.COMPACT);
bytesUsed = pending.ramBytesUsed();
iwBytesUsed.addAndGet(bytesUsed);
}
@ -112,6 +112,7 @@ class SortedDocValuesWriter extends DocValuesWriter {
assert pending.size() == maxDoc;
final int valueCount = hash.size();
final PackedLongValues ords = pending.build();
final int[] sortedValues = hash.sort(BytesRef.getUTF8SortedAsUnicodeComparator());
final int[] ordMap = new int[valueCount];
@ -126,7 +127,7 @@ class SortedDocValuesWriter extends DocValuesWriter {
new Iterable<BytesRef>() {
@Override
public Iterator<BytesRef> iterator() {
return new ValuesIterator(sortedValues, valueCount);
return new ValuesIterator(sortedValues, valueCount, hash);
}
},
@ -134,21 +135,23 @@ class SortedDocValuesWriter extends DocValuesWriter {
new Iterable<Number>() {
@Override
public Iterator<Number> iterator() {
return new OrdsIterator(ordMap, maxDoc);
return new OrdsIterator(ordMap, maxDoc, ords);
}
});
}
// iterates over the unique values we have in ram
private class ValuesIterator implements Iterator<BytesRef> {
private static class ValuesIterator implements Iterator<BytesRef> {
final int sortedValues[];
final BytesRefHash hash;
final BytesRef scratch = new BytesRef();
final int valueCount;
int ordUpto;
ValuesIterator(int sortedValues[], int valueCount) {
ValuesIterator(int sortedValues[], int valueCount, BytesRefHash hash) {
this.sortedValues = sortedValues;
this.valueCount = valueCount;
this.hash = hash;
}
@Override
@ -173,16 +176,17 @@ class SortedDocValuesWriter extends DocValuesWriter {
}
// iterates over the ords for each doc we have in ram
private class OrdsIterator implements Iterator<Number> {
final AppendingDeltaPackedLongBuffer.Iterator iter = pending.iterator();
private static class OrdsIterator implements Iterator<Number> {
final PackedLongValues.Iterator iter;
final int ordMap[];
final int maxDoc;
int docUpto;
OrdsIterator(int ordMap[], int maxDoc) {
OrdsIterator(int ordMap[], int maxDoc, PackedLongValues ords) {
this.ordMap = ordMap;
this.maxDoc = maxDoc;
assert pending.size() == maxDoc;
assert ords.size() == maxDoc;
this.iter = ords.iterator();
}
@Override

View File

@ -26,13 +26,13 @@ import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.packed.AppendingDeltaPackedLongBuffer;
import org.apache.lucene.util.packed.PackedInts;
import org.apache.lucene.util.packed.PackedLongValues;
/** Buffers up pending long[] per doc, sorts, then flushes when segment flushes. */
class SortedNumericDocValuesWriter extends DocValuesWriter {
private AppendingDeltaPackedLongBuffer pending; // stream of all values
private AppendingDeltaPackedLongBuffer pendingCounts; // count of values per doc
private PackedLongValues.Builder pending; // stream of all values
private PackedLongValues.Builder pendingCounts; // count of values per doc
private final Counter iwBytesUsed;
private long bytesUsed; // this only tracks differences in 'pending' and 'pendingCounts'
private final FieldInfo fieldInfo;
@ -43,8 +43,8 @@ class SortedNumericDocValuesWriter extends DocValuesWriter {
public SortedNumericDocValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
this.fieldInfo = fieldInfo;
this.iwBytesUsed = iwBytesUsed;
pending = new AppendingDeltaPackedLongBuffer(PackedInts.COMPACT);
pendingCounts = new AppendingDeltaPackedLongBuffer(PackedInts.COMPACT);
pending = PackedLongValues.deltaPackedBuilder(PackedInts.COMPACT);
pendingCounts = PackedLongValues.deltaPackedBuilder(PackedInts.COMPACT);
bytesUsed = pending.ramBytesUsed() + pendingCounts.ramBytesUsed();
iwBytesUsed.addAndGet(bytesUsed);
}
@ -105,13 +105,15 @@ class SortedNumericDocValuesWriter extends DocValuesWriter {
public void flush(SegmentWriteState state, DocValuesConsumer dvConsumer) throws IOException {
final int maxDoc = state.segmentInfo.getDocCount();
assert pendingCounts.size() == maxDoc;
final PackedLongValues values = pending.build();
final PackedLongValues valueCounts = pendingCounts.build();
dvConsumer.addSortedNumericField(fieldInfo,
// doc -> valueCount
new Iterable<Number>() {
@Override
public Iterator<Number> iterator() {
return new CountIterator();
return new CountIterator(valueCounts);
}
},
@ -119,15 +121,19 @@ class SortedNumericDocValuesWriter extends DocValuesWriter {
new Iterable<Number>() {
@Override
public Iterator<Number> iterator() {
return new ValuesIterator();
return new ValuesIterator(values);
}
});
}
// iterates over the values for each doc we have in ram
private class ValuesIterator implements Iterator<Number> {
final AppendingDeltaPackedLongBuffer.Iterator iter = pending.iterator();
private static class ValuesIterator implements Iterator<Number> {
final PackedLongValues.Iterator iter;
ValuesIterator(PackedLongValues values) {
iter = values.iterator();
}
@Override
public boolean hasNext() {
return iter.hasNext();
@ -147,9 +153,13 @@ class SortedNumericDocValuesWriter extends DocValuesWriter {
}
}
private class CountIterator implements Iterator<Number> {
final AppendingDeltaPackedLongBuffer.Iterator iter = pendingCounts.iterator();
private static class CountIterator implements Iterator<Number> {
final PackedLongValues.Iterator iter;
CountIterator(PackedLongValues valueCounts) {
this.iter = valueCounts.iterator();
}
@Override
public boolean hasNext() {
return iter.hasNext();

View File

@ -32,16 +32,15 @@ import org.apache.lucene.util.BytesRefHash.DirectBytesStartArray;
import org.apache.lucene.util.BytesRefHash;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.packed.AppendingDeltaPackedLongBuffer;
import org.apache.lucene.util.packed.AppendingPackedLongBuffer;
import org.apache.lucene.util.packed.PackedInts;
import org.apache.lucene.util.packed.PackedLongValues;
/** Buffers up pending byte[]s per doc, deref and sorting via
* int ord, then flushes when segment flushes. */
class SortedSetDocValuesWriter extends DocValuesWriter {
final BytesRefHash hash;
private AppendingPackedLongBuffer pending; // stream of all termIDs
private AppendingDeltaPackedLongBuffer pendingCounts; // termIDs per doc
private PackedLongValues.Builder pending; // stream of all termIDs
private PackedLongValues.Builder pendingCounts; // termIDs per doc
private final Counter iwBytesUsed;
private long bytesUsed; // this only tracks differences in 'pending' and 'pendingCounts'
private final FieldInfo fieldInfo;
@ -58,8 +57,8 @@ class SortedSetDocValuesWriter extends DocValuesWriter {
new ByteBlockPool.DirectTrackingAllocator(iwBytesUsed)),
BytesRefHash.DEFAULT_CAPACITY,
new DirectBytesStartArray(BytesRefHash.DEFAULT_CAPACITY, iwBytesUsed));
pending = new AppendingPackedLongBuffer(PackedInts.COMPACT);
pendingCounts = new AppendingDeltaPackedLongBuffer(PackedInts.COMPACT);
pending = PackedLongValues.packedBuilder(PackedInts.COMPACT);
pendingCounts = PackedLongValues.deltaPackedBuilder(PackedInts.COMPACT);
bytesUsed = pending.ramBytesUsed() + pendingCounts.ramBytesUsed();
iwBytesUsed.addAndGet(bytesUsed);
}
@ -152,6 +151,8 @@ class SortedSetDocValuesWriter extends DocValuesWriter {
final int maxCountPerDoc = maxCount;
assert pendingCounts.size() == maxDoc;
final int valueCount = hash.size();
final PackedLongValues ords = pending.build();
final PackedLongValues ordCounts = pendingCounts.build();
final int[] sortedValues = hash.sort(BytesRef.getUTF8SortedAsUnicodeComparator());
final int[] ordMap = new int[valueCount];
@ -166,7 +167,7 @@ class SortedSetDocValuesWriter extends DocValuesWriter {
new Iterable<BytesRef>() {
@Override
public Iterator<BytesRef> iterator() {
return new ValuesIterator(sortedValues, valueCount);
return new ValuesIterator(sortedValues, valueCount, hash);
}
},
@ -174,7 +175,7 @@ class SortedSetDocValuesWriter extends DocValuesWriter {
new Iterable<Number>() {
@Override
public Iterator<Number> iterator() {
return new OrdCountIterator(maxDoc);
return new OrdCountIterator(maxDoc, ordCounts);
}
},
@ -182,21 +183,23 @@ class SortedSetDocValuesWriter extends DocValuesWriter {
new Iterable<Number>() {
@Override
public Iterator<Number> iterator() {
return new OrdsIterator(ordMap, maxCountPerDoc);
return new OrdsIterator(ordMap, maxCountPerDoc, ords, ordCounts);
}
});
}
// iterates over the unique values we have in ram
private class ValuesIterator implements Iterator<BytesRef> {
private static class ValuesIterator implements Iterator<BytesRef> {
final int sortedValues[];
final BytesRefHash hash;
final BytesRef scratch = new BytesRef();
final int valueCount;
int ordUpto;
ValuesIterator(int sortedValues[], int valueCount) {
ValuesIterator(int sortedValues[], int valueCount, BytesRefHash hash) {
this.sortedValues = sortedValues;
this.valueCount = valueCount;
this.hash = hash;
}
@Override
@ -221,9 +224,9 @@ class SortedSetDocValuesWriter extends DocValuesWriter {
}
// iterates over the ords for each doc we have in ram
private class OrdsIterator implements Iterator<Number> {
final AppendingPackedLongBuffer.Iterator iter = pending.iterator();
final AppendingDeltaPackedLongBuffer.Iterator counts = pendingCounts.iterator();
private static class OrdsIterator implements Iterator<Number> {
final PackedLongValues.Iterator iter;
final PackedLongValues.Iterator counts;
final int ordMap[];
final long numOrds;
long ordUpto;
@ -232,10 +235,12 @@ class SortedSetDocValuesWriter extends DocValuesWriter {
int currentUpto;
int currentLength;
OrdsIterator(int ordMap[], int maxCount) {
OrdsIterator(int ordMap[], int maxCount, PackedLongValues ords, PackedLongValues ordCounts) {
this.currentDoc = new int[maxCount];
this.ordMap = ordMap;
this.numOrds = pending.size();
this.numOrds = ords.size();
this.iter = ords.iterator();
this.counts = ordCounts.iterator();
}
@Override
@ -270,14 +275,15 @@ class SortedSetDocValuesWriter extends DocValuesWriter {
}
}
private class OrdCountIterator implements Iterator<Number> {
final AppendingDeltaPackedLongBuffer.Iterator iter = pendingCounts.iterator();
private static class OrdCountIterator implements Iterator<Number> {
final PackedLongValues.Iterator iter;
final int maxDoc;
int docUpto;
OrdCountIterator(int maxDoc) {
OrdCountIterator(int maxDoc, PackedLongValues ordCounts) {
this.maxDoc = maxDoc;
assert pendingCounts.size() == maxDoc;
assert ordCounts.size() == maxDoc;
this.iter = ordCounts.iterator();
}
@Override

View File

@ -22,8 +22,8 @@ import java.util.Arrays;
import org.apache.lucene.search.DocIdSet;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.util.packed.MonotonicAppendingLongBuffer;
import org.apache.lucene.util.packed.PackedInts;
import org.apache.lucene.util.packed.PackedLongValues;
/**
* {@link DocIdSet} implementation based on pfor-delta encoding.
@ -43,14 +43,12 @@ public final class PForDeltaDocIdSet extends DocIdSet implements Accountable {
static final int[] ITERATIONS = new int[32];
static final int[] BYTE_BLOCK_COUNTS = new int[32];
static final int MAX_BYTE_BLOCK_COUNT;
static final MonotonicAppendingLongBuffer SINGLE_ZERO_BUFFER = new MonotonicAppendingLongBuffer(0, 64, PackedInts.COMPACT);
static final PForDeltaDocIdSet EMPTY = new PForDeltaDocIdSet(null, 0, Integer.MAX_VALUE, SINGLE_ZERO_BUFFER, SINGLE_ZERO_BUFFER);
static final PackedLongValues SINGLE_ZERO = PackedLongValues.packedBuilder(PackedInts.COMPACT).add(0L).build();
static final PForDeltaDocIdSet EMPTY = new PForDeltaDocIdSet(null, 0, Integer.MAX_VALUE, SINGLE_ZERO, SINGLE_ZERO);
static final int LAST_BLOCK = 1 << 5; // flag to indicate the last block
static final int HAS_EXCEPTIONS = 1 << 6;
static final int UNARY = 1 << 7;
static {
SINGLE_ZERO_BUFFER.add(0);
SINGLE_ZERO_BUFFER.freeze();
int maxByteBLockCount = 0;
for (int i = 1; i < ITERATIONS.length; ++i) {
DECODERS[i] = PackedInts.getDecoder(PackedInts.Format.PACKED, PackedInts.VERSION_CURRENT, i);
@ -282,20 +280,19 @@ public final class PForDeltaDocIdSet extends DocIdSet implements Accountable {
final byte[] dataArr = Arrays.copyOf(data.bytes, data.length + MAX_BYTE_BLOCK_COUNT);
final int indexSize = (numBlocks - 1) / indexInterval + 1;
final MonotonicAppendingLongBuffer docIDs, offsets;
final PackedLongValues docIDs, offsets;
if (indexSize <= 1) {
docIDs = offsets = SINGLE_ZERO_BUFFER;
docIDs = offsets = SINGLE_ZERO;
} else {
final int pageSize = 128;
final int initialPageCount = (indexSize + pageSize - 1) / pageSize;
docIDs = new MonotonicAppendingLongBuffer(initialPageCount, pageSize, PackedInts.COMPACT);
offsets = new MonotonicAppendingLongBuffer(initialPageCount, pageSize, PackedInts.COMPACT);
final PackedLongValues.Builder docIDsBuilder = PackedLongValues.monotonicBuilder(pageSize, PackedInts.COMPACT);
final PackedLongValues.Builder offsetsBuilder = PackedLongValues.monotonicBuilder(pageSize, PackedInts.COMPACT);
// Now build the index
final Iterator it = new Iterator(dataArr, cardinality, Integer.MAX_VALUE, SINGLE_ZERO_BUFFER, SINGLE_ZERO_BUFFER);
final Iterator it = new Iterator(dataArr, cardinality, Integer.MAX_VALUE, SINGLE_ZERO, SINGLE_ZERO);
index:
for (int k = 0; k < indexSize; ++k) {
docIDs.add(it.docID() + 1);
offsets.add(it.offset);
docIDsBuilder.add(it.docID() + 1);
offsetsBuilder.add(it.offset);
for (int i = 0; i < indexInterval; ++i) {
it.skipBlock();
if (it.docID() == DocIdSetIterator.NO_MORE_DOCS) {
@ -303,8 +300,8 @@ public final class PForDeltaDocIdSet extends DocIdSet implements Accountable {
}
}
}
docIDs.freeze();
offsets.freeze();
docIDs = docIDsBuilder.build();
offsets = offsetsBuilder.build();
}
return new PForDeltaDocIdSet(dataArr, cardinality, indexInterval, docIDs, offsets);
@ -313,10 +310,10 @@ public final class PForDeltaDocIdSet extends DocIdSet implements Accountable {
}
final byte[] data;
final MonotonicAppendingLongBuffer docIDs, offsets; // for the index
final PackedLongValues docIDs, offsets; // for the index
final int cardinality, indexInterval;
PForDeltaDocIdSet(byte[] data, int cardinality, int indexInterval, MonotonicAppendingLongBuffer docIDs, MonotonicAppendingLongBuffer offsets) {
PForDeltaDocIdSet(byte[] data, int cardinality, int indexInterval, PackedLongValues docIDs, PackedLongValues offsets) {
this.data = data;
this.cardinality = cardinality;
this.indexInterval = indexInterval;
@ -342,7 +339,7 @@ public final class PForDeltaDocIdSet extends DocIdSet implements Accountable {
// index
final int indexInterval;
final MonotonicAppendingLongBuffer docIDs, offsets;
final PackedLongValues docIDs, offsets;
final int cardinality;
final byte[] data;
@ -356,7 +353,7 @@ public final class PForDeltaDocIdSet extends DocIdSet implements Accountable {
int blockIdx;
int docID;
Iterator(byte[] data, int cardinality, int indexInterval, MonotonicAppendingLongBuffer docIDs, MonotonicAppendingLongBuffer offsets) {
Iterator(byte[] data, int cardinality, int indexInterval, PackedLongValues docIDs, PackedLongValues offsets) {
this.data = data;
this.cardinality = cardinality;
this.indexInterval = indexInterval;
@ -519,10 +516,10 @@ public final class PForDeltaDocIdSet extends DocIdSet implements Accountable {
return 0L;
}
long ramBytesUsed = BASE_RAM_BYTES_USED + RamUsageEstimator.sizeOf(data);
if (docIDs != SINGLE_ZERO_BUFFER) {
if (docIDs != SINGLE_ZERO) {
ramBytesUsed += docIDs.ramBytesUsed();
}
if (offsets != SINGLE_ZERO_BUFFER) {
if (offsets != SINGLE_ZERO) {
ramBytesUsed += offsets.ramBytesUsed();
}
return ramBytesUsed;

View File

@ -475,4 +475,20 @@ public final class RamUsageEstimator {
return bytes + " bytes";
}
}
/**
* Return the size of the provided array of {@link Accountable}s by summing
* up the shallow size of the array and the
* {@link Accountable#ramBytesUsed() memory usage} reported by each
* {@link Accountable}.
*/
public static long sizeOf(Accountable[] accountables) {
long size = shallowSizeOf(accountables);
for (Accountable accountable : accountables) {
if (accountable != null) {
size += accountable.ramBytesUsed();
}
}
return size;
}
}

View File

@ -26,8 +26,8 @@ import org.apache.lucene.search.DocIdSet;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.util.packed.MonotonicAppendingLongBuffer;
import org.apache.lucene.util.packed.PackedInts;
import org.apache.lucene.util.packed.PackedLongValues;
/**
* {@link DocIdSet} implementation based on word-aligned hybrid encoding on
@ -88,13 +88,8 @@ public final class WAH8DocIdSet extends DocIdSet implements Accountable {
/** Default index interval. */
public static final int DEFAULT_INDEX_INTERVAL = 24;
private static final MonotonicAppendingLongBuffer SINGLE_ZERO_BUFFER = new MonotonicAppendingLongBuffer(1, 64, PackedInts.COMPACT);
private static WAH8DocIdSet EMPTY = new WAH8DocIdSet(new byte[0], 0, 1, SINGLE_ZERO_BUFFER, SINGLE_ZERO_BUFFER);
static {
SINGLE_ZERO_BUFFER.add(0L);
SINGLE_ZERO_BUFFER.freeze();
}
private static final PackedLongValues SINGLE_ZERO = PackedLongValues.packedBuilder(PackedInts.COMPACT).add(0L).build();
private static WAH8DocIdSet EMPTY = new WAH8DocIdSet(new byte[0], 0, 1, SINGLE_ZERO, SINGLE_ZERO);
private static final Comparator<Iterator> SERIALIZED_LENGTH_COMPARATOR = new Comparator<Iterator>() {
@Override
@ -377,18 +372,17 @@ public final class WAH8DocIdSet extends DocIdSet implements Accountable {
// Now build the index
final int valueCount = (numSequences - 1) / indexInterval + 1;
final MonotonicAppendingLongBuffer indexPositions, indexWordNums;
final PackedLongValues indexPositions, indexWordNums;
if (valueCount <= 1) {
indexPositions = indexWordNums = SINGLE_ZERO_BUFFER;
indexPositions = indexWordNums = SINGLE_ZERO;
} else {
final int pageSize = 128;
final int initialPageCount = (valueCount + pageSize - 1) / pageSize;
final MonotonicAppendingLongBuffer positions = new MonotonicAppendingLongBuffer(initialPageCount, pageSize, PackedInts.COMPACT);
final MonotonicAppendingLongBuffer wordNums = new MonotonicAppendingLongBuffer(initialPageCount, pageSize, PackedInts.COMPACT);
final PackedLongValues.Builder positions = PackedLongValues.monotonicBuilder(pageSize, PackedInts.COMPACT);
final PackedLongValues.Builder wordNums = PackedLongValues.monotonicBuilder(pageSize, PackedInts.COMPACT);
positions.add(0L);
wordNums.add(0L);
final Iterator it = new Iterator(data, cardinality, Integer.MAX_VALUE, SINGLE_ZERO_BUFFER, SINGLE_ZERO_BUFFER);
final Iterator it = new Iterator(data, cardinality, Integer.MAX_VALUE, SINGLE_ZERO, SINGLE_ZERO);
assert it.in.getPosition() == 0;
assert it.wordNum == -1;
for (int i = 1; i < valueCount; ++i) {
@ -403,10 +397,8 @@ public final class WAH8DocIdSet extends DocIdSet implements Accountable {
positions.add(position);
wordNums.add(wordNum + 1);
}
positions.freeze();
wordNums.freeze();
indexPositions = positions;
indexWordNums = wordNums;
indexPositions = positions.build();
indexWordNums = wordNums.build();
}
return new WAH8DocIdSet(data, cardinality, indexInterval, indexPositions, indexWordNums);
@ -476,9 +468,9 @@ public final class WAH8DocIdSet extends DocIdSet implements Accountable {
private final int cardinality;
private final int indexInterval;
// index for advance(int)
private final MonotonicAppendingLongBuffer positions, wordNums; // wordNums[i] starts at the sequence at positions[i]
private final PackedLongValues positions, wordNums; // wordNums[i] starts at the sequence at positions[i]
WAH8DocIdSet(byte[] data, int cardinality, int indexInterval, MonotonicAppendingLongBuffer positions, MonotonicAppendingLongBuffer wordNums) {
WAH8DocIdSet(byte[] data, int cardinality, int indexInterval, PackedLongValues positions, PackedLongValues wordNums) {
this.data = data;
this.cardinality = cardinality;
this.indexInterval = indexInterval;
@ -530,7 +522,7 @@ public final class WAH8DocIdSet extends DocIdSet implements Accountable {
final ByteArrayDataInput in;
final int cardinality;
final int indexInterval;
final MonotonicAppendingLongBuffer positions, wordNums;
final PackedLongValues positions, wordNums;
final int indexThreshold;
int allOnesLength;
int dirtyLength;
@ -542,7 +534,7 @@ public final class WAH8DocIdSet extends DocIdSet implements Accountable {
int docID;
Iterator(byte[] data, int cardinality, int indexInterval, MonotonicAppendingLongBuffer positions, MonotonicAppendingLongBuffer wordNums) {
Iterator(byte[] data, int cardinality, int indexInterval, PackedLongValues positions, PackedLongValues wordNums) {
this.in = new ByteArrayDataInput(data);
this.cardinality = cardinality;
this.indexInterval = indexInterval;
@ -744,10 +736,10 @@ public final class WAH8DocIdSet extends DocIdSet implements Accountable {
return 0L;
}
long ramBytesUsed = BASE_RAM_BYTES_USED + RamUsageEstimator.sizeOf(data);
if (positions != SINGLE_ZERO_BUFFER) {
if (positions != SINGLE_ZERO) {
ramBytesUsed += positions.ramBytesUsed();
}
if (wordNums != SINGLE_ZERO_BUFFER) {
if (wordNums != SINGLE_ZERO) {
ramBytesUsed += wordNums.ramBytesUsed();
}
return ramBytesUsed;

View File

@ -1,216 +0,0 @@
package org.apache.lucene.util.packed;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.LongValues;
import org.apache.lucene.util.RamUsageEstimator;
import java.util.Arrays;
import static org.apache.lucene.util.packed.PackedInts.checkBlockSize;
/** Common functionality shared by {@link AppendingDeltaPackedLongBuffer} and {@link MonotonicAppendingLongBuffer}. */
abstract class AbstractAppendingLongBuffer extends LongValues implements Accountable {
static final int MIN_PAGE_SIZE = 64;
// More than 1M doesn't really makes sense with these appending buffers
// since their goal is to try to have small numbers of bits per value
static final int MAX_PAGE_SIZE = 1 << 20;
final int pageShift, pageMask;
PackedInts.Reader[] values;
private long valuesBytes;
int valuesOff;
long[] pending;
int pendingOff;
float acceptableOverheadRatio;
AbstractAppendingLongBuffer(int initialBlockCount, int pageSize, float acceptableOverheadRatio) {
values = new PackedInts.Reader[initialBlockCount];
pending = new long[pageSize];
pageShift = checkBlockSize(pageSize, MIN_PAGE_SIZE, MAX_PAGE_SIZE);
pageMask = pageSize - 1;
valuesOff = 0;
pendingOff = 0;
this.acceptableOverheadRatio = acceptableOverheadRatio;
}
final int pageSize() {
return pageMask + 1;
}
/** Get the number of values that have been added to the buffer. */
public final long size() {
long size = pendingOff;
if (valuesOff > 0) {
size += values[valuesOff - 1].size();
}
if (valuesOff > 1) {
size += (long) (valuesOff - 1) * pageSize();
}
return size;
}
/** Append a value to this buffer. */
public final void add(long l) {
if (pending == null) {
throw new IllegalStateException("This buffer is frozen");
}
if (pendingOff == pending.length) {
// check size
if (values.length == valuesOff) {
final int newLength = ArrayUtil.oversize(valuesOff + 1, 8);
grow(newLength);
}
packPendingValues();
valuesBytes += values[valuesOff].ramBytesUsed();
++valuesOff;
// reset pending buffer
pendingOff = 0;
}
pending[pendingOff++] = l;
}
void grow(int newBlockCount) {
values = Arrays.copyOf(values, newBlockCount);
}
abstract void packPendingValues();
@Override
public final long get(long index) {
assert index >= 0 && index < size();
final int block = (int) (index >> pageShift);
final int element = (int) (index & pageMask);
return get(block, element);
}
/**
* Bulk get: read at least one and at most <code>len</code> longs starting
* from <code>index</code> into <code>arr[off:off+len]</code> and return
* the actual number of values that have been read.
*/
public final int get(long index, long[] arr, int off, int len) {
assert len > 0 : "len must be > 0 (got " + len + ")";
assert index >= 0 && index < size();
assert off + len <= arr.length;
int block = (int) (index >> pageShift);
int element = (int) (index & pageMask);
return get(block, element, arr, off, len);
}
abstract long get(int block, int element);
abstract int get(int block, int element, long[] arr, int off, int len);
/** Return an iterator over the values of this buffer. */
public Iterator iterator() {
return new Iterator();
}
final public class Iterator {
long[] currentValues;
int vOff, pOff;
int currentCount; // number of entries of the current page
Iterator() {
vOff = pOff = 0;
if (valuesOff == 0) {
currentValues = pending;
currentCount = pendingOff;
} else {
currentValues = new long[values[0].size()];
fillValues();
}
}
void fillValues() {
if (vOff == valuesOff) {
currentValues = pending;
currentCount = pendingOff;
} else {
currentCount = values[vOff].size();
for (int k = 0; k < currentCount; ) {
k += get(vOff, k, currentValues, k, currentCount - k);
}
}
}
/** Whether or not there are remaining values. */
public final boolean hasNext() {
return pOff < currentCount;
}
/** Return the next long in the buffer. */
public final long next() {
assert hasNext();
long result = currentValues[pOff++];
if (pOff == currentCount) {
vOff += 1;
pOff = 0;
if (vOff <= valuesOff) {
fillValues();
} else {
currentCount = 0;
}
}
return result;
}
}
long baseRamBytesUsed() {
return RamUsageEstimator.NUM_BYTES_OBJECT_HEADER
+ 2 * RamUsageEstimator.NUM_BYTES_OBJECT_REF // the 2 arrays
+ 2 * RamUsageEstimator.NUM_BYTES_INT // the 2 offsets
+ 2 * RamUsageEstimator.NUM_BYTES_INT // pageShift, pageMask
+ RamUsageEstimator.NUM_BYTES_FLOAT // acceptable overhead
+ RamUsageEstimator.NUM_BYTES_LONG; // valuesBytes
}
@Override
public long ramBytesUsed() {
// TODO: this is called per-doc-per-norms/dv-field, can we optimize this?
long bytesUsed = RamUsageEstimator.alignObjectSize(baseRamBytesUsed())
+ (pending != null ? RamUsageEstimator.sizeOf(pending) : 0L)
+ RamUsageEstimator.shallowSizeOf(values);
return bytesUsed + valuesBytes;
}
/** Pack all pending values in this buffer. Subsequent calls to {@link #add(long)} will fail. */
public void freeze() {
if (pendingOff > 0) {
if (values.length == valuesOff) {
grow(valuesOff + 1); // don't oversize!
}
packPendingValues();
valuesBytes += values[valuesOff].ramBytesUsed();
++valuesOff;
pendingOff = 0;
}
pending = null;
}
}

View File

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

View File

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

View File

@ -0,0 +1,103 @@
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.RamUsageEstimator;
import org.apache.lucene.util.packed.PackedInts.Reader;
class DeltaPackedLongValues extends PackedLongValues {
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(DeltaPackedLongValues.class);
final long[] mins;
DeltaPackedLongValues(int pageShift, int pageMask, Reader[] values, long[] mins, long size, long ramBytesUsed) {
super(pageShift, pageMask, values, size, ramBytesUsed);
assert values.length == mins.length;
this.mins = mins;
}
@Override
long get(int block, int element) {
return mins[block] + values[block].get(element);
}
@Override
int decodeBlock(int block, long[] dest) {
final int count = super.decodeBlock(block, dest);
final long min = mins[block];
for (int i = 0; i < count; ++i) {
dest[i] += min;
}
return count;
}
static class Builder extends PackedLongValues.Builder {
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(Builder.class);
long[] mins;
Builder(int pageSize, float acceptableOverheadRatio) {
super(pageSize, acceptableOverheadRatio);
mins = new long[values.length];
ramBytesUsed += RamUsageEstimator.sizeOf(mins);
}
@Override
long baseRamBytesUsed() {
return BASE_RAM_BYTES_USED;
}
@Override
public DeltaPackedLongValues build() {
finish();
pending = null;
final PackedInts.Reader[] values = Arrays.copyOf(this.values, valuesOff);
final long[] mins = Arrays.copyOf(this.mins, valuesOff);
final long ramBytesUsed = DeltaPackedLongValues.BASE_RAM_BYTES_USED
+ RamUsageEstimator.sizeOf(values) + RamUsageEstimator.sizeOf(mins);
return new DeltaPackedLongValues(pageShift, pageMask, values, mins, size, ramBytesUsed);
}
@Override
void pack(long[] values, int numValues, int block, float acceptableOverheadRatio) {
long min = values[0];
for (int i = 1; i < numValues; ++i) {
min = Math.min(min, values[i]);
}
for (int i = 0; i < numValues; ++i) {
values[i] -= min;
}
super.pack(values, numValues, block, acceptableOverheadRatio);
mins[block] = min;
}
@Override
void grow(int newBlockCount) {
super.grow(newBlockCount);
ramBytesUsed -= RamUsageEstimator.sizeOf(mins);
mins = Arrays.copyOf(mins, newBlockCount);
ramBytesUsed += RamUsageEstimator.sizeOf(mins);
}
}
}

View File

@ -1,149 +0,0 @@
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 static org.apache.lucene.util.packed.MonotonicBlockPackedReader.expected;
import java.util.Arrays;
import org.apache.lucene.util.RamUsageEstimator;
/**
* Utility class to buffer signed longs in memory, which is optimized for the
* case where the sequence is monotonic, although it can encode any sequence of
* arbitrary longs. It only supports appending.
*
* @lucene.internal
*/
public final class MonotonicAppendingLongBuffer extends AbstractAppendingLongBuffer {
float[] averages;
long[] minValues;
/**
* @param initialPageCount the initial number of pages
* @param pageSize the size of a single page
* @param acceptableOverheadRatio an acceptable overhead ratio per value
*/
public MonotonicAppendingLongBuffer(int initialPageCount, int pageSize, float acceptableOverheadRatio) {
super(initialPageCount, pageSize, acceptableOverheadRatio);
averages = new float[values.length];
minValues = new long[values.length];
}
/**
* Create an {@link MonotonicAppendingLongBuffer} with initialPageCount=16,
* pageSize=1024 and acceptableOverheadRatio={@link PackedInts#DEFAULT}
*/
public MonotonicAppendingLongBuffer() {
this(16, 1024, PackedInts.DEFAULT);
}
/**
* Create an {@link AppendingDeltaPackedLongBuffer} with initialPageCount=16,
* pageSize=1024
*/
public MonotonicAppendingLongBuffer(float acceptableOverheadRatio) {
this(16, 1024, acceptableOverheadRatio);
}
@Override
long get(int block, int element) {
if (block == valuesOff) {
return pending[element];
} else {
return expected(minValues[block], averages[block], element) + values[block].get(element);
}
}
@Override
int get(int block, int element, long[] arr, int off, int len) {
if (block == valuesOff) {
int sysCopyToRead = Math.min(len, pendingOff - element);
System.arraycopy(pending, element, arr, off, sysCopyToRead);
return sysCopyToRead;
} else {
int read = values[block].get(element, arr, off, len);
for (int r = 0; r < read; r++, off++, element++) {
arr[off] += expected(minValues[block], averages[block], element);
}
return read;
}
}
@Override
void grow(int newBlockCount) {
super.grow(newBlockCount);
this.averages = Arrays.copyOf(averages, newBlockCount);
this.minValues = Arrays.copyOf(minValues, newBlockCount);
}
@Override
void packPendingValues() {
assert pendingOff > 0;
final float average = pendingOff == 1 ? 0 : (float) (pending[pendingOff - 1] - pending[0]) / (pendingOff - 1);
long minValue = pending[0];
// adjust minValue so that all deltas will be positive
for (int i = 1; i < pendingOff; ++i) {
final long actual = pending[i];
final long expected = expected(minValue, average, i);
if (expected > actual) {
minValue -= (expected - actual);
}
}
minValues[valuesOff] = minValue;
averages[valuesOff] = average;
for (int i = 0; i < pendingOff; ++i) {
pending[i] = pending[i] - expected(minValue, average, i);
}
long maxDelta = 0;
for (int i = 0; i < pendingOff; ++i) {
if (pending[i] < 0) {
maxDelta = -1;
break;
} else {
maxDelta = Math.max(maxDelta, pending[i]);
}
}
if (maxDelta == 0) {
values[valuesOff] = new PackedInts.NullReader(pendingOff);
} else {
final int bitsRequired = PackedInts.unsignedBitsRequired(maxDelta);
final PackedInts.Mutable mutable = PackedInts.getMutable(pendingOff, bitsRequired, acceptableOverheadRatio);
for (int i = 0; i < pendingOff; ) {
i += mutable.set(i, pending, i, pendingOff - i);
}
values[valuesOff] = mutable;
}
}
@Override
long baseRamBytesUsed() {
return super.baseRamBytesUsed()
+ 2 * RamUsageEstimator.NUM_BYTES_OBJECT_REF; // 2 additional arrays
}
@Override
public long ramBytesUsed() {
return super.ramBytesUsed()
+ RamUsageEstimator.sizeOf(averages) + RamUsageEstimator.sizeOf(minValues);
}
}

View File

@ -0,0 +1,105 @@
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 static org.apache.lucene.util.packed.MonotonicBlockPackedReader.expected;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.packed.DeltaPackedLongValues.Builder;
import org.apache.lucene.util.packed.PackedInts.Reader;
class MonotonicLongValues extends DeltaPackedLongValues {
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(MonotonicLongValues.class);
final float[] averages;
MonotonicLongValues(int pageShift, int pageMask, Reader[] values, long[] mins, float[] averages, long size, long ramBytesUsed) {
super(pageShift, pageMask, values, mins, size, ramBytesUsed);
assert values.length == averages.length;
this.averages = averages;
}
@Override
long get(int block, int element) {
return expected(mins[block], averages[block], element) + values[block].get(element);
}
@Override
int decodeBlock(int block, long[] dest) {
final int count = super.decodeBlock(block, dest);
final float average = averages[block];
for (int i = 0; i < count; ++i) {
dest[i] += expected(0, average, i);
}
return count;
}
static class Builder extends DeltaPackedLongValues.Builder {
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(Builder.class);
float[] averages;
Builder(int pageSize, float acceptableOverheadRatio) {
super(pageSize, acceptableOverheadRatio);
averages = new float[values.length];
ramBytesUsed += RamUsageEstimator.sizeOf(averages);
}
@Override
long baseRamBytesUsed() {
return BASE_RAM_BYTES_USED;
}
@Override
public MonotonicLongValues build() {
finish();
pending = null;
final PackedInts.Reader[] values = Arrays.copyOf(this.values, valuesOff);
final long[] mins = Arrays.copyOf(this.mins, valuesOff);
final float[] averages = Arrays.copyOf(this.averages, valuesOff);
final long ramBytesUsed = MonotonicLongValues.BASE_RAM_BYTES_USED
+ RamUsageEstimator.sizeOf(values) + RamUsageEstimator.sizeOf(mins)
+ RamUsageEstimator.sizeOf(averages);
return new MonotonicLongValues(pageShift, pageMask, values, mins, averages, size, ramBytesUsed);
}
@Override
void pack(long[] values, int numValues, int block, float acceptableOverheadRatio) {
final float average = numValues == 1 ? 0 : (float) (values[numValues - 1] - values[0]) / (numValues - 1);
for (int i = 0; i < numValues; ++i) {
values[i] -= expected(0, average, i);
}
super.pack(values, numValues, block, acceptableOverheadRatio);
averages[block] = average;
}
@Override
void grow(int newBlockCount) {
super.grow(newBlockCount);
ramBytesUsed -= RamUsageEstimator.sizeOf(averages);
averages = Arrays.copyOf(averages, newBlockCount);
ramBytesUsed += RamUsageEstimator.sizeOf(averages);
}
}
}

View File

@ -0,0 +1,282 @@
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 static org.apache.lucene.util.packed.PackedInts.checkBlockSize;
import java.util.Arrays;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.LongValues;
import org.apache.lucene.util.RamUsageEstimator;
/**
* Utility class to compress integers into a {@link LongValues} instance.
*/
public class PackedLongValues extends LongValues implements Accountable {
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(PackedLongValues.class);
static final int DEFAULT_PAGE_SIZE = 1024;
static final int MIN_PAGE_SIZE = 64;
// More than 1M doesn't really makes sense with these appending buffers
// since their goal is to try to have small numbers of bits per value
static final int MAX_PAGE_SIZE = 1 << 20;
/** Return a new {@link Builder} that will compress efficiently positive integers. */
public static PackedLongValues.Builder packedBuilder(int pageSize, float acceptableOverheadRatio) {
return new PackedLongValues.Builder(pageSize, acceptableOverheadRatio);
}
/** @see #packedBuilder(int, float) */
public static PackedLongValues.Builder packedBuilder(float acceptableOverheadRatio) {
return packedBuilder(DEFAULT_PAGE_SIZE, acceptableOverheadRatio);
}
/** Return a new {@link Builder} that will compress efficiently integers that
* are close to each other. */
public static PackedLongValues.Builder deltaPackedBuilder(int pageSize, float acceptableOverheadRatio) {
return new DeltaPackedLongValues.Builder(pageSize, acceptableOverheadRatio);
}
/** @see #deltaPackedBuilder(int, float) */
public static PackedLongValues.Builder deltaPackedBuilder(float acceptableOverheadRatio) {
return deltaPackedBuilder(DEFAULT_PAGE_SIZE, acceptableOverheadRatio);
}
/** Return a new {@link Builder} that will compress efficiently integers that
* would be a monotonic function of their index. */
public static PackedLongValues.Builder monotonicBuilder(int pageSize, float acceptableOverheadRatio) {
return new MonotonicLongValues.Builder(pageSize, acceptableOverheadRatio);
}
/** @see #monotonicBuilder(int, float) */
public static PackedLongValues.Builder monotonicBuilder(float acceptableOverheadRatio) {
return monotonicBuilder(DEFAULT_PAGE_SIZE, acceptableOverheadRatio);
}
final PackedInts.Reader[] values;
final int pageShift, pageMask;
private final long size;
private final long ramBytesUsed;
PackedLongValues(int pageShift, int pageMask, PackedInts.Reader[] values, long size, long ramBytesUsed) {
this.pageShift = pageShift;
this.pageMask = pageMask;
this.values = values;
this.size = size;
this.ramBytesUsed = ramBytesUsed;
}
/** Get the number of values in this array. */
public final long size() {
return size;
}
int decodeBlock(int block, long[] dest) {
final PackedInts.Reader vals = values[block];
final int size = vals.size();
for (int k = 0; k < size; ) {
k += vals.get(k, dest, k, size - k);
}
return size;
}
long get(int block, int element) {
return values[block].get(element);
}
@Override
public final long get(long index) {
assert index >= 0 && index < size();
final int block = (int) (index >> pageShift);
final int element = (int) (index & pageMask);
return get(block, element);
}
@Override
public long ramBytesUsed() {
return ramBytesUsed;
}
/** Return an iterator over the values of this array. */
public Iterator iterator() {
return new Iterator();
}
/** An iterator over long values. */
final public class Iterator {
final long[] currentValues;
int vOff, pOff;
int currentCount; // number of entries of the current page
Iterator() {
currentValues = new long[pageMask + 1];
vOff = pOff = 0;
fillBlock();
}
private void fillBlock() {
if (vOff == values.length) {
currentCount = 0;
} else {
currentCount = decodeBlock(vOff, currentValues);
assert currentCount > 0;
}
}
/** Whether or not there are remaining values. */
public final boolean hasNext() {
return pOff < currentCount;
}
/** Return the next long in the buffer. */
public final long next() {
assert hasNext();
long result = currentValues[pOff++];
if (pOff == currentCount) {
vOff += 1;
pOff = 0;
fillBlock();
}
return result;
}
}
/** A Builder for a {@link PackedLongValues} instance. */
public static class Builder implements Accountable {
private static final int INITIAL_PAGE_COUNT = 16;
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(Builder.class);
final int pageShift, pageMask;
final float acceptableOverheadRatio;
long[] pending;
long size;
PackedInts.Reader[] values;
long ramBytesUsed;
int valuesOff;
int pendingOff;
Builder(int pageSize, float acceptableOverheadRatio) {
pageShift = checkBlockSize(pageSize, MIN_PAGE_SIZE, MAX_PAGE_SIZE);
pageMask = pageSize - 1;
this.acceptableOverheadRatio = acceptableOverheadRatio;
values = new PackedInts.Reader[INITIAL_PAGE_COUNT];
pending = new long[pageSize];
valuesOff = 0;
pendingOff = 0;
size = 0;
ramBytesUsed = baseRamBytesUsed() + RamUsageEstimator.sizeOf(pending) + RamUsageEstimator.shallowSizeOf(values);
}
/** Build a {@link PackedLongValues} instance that contains values that
* have been added to this builder. This operation is destructive. */
public PackedLongValues build() {
finish();
pending = null;
final PackedInts.Reader[] values = Arrays.copyOf(this.values, valuesOff);
final long ramBytesUsed = PackedLongValues.BASE_RAM_BYTES_USED + RamUsageEstimator.sizeOf(values);
return new PackedLongValues(pageShift, pageMask, values, size, ramBytesUsed);
}
long baseRamBytesUsed() {
return BASE_RAM_BYTES_USED;
}
@Override
public final long ramBytesUsed() {
return ramBytesUsed;
}
/** Return the number of elements that have been added to this builder. */
public final long size() {
return size;
}
/** Add a new element to this builder. */
public Builder add(long l) {
if (pending == null) {
throw new IllegalStateException("Cannot be reused after build()");
}
if (pendingOff == pending.length) {
// check size
if (values.length == valuesOff) {
final int newLength = ArrayUtil.oversize(valuesOff + 1, 8);
grow(newLength);
}
pack();
}
pending[pendingOff++] = l;
size += 1;
return this;
}
final void finish() {
if (pendingOff > 0) {
if (values.length == valuesOff) {
grow(valuesOff + 1);
}
pack();
}
}
private void pack() {
pack(pending, pendingOff, valuesOff, acceptableOverheadRatio);
ramBytesUsed += values[valuesOff].ramBytesUsed();
valuesOff += 1;
// reset pending buffer
pendingOff = 0;
}
void pack(long[] values, int numValues, int block, float acceptableOverheadRatio) {
assert numValues > 0;
// compute max delta
long minValue = values[0];
long maxValue = values[0];
for (int i = 1; i < numValues; ++i) {
minValue = Math.min(minValue, values[i]);
maxValue = Math.max(maxValue, values[i]);
}
// build a new packed reader
if (minValue == 0 && maxValue == 0) {
this.values[block] = new PackedInts.NullReader(numValues);
} else {
final int bitsRequired = minValue < 0 ? 64 : PackedInts.bitsRequired(maxValue);
final PackedInts.Mutable mutable = PackedInts.getMutable(numValues, bitsRequired, acceptableOverheadRatio);
for (int i = 0; i < numValues; ) {
i += mutable.set(i, values, i, numValues - i);
}
this.values[block] = mutable;
}
}
void grow(int newBlockCount) {
ramBytesUsed -= RamUsageEstimator.shallowSizeOf(values);
values = Arrays.copyOf(values, newBlockCount);
ramBytesUsed += RamUsageEstimator.shallowSizeOf(values);
}
}
}

View File

@ -23,7 +23,7 @@ import org.apache.lucene.util.packed.PackedInts.Mutable;
/**
* A {@link PagedGrowableWriter}. This class slices data into fixed-size blocks
* which have independent numbers of bits per value and grow on-demand.
* <p>You should use this class instead of the {@link AbstractAppendingLongBuffer} related ones only when
* <p>You should use this class instead of the {@link PackedLongValues} related ones only when
* you need random write-access. Otherwise this class will likely be slower and
* less memory-efficient.
* @lucene.internal

View File

@ -50,19 +50,19 @@
<li><b>{@link org.apache.lucene.util.packed.PagedGrowableWriter}</b><ul>
<li>Slices data into fixed-size blocks stored in GrowableWriters.</li>
<li>Supports more than 2B values.</li>
<li>You should use Appending(Delta)PackedLongBuffer instead if you don't need random write access.</li>
<li>You should use PackedLongValues instead if you don't need random write access.</li>
</ul></li>
<li><b>{@link org.apache.lucene.util.packed.AppendingDeltaPackedLongBuffer}</b><ul>
<li><b>{@link org.apache.lucene.util.packed.PackedLongValues#deltaPackedBuilder}</b><ul>
<li>Can store any sequence of longs.</li>
<li>Compression is good when values are close to each other.</li>
<li>Supports random reads, but only sequential writes.</li>
<li>Can address up to 2^42 values.</li>
</ul></li>
<li><b>{@link org.apache.lucene.util.packed.AppendingPackedLongBuffer}</b><ul>
<li>Same as AppendingDeltaPackedLongBuffer but assumes values are 0-based.</li>
<li><b>{@link org.apache.lucene.util.packed.PackedLongValues#packedBuilder}</b><ul>
<li>Same as deltaPackedBuilder but assumes values are 0-based.</li>
</ul></li>
<li><b>{@link org.apache.lucene.util.packed.MonotonicAppendingLongBuffer}</b><ul>
<li>Same as AppendingDeltaPackedLongBuffer except that compression is good when the stream is a succession of affine functions.</li>
<li><b>{@link org.apache.lucene.util.packed.PackedLongValues#monotonicBuilder}</b><ul>
<li>Same as deltaPackedBuilder except that compression is good when the stream is a succession of affine functions.</li>
</ul></li>
</ul>

View File

@ -34,10 +34,10 @@ import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.RAMDirectory;
import org.apache.lucene.util.LongValues;
import org.apache.lucene.util.LongsRef;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.LuceneTestCase.Slow;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.RamUsageTester;
import org.apache.lucene.util.TestUtil;
import org.apache.lucene.util.packed.PackedInts.Reader;
@ -701,7 +701,7 @@ public class TestPackedInts extends LuceneTestCase {
assertEquals(0, writer.size());
// compare against AppendingDeltaPackedLongBuffer
AppendingDeltaPackedLongBuffer buf = new AppendingDeltaPackedLongBuffer();
PackedLongValues.Builder buf = PackedLongValues.deltaPackedBuilder(random().nextFloat());
int size = random().nextInt(1000000);
long max = 5;
for (int i = 0; i < size; ++i) {
@ -712,11 +712,12 @@ public class TestPackedInts extends LuceneTestCase {
}
writer = new PagedGrowableWriter(size, pageSize, TestUtil.nextInt(random(), 1, 64), random().nextFloat());
assertEquals(size, writer.size());
final LongValues values = buf.build();
for (int i = size - 1; i >= 0; --i) {
writer.set(i, buf.get(i));
writer.set(i, values.get(i));
}
for (int i = 0; i < size; ++i) {
assertEquals(buf.get(i), writer.get(i));
assertEquals(values.get(i), writer.get(i));
}
// test ramBytesUsed
@ -752,7 +753,7 @@ public class TestPackedInts extends LuceneTestCase {
assertEquals(0, writer.size());
// compare against AppendingDeltaPackedLongBuffer
AppendingDeltaPackedLongBuffer buf = new AppendingDeltaPackedLongBuffer();
PackedLongValues.Builder buf = PackedLongValues.deltaPackedBuilder(random().nextFloat());
int size = random().nextInt(1000000);
for (int i = 0; i < size; ++i) {
@ -760,11 +761,12 @@ public class TestPackedInts extends LuceneTestCase {
}
writer = new PagedMutable(size, pageSize, bitsPerValue, random().nextFloat());
assertEquals(size, writer.size());
final LongValues values = buf.build();
for (int i = size - 1; i >= 0; --i) {
writer.set(i, buf.get(i));
writer.set(i, values.get(i));
}
for (int i = 0; i < size; ++i) {
assertEquals(buf.get(i), writer.get(i));
assertEquals(values.get(i), writer.get(i));
}
// test ramBytesUsed
@ -960,29 +962,46 @@ public class TestPackedInts extends LuceneTestCase {
MONOTONIC
}
public void testPackedLongValuesOnZeros() {
// Make sure that when all values are the same, they use 0 bits per value
final int pageSize = 1 << TestUtil.nextInt(random(), 6, 20);
final float acceptableOverheadRatio = random().nextFloat();
public void testAppendingLongBuffer() {
assertEquals(
PackedLongValues.packedBuilder(pageSize, acceptableOverheadRatio).add(0).build().ramBytesUsed(),
PackedLongValues.packedBuilder(pageSize, acceptableOverheadRatio).add(0).add(0).build().ramBytesUsed());
final long l = random().nextLong();
assertEquals(
PackedLongValues.deltaPackedBuilder(pageSize, acceptableOverheadRatio).add(l).build().ramBytesUsed(),
PackedLongValues.deltaPackedBuilder(pageSize, acceptableOverheadRatio).add(l).add(l).build().ramBytesUsed());
final long avg = random().nextInt(100);
assertEquals(
PackedLongValues.monotonicBuilder(pageSize, acceptableOverheadRatio).add(l).add(l + avg).build().ramBytesUsed(),
PackedLongValues.monotonicBuilder(pageSize, acceptableOverheadRatio).add(l).add(l + avg).add(l + 2 * avg).build().ramBytesUsed());
}
public void testPackedLongValues() {
final long[] arr = new long[RandomInts.randomIntBetween(random(), 1, 1000000)];
float[] ratioOptions = new float[]{PackedInts.DEFAULT, PackedInts.COMPACT, PackedInts.FAST};
for (int bpv : new int[]{0, 1, 63, 64, RandomInts.randomIntBetween(random(), 2, 62)}) {
for (DataType dataType : DataType.values()) {
for (DataType dataType : Arrays.asList(DataType.DELTA_PACKED)) {
final int pageSize = 1 << TestUtil.nextInt(random(), 6, 20);
final int initialPageCount = TestUtil.nextInt(random(), 0, 16);
float acceptableOverheadRatio = ratioOptions[TestUtil.nextInt(random(), 0, ratioOptions.length - 1)];
AbstractAppendingLongBuffer buf;
PackedLongValues.Builder buf;
final int inc;
switch (dataType) {
case PACKED:
buf = new AppendingPackedLongBuffer(initialPageCount, pageSize, acceptableOverheadRatio);
buf = PackedLongValues.packedBuilder(pageSize, acceptableOverheadRatio);
inc = 0;
break;
case DELTA_PACKED:
buf = new AppendingDeltaPackedLongBuffer(initialPageCount, pageSize, acceptableOverheadRatio);
buf = PackedLongValues.deltaPackedBuilder(pageSize, acceptableOverheadRatio);
inc = 0;
break;
case MONOTONIC:
buf = new MonotonicAppendingLongBuffer(initialPageCount, pageSize, acceptableOverheadRatio);
buf = PackedLongValues.monotonicBuilder(pageSize, acceptableOverheadRatio);
inc = TestUtil.nextInt(random(), -1000, 1000);
break;
default:
@ -1008,22 +1027,27 @@ public class TestPackedInts extends LuceneTestCase {
for (int i = 0; i < arr.length; ++i) {
buf.add(arr[i]);
}
assertEquals(arr.length, buf.size());
if (random().nextBoolean()) {
buf.freeze();
if (random().nextBoolean()) {
// Make sure double freeze doesn't break anything
buf.freeze();
if (rarely()) {
final long expectedBytesUsed = RamUsageTester.sizeOf(buf);
final long computedBytesUsed = buf.ramBytesUsed();
assertEquals(expectedBytesUsed, computedBytesUsed);
}
}
assertEquals(arr.length, buf.size());
final PackedLongValues values = buf.build();
try {
buf.add(random().nextLong());
fail("expected an exception");
} catch (IllegalStateException e) {
// ok
}
assertEquals(arr.length, values.size());
for (int i = 0; i < arr.length; ++i) {
assertEquals(arr[i], buf.get(i));
assertEquals(arr[i], values.get(i));
}
final AbstractAppendingLongBuffer.Iterator it = buf.iterator();
final PackedLongValues.Iterator it = values.iterator();
for (int i = 0; i < arr.length; ++i) {
if (random().nextBoolean()) {
assertTrue(it.hasNext());
@ -1032,28 +1056,8 @@ public class TestPackedInts extends LuceneTestCase {
}
assertFalse(it.hasNext());
long[] target = new long[arr.length + 1024]; // check the request for more is OK.
for (int i = 0; i < arr.length; i += TestUtil.nextInt(random(), 0, 10000)) {
int lenToRead = random().nextInt(buf.pageSize() * 2) + 1;
lenToRead = Math.min(lenToRead, target.length - i);
int lenToCheck = Math.min(lenToRead, arr.length - i);
int off = i;
while (off < arr.length && lenToRead > 0) {
int read = buf.get(off, target, off, lenToRead);
assertTrue(read > 0);
assertTrue(read <= lenToRead);
lenToRead -= read;
off += read;
}
for (int j = 0; j < lenToCheck; j++) {
assertEquals(arr[j + i], target[j + i]);
}
}
final long expectedBytesUsed = RamUsageTester.sizeOf(buf);
final long computedBytesUsed = buf.ramBytesUsed();
final long expectedBytesUsed = RamUsageTester.sizeOf(values);
final long computedBytesUsed = values.ramBytesUsed();
assertEquals(expectedBytesUsed, computedBytesUsed);
}
}

View File

@ -27,7 +27,8 @@ import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.util.TimSorter;
import org.apache.lucene.util.packed.MonotonicAppendingLongBuffer;
import org.apache.lucene.util.packed.PackedInts;
import org.apache.lucene.util.packed.PackedLongValues;
/**
* Sorts documents of a given index by returning a permutation on the document
@ -163,21 +164,21 @@ final class Sorter {
// The reason why we use MonotonicAppendingLongBuffer here is that it
// wastes very little memory if the index is in random order but can save
// a lot of memory if the index is already "almost" sorted
final MonotonicAppendingLongBuffer newToOld = new MonotonicAppendingLongBuffer();
final PackedLongValues.Builder newToOldBuilder = PackedLongValues.monotonicBuilder(PackedInts.COMPACT);
for (int i = 0; i < maxDoc; ++i) {
newToOld.add(docs[i]);
newToOldBuilder.add(docs[i]);
}
newToOld.freeze();
final PackedLongValues newToOld = newToOldBuilder.build();
for (int i = 0; i < maxDoc; ++i) {
docs[(int) newToOld.get(i)] = i;
} // docs is now the oldToNew mapping
final MonotonicAppendingLongBuffer oldToNew = new MonotonicAppendingLongBuffer();
final PackedLongValues.Builder oldToNewBuilder = PackedLongValues.monotonicBuilder(PackedInts.COMPACT);
for (int i = 0; i < maxDoc; ++i) {
oldToNew.add(docs[i]);
oldToNewBuilder.add(docs[i]);
}
oldToNew.freeze();
final PackedLongValues oldToNew = oldToNewBuilder.build();
return new Sorter.DocMap() {

View File

@ -38,7 +38,8 @@ import org.apache.lucene.index.SlowCompositeReaderWrapper;
import org.apache.lucene.search.Sort;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.packed.MonotonicAppendingLongBuffer;
import org.apache.lucene.util.packed.PackedInts;
import org.apache.lucene.util.packed.PackedLongValues;
/** A {@link MergePolicy} that reorders documents according to a {@link Sort}
* before merging them. As a consequence, all segments resulting from a merge
@ -96,8 +97,8 @@ public final class SortingMergePolicy extends MergePolicy {
super.setInfo(info);
}
private MonotonicAppendingLongBuffer getDeletes(List<AtomicReader> readers) {
MonotonicAppendingLongBuffer deletes = new MonotonicAppendingLongBuffer();
private PackedLongValues getDeletes(List<AtomicReader> readers) {
PackedLongValues.Builder deletes = PackedLongValues.monotonicBuilder(PackedInts.COMPACT);
int deleteCount = 0;
for (AtomicReader reader : readers) {
final int maxDoc = reader.maxDoc();
@ -110,8 +111,7 @@ public final class SortingMergePolicy extends MergePolicy {
}
}
}
deletes.freeze();
return deletes;
return deletes.build();
}
@Override
@ -123,7 +123,7 @@ public final class SortingMergePolicy extends MergePolicy {
return super.getDocMap(mergeState);
}
assert mergeState.docMaps.length == 1; // we returned a singleton reader
final MonotonicAppendingLongBuffer deletes = getDeletes(unsortedReaders);
final PackedLongValues deletes = getDeletes(unsortedReaders);
return new MergePolicy.DocMap() {
@Override
public int map(int old) {

View File

@ -30,7 +30,6 @@ import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentReader;
import org.apache.lucene.index.SortedDocValues;
@ -45,8 +44,8 @@ import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.PagedBytes;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.packed.GrowableWriter;
import org.apache.lucene.util.packed.MonotonicAppendingLongBuffer;
import org.apache.lucene.util.packed.PackedInts;
import org.apache.lucene.util.packed.PackedLongValues;
/**
* Expert: The default cache implementation, storing all values in memory.
@ -551,11 +550,11 @@ class FieldCacheImpl implements FieldCache {
public static class SortedDocValuesImpl implements Accountable {
private final PagedBytes.Reader bytes;
private final MonotonicAppendingLongBuffer termOrdToBytesOffset;
private final PackedLongValues termOrdToBytesOffset;
private final PackedInts.Reader docToTermOrd;
private final int numOrd;
public SortedDocValuesImpl(PagedBytes.Reader bytes, MonotonicAppendingLongBuffer termOrdToBytesOffset, PackedInts.Reader docToTermOrd, int numOrd) {
public SortedDocValuesImpl(PagedBytes.Reader bytes, PackedLongValues termOrdToBytesOffset, PackedInts.Reader docToTermOrd, int numOrd) {
this.bytes = bytes;
this.docToTermOrd = docToTermOrd;
this.termOrdToBytesOffset = termOrdToBytesOffset;
@ -674,7 +673,7 @@ class FieldCacheImpl implements FieldCache {
startTermsBPV = 1;
}
MonotonicAppendingLongBuffer termOrdToBytesOffset = new MonotonicAppendingLongBuffer();
PackedLongValues.Builder termOrdToBytesOffset = PackedLongValues.monotonicBuilder(PackedInts.COMPACT);
final GrowableWriter docToTermOrd = new GrowableWriter(startTermsBPV, maxDoc, acceptableOverheadRatio);
int termOrd = 0;
@ -707,10 +706,9 @@ class FieldCacheImpl implements FieldCache {
termOrd++;
}
}
termOrdToBytesOffset.freeze();
// maybe an int-only impl?
return new SortedDocValuesImpl(bytes.freeze(true), termOrdToBytesOffset, docToTermOrd.getMutable(), termOrd);
return new SortedDocValuesImpl(bytes.freeze(true), termOrdToBytesOffset.build(), docToTermOrd.getMutable(), termOrd);
}
}