LUCENE-5773: Test SegmentReader.ramBytesUsed.

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1603430 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Adrien Grand 2014-06-18 12:24:14 +00:00
parent 2bac242329
commit b517365d54
33 changed files with 467 additions and 148 deletions

View File

@ -59,8 +59,7 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
private final static int PAGED_BYTES_BITS = 15;
// all fields share this single logical byte[]
private final PagedBytes termBytes = new PagedBytes(PAGED_BYTES_BITS);
private PagedBytes.Reader termBytesReader;
private final PagedBytes.Reader termBytesReader;
final HashMap<FieldInfo,FieldIndexData> fields = new HashMap<>();
@ -71,6 +70,7 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
public FixedGapTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, Comparator<BytesRef> termComp, String segmentSuffix, IOContext context)
throws IOException {
final PagedBytes termBytes = new PagedBytes(PAGED_BYTES_BITS);
this.termComp = termComp;
@ -115,7 +115,7 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
throw new CorruptIndexException("invalid packedIndexStart: " + packedIndexStart + " indexStart: " + indexStart + "numIndexTerms: " + numIndexTerms + " (resource=" + in + ")");
}
final FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
FieldIndexData previous = fields.put(fieldInfo, new FieldIndexData(in, indexStart, termsStart, packedIndexStart, packedOffsetsStart, numIndexTerms));
FieldIndexData previous = fields.put(fieldInfo, new FieldIndexData(in, termBytes, indexStart, termsStart, packedIndexStart, packedOffsetsStart, numIndexTerms));
if (previous != null) {
throw new CorruptIndexException("duplicate field: " + fieldInfo.name + " (resource=" + in + ")");
}
@ -238,7 +238,7 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
final long numIndexTerms;
final long termsStart;
public FieldIndexData(IndexInput in, long indexStart, long termsStart, long packedIndexStart, long packedOffsetsStart, long numIndexTerms) throws IOException {
public FieldIndexData(IndexInput in, PagedBytes termBytes, long indexStart, long termsStart, long packedIndexStart, long packedOffsetsStart, long numIndexTerms) throws IOException {
this.termsStart = termsStart;
termBytesStart = termBytes.getPointer();
@ -292,9 +292,7 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
@Override
public long ramBytesUsed() {
long sizeInBytes = ((termBytes!=null) ? termBytes.ramBytesUsed() : 0) +
((termBytesReader!=null)? termBytesReader.ramBytesUsed() : 0);
long sizeInBytes = ((termBytesReader!=null)? termBytesReader.ramBytesUsed() : 0);
for(FieldIndexData entry : fields.values()) {
sizeInBytes += entry.ramBytesUsed();
}

View File

@ -151,7 +151,7 @@ public final class BloomFilteringPostingsFormat extends PostingsFormat {
return new BloomFilteredFieldsProducer(state);
}
public class BloomFilteredFieldsProducer extends FieldsProducer {
static class BloomFilteredFieldsProducer extends FieldsProducer {
private FieldsProducer delegateFieldsProducer;
HashMap<String,FuzzySet> bloomsByFieldName = new HashMap<>();

View File

@ -169,11 +169,16 @@ public final class DirectPostingsFormat extends PostingsFormat {
private final static class DirectField extends Terms implements Accountable {
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(DirectField.class);
private static abstract class TermAndSkip implements Accountable {
public int[] skips;
}
private static final class LowFreqTerm extends TermAndSkip {
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(HighFreqTerm.class);
public final int[] postings;
public final byte[] payloads;
public final int docFreq;
@ -188,13 +193,17 @@ public final class DirectPostingsFormat extends PostingsFormat {
@Override
public long ramBytesUsed() {
return ((postings!=null) ? RamUsageEstimator.sizeOf(postings) : 0) +
return BASE_RAM_BYTES_USED +
((postings!=null) ? RamUsageEstimator.sizeOf(postings) : 0) +
((payloads!=null) ? RamUsageEstimator.sizeOf(payloads) : 0);
}
}
// TODO: maybe specialize into prx/no-prx/no-frq cases?
private static final class HighFreqTerm extends TermAndSkip {
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(HighFreqTerm.class);
public final long totalTermFreq;
public final int[] docIDs;
public final int[] freqs;
@ -211,19 +220,22 @@ public final class DirectPostingsFormat extends PostingsFormat {
@Override
public long ramBytesUsed() {
long sizeInBytes = 0;
long sizeInBytes = BASE_RAM_BYTES_USED;
sizeInBytes += (docIDs!=null)? RamUsageEstimator.sizeOf(docIDs) : 0;
sizeInBytes += (freqs!=null)? RamUsageEstimator.sizeOf(freqs) : 0;
if(positions != null) {
sizeInBytes += RamUsageEstimator.shallowSizeOf(positions);
for(int[] position : positions) {
sizeInBytes += (position!=null) ? RamUsageEstimator.sizeOf(position) : 0;
}
}
if (payloads != null) {
sizeInBytes += RamUsageEstimator.shallowSizeOf(payloads);
for(byte[][] payload : payloads) {
if(payload != null) {
sizeInBytes += RamUsageEstimator.shallowSizeOf(payload);
for(byte[] pload : payload) {
sizeInBytes += (pload!=null) ? RamUsageEstimator.sizeOf(pload) : 0;
}
@ -494,7 +506,7 @@ public final class DirectPostingsFormat extends PostingsFormat {
@Override
public long ramBytesUsed() {
long sizeInBytes = 0;
long sizeInBytes = BASE_RAM_BYTES_USED;
sizeInBytes += ((termBytes!=null) ? RamUsageEstimator.sizeOf(termBytes) : 0);
sizeInBytes += ((termOffsets!=null) ? RamUsageEstimator.sizeOf(termOffsets) : 0);
sizeInBytes += ((skips!=null) ? RamUsageEstimator.sizeOf(skips) : 0);
@ -502,6 +514,7 @@ public final class DirectPostingsFormat extends PostingsFormat {
sizeInBytes += ((sameCounts!=null) ? RamUsageEstimator.sizeOf(sameCounts) : 0);
if(terms!=null) {
sizeInBytes += RamUsageEstimator.shallowSizeOf(terms);
for(TermAndSkip termAndSkip : terms) {
sizeInBytes += (termAndSkip!=null) ? termAndSkip.ramBytesUsed() : 0;
}

View File

@ -827,7 +827,7 @@ public class FSTOrdTermsReader extends FieldsProducer {
@Override
public long ramBytesUsed() {
long ramBytesUsed = 0;
long ramBytesUsed = postingsReader.ramBytesUsed();
for (TermsReader r : fields.values()) {
if (r.index != null) {
ramBytesUsed += r.index.ramBytesUsed();

View File

@ -24,6 +24,8 @@ import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.fst.Outputs;
/**
@ -46,7 +48,8 @@ class FSTTermOutputs extends Outputs<FSTTermOutputs.TermData> {
* On an FST, only long[] part is 'shared' and pushed towards root.
* byte[] and term stats will be kept on deeper arcs.
*/
static class TermData {
static class TermData implements Accountable {
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(TermData.class);
long[] longs;
byte[] bytes;
int docFreq;
@ -64,6 +67,18 @@ class FSTTermOutputs extends Outputs<FSTTermOutputs.TermData> {
this.totalTermFreq = totalTermFreq;
}
@Override
public long ramBytesUsed() {
long ramBytesUsed = BASE_RAM_BYTES_USED;
if (longs != null) {
ramBytesUsed += RamUsageEstimator.sizeOf(longs);
}
if (bytes != null) {
ramBytesUsed += RamUsageEstimator.sizeOf(bytes);
}
return ramBytesUsed;
}
// NOTE: actually, FST nodes are seldom
// identical when outputs on their arcs
// aren't NO_OUTPUTs.
@ -112,6 +127,11 @@ class FSTTermOutputs extends Outputs<FSTTermOutputs.TermData> {
this.longsSize = longsSize;
}
@Override
public long ramBytesUsed(TermData output) {
return output.ramBytesUsed();
}
@Override
//
// The return value will be the smaller one, when these two are

View File

@ -41,6 +41,7 @@ import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.automaton.ByteRunAutomaton;
import org.apache.lucene.util.automaton.CompiledAutomaton;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
@ -165,7 +166,9 @@ public class FSTTermsReader extends FieldsProducer {
}
}
final class TermsReader extends Terms {
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(TermsReader.class);
final class TermsReader extends Terms implements Accountable {
final FieldInfo fieldInfo;
final long numTerms;
final long sumTotalTermFreq;
@ -184,6 +187,11 @@ public class FSTTermsReader extends FieldsProducer {
this.dict = new FST<>(in, new FSTTermOutputs(fieldInfo, longsSize));
}
@Override
public long ramBytesUsed() {
return BASE_RAM_BYTES_USED + dict.ramBytesUsed();
}
@Override
public boolean hasFreqs() {
return fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0;
@ -732,9 +740,9 @@ public class FSTTermsReader extends FieldsProducer {
@Override
public long ramBytesUsed() {
long ramBytesUsed = 0;
long ramBytesUsed = postingsReader.ramBytesUsed();
for (TermsReader r : fields.values()) {
ramBytesUsed += r.dict == null ? 0 : r.dict.ramBytesUsed();
ramBytesUsed += r.ramBytesUsed();
}
return ramBytesUsed;
}

View File

@ -55,11 +55,17 @@ import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.StringHelper;
class SimpleTextDocValuesReader extends DocValuesProducer {
private static final long BASE_RAM_BYTES_USED =
RamUsageEstimator.shallowSizeOfInstance(SimpleTextDocValuesReader.class)
+ RamUsageEstimator.shallowSizeOfInstance(BytesRef.class);
static class OneField {
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(OneField.class);
long dataStartFilePointer;
String pattern;
String ordPattern;
@ -500,7 +506,8 @@ class SimpleTextDocValuesReader extends DocValuesProducer {
@Override
public long ramBytesUsed() {
return 0;
return BASE_RAM_BYTES_USED + RamUsageEstimator.sizeOf(scratch.bytes)
+ fields.size() * (RamUsageEstimator.NUM_BYTES_OBJECT_REF * 2L + OneField.BASE_RAM_BYTES_USED);
}
@Override

View File

@ -55,6 +55,7 @@ import org.apache.lucene.util.CharsRef;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.StringHelper;
import org.apache.lucene.util.UnicodeUtil;
import org.apache.lucene.util.fst.Builder;
@ -65,6 +66,11 @@ import org.apache.lucene.util.fst.PositiveIntOutputs;
import org.apache.lucene.util.fst.Util;
class SimpleTextFieldsReader extends FieldsProducer {
private static final long BASE_RAM_BYTES_USED =
RamUsageEstimator.shallowSizeOfInstance(SimpleTextFieldsReader.class)
+ RamUsageEstimator.shallowSizeOfInstance(TreeMap.class);
private final TreeMap<String,Long> fields;
private final IndexInput in;
private final FieldInfos fieldInfos;
@ -497,6 +503,10 @@ class SimpleTextFieldsReader extends FieldsProducer {
}
}
private static final long TERMS_BASE_RAM_BYTES_USED =
RamUsageEstimator.shallowSizeOfInstance(SimpleTextTerms.class)
+ RamUsageEstimator.shallowSizeOfInstance(BytesRef.class)
+ RamUsageEstimator.shallowSizeOfInstance(CharsRef.class);
private class SimpleTextTerms extends Terms implements Accountable {
private final long termsStart;
private final FieldInfo fieldInfo;
@ -581,7 +591,8 @@ class SimpleTextFieldsReader extends FieldsProducer {
@Override
public long ramBytesUsed() {
return (fst!=null) ? fst.ramBytesUsed() : 0;
return TERMS_BASE_RAM_BYTES_USED + (fst!=null ? fst.ramBytesUsed() : 0)
+ RamUsageEstimator.sizeOf(scratch.bytes) + RamUsageEstimator.sizeOf(scratchUTF16.chars);
}
@Override
@ -643,14 +654,14 @@ class SimpleTextFieldsReader extends FieldsProducer {
@Override
synchronized public Terms terms(String field) throws IOException {
Terms terms = termsCache.get(field);
SimpleTextTerms terms = termsCache.get(field);
if (terms == null) {
Long fp = fields.get(field);
if (fp == null) {
return null;
} else {
terms = new SimpleTextTerms(field, fp, maxDoc);
termsCache.put(field, (SimpleTextTerms) terms);
termsCache.put(field, terms);
}
}
return terms;
@ -667,8 +678,8 @@ class SimpleTextFieldsReader extends FieldsProducer {
}
@Override
public long ramBytesUsed() {
long sizeInBytes = 0;
public synchronized long ramBytesUsed() {
long sizeInBytes = BASE_RAM_BYTES_USED + fields.size() * 2 * RamUsageEstimator.NUM_BYTES_OBJECT_REF;
for(SimpleTextTerms simpleTextTerms : termsCache.values()) {
sizeInBytes += (simpleTextTerms!=null) ? simpleTextTerms.ramBytesUsed() : 0;
}

View File

@ -36,6 +36,7 @@ import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CharsRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.StringHelper;
import org.apache.lucene.util.UnicodeUtil;
@ -48,6 +49,12 @@ import static org.apache.lucene.codecs.simpletext.SimpleTextStoredFieldsWriter.*
* @lucene.experimental
*/
public class SimpleTextStoredFieldsReader extends StoredFieldsReader {
private static final long BASE_RAM_BYTES_USED =
RamUsageEstimator.shallowSizeOfInstance(SimpleTextStoredFieldsReader.class)
+ RamUsageEstimator.shallowSizeOfInstance(BytesRef.class)
+ RamUsageEstimator.shallowSizeOfInstance(CharsRef.class);
private long offsets[]; /* docid -> offset in .fld file */
private IndexInput in;
private BytesRef scratch = new BytesRef();
@ -199,7 +206,8 @@ public class SimpleTextStoredFieldsReader extends StoredFieldsReader {
@Override
public long ramBytesUsed() {
return 0;
return BASE_RAM_BYTES_USED + RamUsageEstimator.sizeOf(offsets)
+ RamUsageEstimator.sizeOf(scratch.bytes) + RamUsageEstimator.sizeOf(scratchUTF16.chars);
}
@Override

View File

@ -43,8 +43,10 @@ import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CharsRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.StringHelper;
import org.apache.lucene.util.UnicodeUtil;
import static org.apache.lucene.codecs.simpletext.SimpleTextTermVectorsWriter.*;
/**
@ -54,6 +56,12 @@ import static org.apache.lucene.codecs.simpletext.SimpleTextTermVectorsWriter.*;
* @lucene.experimental
*/
public class SimpleTextTermVectorsReader extends TermVectorsReader {
private static final long BASE_RAM_BYTES_USED =
RamUsageEstimator.shallowSizeOfInstance(SimpleTextTermVectorsReader.class)
+ RamUsageEstimator.shallowSizeOfInstance(BytesRef.class)
+ RamUsageEstimator.shallowSizeOfInstance(CharsRef.class);
private long offsets[]; /* docid -> offset in .vec file */
private IndexInput in;
private BytesRef scratch = new BytesRef();
@ -539,7 +547,7 @@ public class SimpleTextTermVectorsReader extends TermVectorsReader {
@Override
public long ramBytesUsed() {
return 0;
return BASE_RAM_BYTES_USED + RamUsageEstimator.sizeOf(offsets);
}
@Override

View File

@ -21,7 +21,6 @@ import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.lucene41ords.Lucene41WithOrds;
import org.apache.lucene.index.BasePostingsFormatTestCase;
import org.apache.lucene.util.TestUtil;
import org.apache.lucene.util.TestUtil;
/**
* Basic tests of a PF using FixedGap terms dictionary

View File

@ -20,7 +20,6 @@ package org.apache.lucene.codecs.memory;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.index.BasePostingsFormatTestCase;
import org.apache.lucene.util.TestUtil;
import org.apache.lucene.util.TestUtil;
/**
* Tests DirectPostingsFormat

View File

@ -20,7 +20,6 @@ package org.apache.lucene.codecs.memory;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.index.BasePostingsFormatTestCase;
import org.apache.lucene.util.TestUtil;
import org.apache.lucene.util.TestUtil;
/**
* Tests MemoryPostingsFormat

View File

@ -27,6 +27,7 @@ import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.automaton.CompiledAutomaton;
import org.apache.lucene.util.fst.ByteSequenceOutputs;
import org.apache.lucene.util.fst.FST;
@ -34,6 +35,11 @@ import org.apache.lucene.util.fst.FST;
/** BlockTree's implementation of {@link Terms}. */
// public for CheckIndex:
public final class FieldReader extends Terms implements Accountable {
private static final long BASE_RAM_BYTES_USED =
RamUsageEstimator.shallowSizeOfInstance(FieldReader.class)
+ 3 * RamUsageEstimator.shallowSizeOfInstance(BytesRef.class);
final long numTerms;
final FieldInfo fieldInfo;
final long sumTotalTermFreq;
@ -172,6 +178,6 @@ public final class FieldReader extends Terms implements Accountable {
@Override
public long ramBytesUsed() {
return ((index!=null)? index.ramBytesUsed() : 0);
return BASE_RAM_BYTES_USED + ((index!=null)? index.ramBytesUsed() : 0);
}
}

View File

@ -36,6 +36,8 @@ import org.apache.lucene.util.packed.PackedInts;
*/
public final class CompressingStoredFieldsIndexReader implements Cloneable, Accountable {
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(CompressingStoredFieldsIndexReader.class);
final int maxDoc;
final int[] docBases;
final long[] startPointers;
@ -163,12 +165,14 @@ public final class CompressingStoredFieldsIndexReader implements Cloneable, Acco
@Override
public long ramBytesUsed() {
long res = 0;
for(PackedInts.Reader r : docBasesDeltas) {
long res = BASE_RAM_BYTES_USED;
res += RamUsageEstimator.shallowSizeOf(docBasesDeltas);
for (PackedInts.Reader r : docBasesDeltas) {
res += r.ramBytesUsed();
}
for(PackedInts.Reader r : startPointersDeltas) {
res += RamUsageEstimator.shallowSizeOf(startPointersDeltas);
for (PackedInts.Reader r : startPointersDeltas) {
res += r.ramBytesUsed();
}

View File

@ -328,7 +328,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
final PagedBytes.Reader bytesReader = bytes.freeze(true);
CodecUtil.checkEOF(input);
success = true;
ramBytesUsed.addAndGet(bytes.ramBytesUsed());
ramBytesUsed.addAndGet(bytesReader.ramBytesUsed());
return new BinaryDocValues() {
@Override
@ -370,7 +370,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
CodecUtil.checkEOF(data);
CodecUtil.checkEOF(index);
success = true;
ramBytesUsed.addAndGet(bytes.ramBytesUsed() + reader.ramBytesUsed());
ramBytesUsed.addAndGet(bytesReader.ramBytesUsed() + reader.ramBytesUsed());
return new BinaryDocValues() {
@Override
public BytesRef get(int docID) {
@ -414,7 +414,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
final PackedInts.Reader reader = PackedInts.getReader(index);
CodecUtil.checkEOF(data);
CodecUtil.checkEOF(index);
ramBytesUsed.addAndGet(bytes.ramBytesUsed() + reader.ramBytesUsed());
ramBytesUsed.addAndGet(bytesReader.ramBytesUsed() + reader.ramBytesUsed());
success = true;
return new BinaryDocValues() {
@Override
@ -457,7 +457,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
final PackedInts.Reader reader = PackedInts.getReader(index);
CodecUtil.checkEOF(data);
CodecUtil.checkEOF(index);
ramBytesUsed.addAndGet(bytes.ramBytesUsed() + reader.ramBytesUsed());
ramBytesUsed.addAndGet(bytesReader.ramBytesUsed() + reader.ramBytesUsed());
success = true;
return new BinaryDocValues() {
@ -540,7 +540,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
bytes.copy(data, fixedLength * (long) valueCount);
final PagedBytes.Reader bytesReader = bytes.freeze(true);
final PackedInts.Reader reader = PackedInts.getReader(index);
ramBytesUsed.addAndGet(bytes.ramBytesUsed() + reader.ramBytesUsed());
ramBytesUsed.addAndGet(bytesReader.ramBytesUsed() + reader.ramBytesUsed());
return correctBuggyOrds(new SortedDocValues() {
@Override
@ -578,7 +578,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
final PackedInts.Reader ordsReader = PackedInts.getReader(index);
final int valueCount = addressReader.size() - 1;
ramBytesUsed.addAndGet(bytes.ramBytesUsed() + addressReader.ramBytesUsed() + ordsReader.ramBytesUsed());
ramBytesUsed.addAndGet(bytesReader.ramBytesUsed() + addressReader.ramBytesUsed() + ordsReader.ramBytesUsed());
return correctBuggyOrds(new SortedDocValues() {
@Override

View File

@ -32,6 +32,7 @@ import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.RamUsageEstimator;
import java.io.Closeable;
import java.nio.charset.StandardCharsets;
@ -47,6 +48,9 @@ import static org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsWriter.*;
* @lucene.internal
*/
public final class Lucene40StoredFieldsReader extends StoredFieldsReader implements Cloneable, Closeable {
private static final long RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(Lucene40StoredFieldsReader.class);
private final FieldInfos fieldInfos;
private final IndexInput fieldsStream;
private final IndexInput indexStream;
@ -248,7 +252,7 @@ public final class Lucene40StoredFieldsReader extends StoredFieldsReader impleme
@Override
public long ramBytesUsed() {
return 0;
return RAM_BYTES_USED;
}
@Override

View File

@ -43,6 +43,7 @@ import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.RamUsageEstimator;
/**
* Concrete class that reads docId(maybe frq,pos,offset,payloads) list
@ -53,6 +54,8 @@ import org.apache.lucene.util.IOUtils;
*/
public final class Lucene41PostingsReader extends PostingsReaderBase {
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(Lucene41PostingsReader.class);
private final IndexInput docIn;
private final IndexInput posIn;
private final IndexInput payIn;
@ -1543,7 +1546,7 @@ public final class Lucene41PostingsReader extends PostingsReaderBase {
@Override
public long ramBytesUsed() {
return 0;
return BASE_RAM_BYTES_USED;
}
@Override

View File

@ -291,7 +291,7 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
final PagedBytes.Reader bytesReader = bytes.freeze(true);
if (entry.minLength == entry.maxLength) {
final int fixedLength = entry.minLength;
ramBytesUsed.addAndGet(bytes.ramBytesUsed());
ramBytesUsed.addAndGet(bytesReader.ramBytesUsed());
return new BinaryDocValues() {
@Override
public BytesRef get(int docID) {
@ -302,7 +302,7 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
};
} else {
final MonotonicBlockPackedReader addresses = MonotonicBlockPackedReader.of(data, entry.packedIntsVersion, entry.blockSize, maxDoc, false);
ramBytesUsed.addAndGet(bytes.ramBytesUsed() + addresses.ramBytesUsed());
ramBytesUsed.addAndGet(bytesReader.ramBytesUsed() + addresses.ramBytesUsed());
return new BinaryDocValues() {
@Override

View File

@ -199,7 +199,9 @@ public abstract class PerFieldPostingsFormat extends PostingsFormat {
}
}
private class FieldsReader extends FieldsProducer {
private static class FieldsReader extends FieldsProducer {
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(FieldsReader.class);
private final Map<String,FieldsProducer> fields = new TreeMap<>();
private final Map<String,FieldsProducer> formats = new HashMap<>();
@ -258,12 +260,13 @@ public abstract class PerFieldPostingsFormat extends PostingsFormat {
@Override
public long ramBytesUsed() {
long sizeInBytes = 0;
long ramBytesUsed = BASE_RAM_BYTES_USED;
ramBytesUsed += fields.size() * 2L * RamUsageEstimator.NUM_BYTES_OBJECT_REF;
ramBytesUsed += formats.size() * 2L * RamUsageEstimator.NUM_BYTES_OBJECT_REF;
for(Map.Entry<String,FieldsProducer> entry: formats.entrySet()) {
sizeInBytes += entry.getKey().length() * RamUsageEstimator.NUM_BYTES_CHAR;
sizeInBytes += entry.getValue().ramBytesUsed();
ramBytesUsed += entry.getValue().ramBytesUsed();
}
return sizeInBytes;
return ramBytesUsed;
}
@Override

View File

@ -39,11 +39,14 @@ import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.CloseableThreadLocal;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.RamUsageEstimator;
/** Holds core readers that are shared (unchanged) when
* SegmentReader is cloned or reopened */
final class SegmentCoreReaders implements Accountable {
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(SegmentCoreReaders.class);
// Counts how many other readers share the core objects
// (freqStream, proxStream, tis, etc.) of this reader;
// when coreRef drops to 0, these core objects may be
@ -213,7 +216,8 @@ final class SegmentCoreReaders implements Accountable {
@Override
public long ramBytesUsed() {
return ((normsProducer!=null) ? normsProducer.ramBytesUsed() : 0) +
return BASE_RAM_BYTES_USED +
((normsProducer!=null) ? normsProducer.ramBytesUsed() : 0) +
((fields!=null) ? fields.ramBytesUsed() : 0) +
((fieldsReaderOrig!=null)? fieldsReaderOrig.ramBytesUsed() : 0) +
((termVectorsReaderOrig!=null) ? termVectorsReaderOrig.ramBytesUsed() : 0);

View File

@ -40,6 +40,7 @@ import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.CloseableThreadLocal;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.Version;
/**
@ -51,6 +52,11 @@ import org.apache.lucene.util.Version;
*/
public final class SegmentReader extends AtomicReader implements Accountable {
private static final long BASE_RAM_BYTES_USED =
RamUsageEstimator.shallowSizeOfInstance(SegmentReader.class)
+ RamUsageEstimator.shallowSizeOfInstance(SegmentDocValues.class);
private static final long LONG_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(Long.class);
private final SegmentCommitInfo si;
private final Bits liveDocs;
@ -594,7 +600,10 @@ public final class SegmentReader extends AtomicReader implements Accountable {
@Override
public long ramBytesUsed() {
ensureOpen();
long ramBytesUsed = 0;
long ramBytesUsed = BASE_RAM_BYTES_USED;
ramBytesUsed += dvGens.size() * LONG_RAM_BYTES_USED;
ramBytesUsed += dvProducers.size() * RamUsageEstimator.NUM_BYTES_OBJECT_REF;
ramBytesUsed += dvProducersByField.size() * 2 * RamUsageEstimator.NUM_BYTES_OBJECT_REF;
if (dvProducers != null) {
for (DocValuesProducer producer : dvProducers) {
ramBytesUsed += producer.ramBytesUsed();

View File

@ -18,8 +18,7 @@ package org.apache.lucene.util;
*/
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Arrays;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
@ -35,9 +34,10 @@ import org.apache.lucene.store.IndexInput;
// TODO: refactor this, byteblockpool, fst.bytestore, and any
// other "shift/mask big arrays". there are too many of these classes!
public final class PagedBytes implements Accountable {
private final List<byte[]> blocks = new ArrayList<>();
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(PagedBytes.class);
private byte[][] blocks = new byte[16][];
private int numBlocks;
// TODO: these are unused?
private final List<Integer> blockEnd = new ArrayList<>();
private final int blockSize;
private final int blockBits;
private final int blockMask;
@ -54,24 +54,19 @@ public final class PagedBytes implements Accountable {
*
* @see #freeze */
public final static class Reader implements Accountable {
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(Reader.class);
private final byte[][] blocks;
private final int[] blockEnds;
private final int blockBits;
private final int blockMask;
private final int blockSize;
private final long bytesUsedPerBlock;
private Reader(PagedBytes pagedBytes) {
blocks = new byte[pagedBytes.blocks.size()][];
for(int i=0;i<blocks.length;i++) {
blocks[i] = pagedBytes.blocks.get(i);
}
blockEnds = new int[blocks.length];
for(int i=0;i< blockEnds.length;i++) {
blockEnds[i] = pagedBytes.blockEnd.get(i);
}
blocks = Arrays.copyOf(pagedBytes.blocks, pagedBytes.numBlocks);
blockBits = pagedBytes.blockBits;
blockMask = pagedBytes.blockMask;
blockSize = pagedBytes.blockSize;
bytesUsedPerBlock = pagedBytes.bytesUsedPerBlock;
}
/**
@ -132,7 +127,12 @@ public final class PagedBytes implements Accountable {
@Override
public long ramBytesUsed() {
return ((blocks!=null) ? (blockSize * blocks.length) : 0);
long size = BASE_RAM_BYTES_USED + RamUsageEstimator.shallowSizeOf(blocks);
if (blocks.length > 0) {
size += (blocks.length - 1) * bytesUsedPerBlock;
size += RamUsageEstimator.sizeOf(blocks[blocks.length - 1]);
}
return size;
}
}
@ -144,7 +144,15 @@ public final class PagedBytes implements Accountable {
this.blockBits = blockBits;
blockMask = blockSize-1;
upto = blockSize;
bytesUsedPerBlock = blockSize + RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + RamUsageEstimator.NUM_BYTES_OBJECT_REF;
bytesUsedPerBlock = RamUsageEstimator.alignObjectSize(blockSize + RamUsageEstimator.NUM_BYTES_ARRAY_HEADER);
numBlocks = 0;
}
private void addBlock(byte[] block) {
if (blocks.length == numBlocks) {
blocks = Arrays.copyOf(blocks, ArrayUtil.oversize(numBlocks, RamUsageEstimator.NUM_BYTES_OBJECT_REF));
}
blocks[numBlocks++] = block;
}
/** Read this many bytes from in */
@ -153,8 +161,7 @@ public final class PagedBytes implements Accountable {
int left = blockSize - upto;
if (left == 0) {
if (currentBlock != null) {
blocks.add(currentBlock);
blockEnd.add(upto);
addBlock(currentBlock);
}
currentBlock = new byte[blockSize];
upto = 0;
@ -179,8 +186,7 @@ public final class PagedBytes implements Accountable {
int left = blockSize - upto;
if (bytes.length > left || currentBlock==null) {
if (currentBlock != null) {
blocks.add(currentBlock);
blockEnd.add(upto);
addBlock(currentBlock);
didSkipBytes = true;
}
currentBlock = new byte[blockSize];
@ -214,8 +220,7 @@ public final class PagedBytes implements Accountable {
if (currentBlock == null) {
currentBlock = EMPTY_BYTES;
}
blocks.add(currentBlock);
blockEnd.add(upto);
addBlock(currentBlock);
frozen = true;
currentBlock = null;
return new PagedBytes.Reader(this);
@ -225,13 +230,18 @@ public final class PagedBytes implements Accountable {
if (currentBlock == null) {
return 0;
} else {
return (blocks.size() * ((long) blockSize)) + upto;
return (numBlocks * ((long) blockSize)) + upto;
}
}
@Override
public long ramBytesUsed() {
return (blocks.size() + (currentBlock != null ? 1 : 0)) * bytesUsedPerBlock;
long size = BASE_RAM_BYTES_USED + RamUsageEstimator.shallowSizeOf(blocks)
+ bytesUsedPerBlock * numBlocks;
if (currentBlock != null) {
size += RamUsageEstimator.sizeOf(currentBlock);
}
return size;
}
/** Copy bytes in, writing the length as a 1 or 2 byte
@ -247,8 +257,7 @@ public final class PagedBytes implements Accountable {
throw new IllegalArgumentException("block size " + blockSize + " is too small to store length " + bytes.length + " bytes");
}
if (currentBlock != null) {
blocks.add(currentBlock);
blockEnd.add(upto);
addBlock(currentBlock);
}
currentBlock = new byte[blockSize];
upto = 0;
@ -274,7 +283,7 @@ public final class PagedBytes implements Accountable {
private byte[] currentBlock;
PagedBytesDataInput() {
currentBlock = blocks.get(0);
currentBlock = blocks[0];
}
@Override
@ -293,7 +302,7 @@ public final class PagedBytes implements Accountable {
* {@link #getPosition}. */
public void setPosition(long pos) {
currentBlockIndex = (int) (pos >> blockBits);
currentBlock = blocks.get(currentBlockIndex);
currentBlock = blocks[currentBlockIndex];
currentBlockUpto = (int) (pos & blockMask);
}
@ -332,7 +341,7 @@ public final class PagedBytes implements Accountable {
private void nextBlock() {
currentBlockIndex++;
currentBlockUpto = 0;
currentBlock = blocks.get(currentBlockIndex);
currentBlock = blocks[currentBlockIndex];
}
}
@ -341,8 +350,7 @@ public final class PagedBytes implements Accountable {
public void writeByte(byte b) {
if (upto == blockSize) {
if (currentBlock != null) {
blocks.add(currentBlock);
blockEnd.add(upto);
addBlock(currentBlock);
}
currentBlock = new byte[blockSize];
upto = 0;
@ -359,8 +367,7 @@ public final class PagedBytes implements Accountable {
if (upto == blockSize) {
if (currentBlock != null) {
blocks.add(currentBlock);
blockEnd.add(upto);
addBlock(currentBlock);
}
currentBlock = new byte[blockSize];
upto = 0;
@ -372,8 +379,7 @@ public final class PagedBytes implements Accountable {
final int blockLeft = blockSize - upto;
if (blockLeft < left) {
System.arraycopy(b, offset, currentBlock, upto, blockLeft);
blocks.add(currentBlock);
blockEnd.add(blockSize);
addBlock(currentBlock);
currentBlock = new byte[blockSize];
upto = 0;
offset += blockLeft;

View File

@ -22,6 +22,7 @@ import java.io.IOException;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.RamUsageEstimator;
/**
* An FST {@link Outputs} implementation where each output
@ -145,4 +146,9 @@ public final class ByteSequenceOutputs extends Outputs<BytesRef> {
public String outputToString(BytesRef output) {
return output.toString();
}
@Override
public long ramBytesUsed(BytesRef output) {
return super.ramBytesUsed(output) + RamUsageEstimator.sizeOf(output.bytes);
}
}

View File

@ -23,11 +23,17 @@ import java.util.List;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.RamUsageEstimator;
// TODO: merge with PagedBytes, except PagedBytes doesn't
// let you read while writing which FST needs
class BytesStore extends DataOutput {
class BytesStore extends DataOutput implements Accountable {
private static final long BASE_RAM_BYTES_USED =
RamUsageEstimator.shallowSizeOfInstance(BytesStore.class)
+ RamUsageEstimator.shallowSizeOfInstance(ArrayList.class);
private final List<byte[]> blocks = new ArrayList<>();
@ -465,4 +471,14 @@ class BytesStore extends DataOutput {
}
};
}
@Override
public long ramBytesUsed() {
long size = BASE_RAM_BYTES_USED;
for (byte[] block : blocks) {
size += RamUsageEstimator.sizeOf(block);
}
return size;
}
}

View File

@ -41,6 +41,7 @@ import org.apache.lucene.util.Constants;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.PriorityQueue;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.fst.Builder.UnCompiledNode;
import org.apache.lucene.util.packed.GrowableWriter;
import org.apache.lucene.util.packed.PackedInts;
@ -69,6 +70,10 @@ import org.apache.lucene.util.packed.PackedInts;
* @lucene.experimental
*/
public final class FST<T> implements Accountable {
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(FST.class);
private static final long ARC_SHALLOW_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(Arc.class);
/** Specifies allowed range of each int input label for
* this FST. */
public static enum INPUT_TYPE {BYTE1, BYTE2, BYTE4};
@ -390,15 +395,38 @@ public final class FST<T> implements Accountable {
return inputType;
}
private long ramBytesUsed(Arc<T>[] arcs) {
long size = 0;
if (arcs != null) {
size += RamUsageEstimator.shallowSizeOf(arcs);
for (Arc<T> arc : arcs) {
if (arc != null) {
size += ARC_SHALLOW_RAM_BYTES_USED;
if (arc.output != null) {
size += outputs.ramBytesUsed(arc.output);
}
if (arc.nextFinalOutput != null) {
size += outputs.ramBytesUsed(arc.nextFinalOutput);
}
}
}
}
return size;
}
@Override
public long ramBytesUsed() {
long size = bytes.getPosition();
long size = BASE_RAM_BYTES_USED;
size += bytes.ramBytesUsed();
if (packed) {
size += nodeRefToAddress.ramBytesUsed();
} else if (nodeAddress != null) {
size += nodeAddress.ramBytesUsed();
size += inCounts.ramBytesUsed();
}
size += ramBytesUsed(cachedRootArcs);
size += ramBytesUsed(assertingCachedRootArcs);
size += RamUsageEstimator.sizeOf(bytesPerArc);
return size;
}

View File

@ -21,6 +21,8 @@ import java.io.IOException;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.RamUsageEstimator;
/**
* Represents the outputs for an FST, providing the basic
@ -95,4 +97,10 @@ public abstract class Outputs<T> {
public T merge(T first, T second) {
throw new UnsupportedOperationException();
}
/** Return memory usage for the provided output.
* @see Accountable */
public long ramBytesUsed(T output) {
return RamUsageEstimator.shallowSizeOf(output);
}
}

View File

@ -175,4 +175,16 @@ public class PairOutputs<A,B> extends Outputs<PairOutputs.Pair<A,B>> {
public String toString() {
return "PairOutputs<" + outputs1 + "," + outputs2 + ">";
}
@Override
public long ramBytesUsed(Pair<A,B> output) {
long ramBytesUsed = super.ramBytesUsed(output);
if (output.output1 != null) {
ramBytesUsed += outputs1.ramBytesUsed(output.output1);
}
if (output.output2 != null) {
ramBytesUsed += outputs2.ramBytesUsed(output.output2);
}
return ramBytesUsed;
}
}

View File

@ -19,6 +19,7 @@ package org.apache.lucene.index;
import java.io.IOException;
import java.lang.reflect.Field;
import java.util.HashMap;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.lucene49.Lucene49DocValuesFormat;
@ -37,18 +38,28 @@ import org.apache.lucene.util.TestUtil;
public class TestOrdinalMap extends LuceneTestCase {
private static final RamUsageTester.Filter ORDINAL_MAP_FILTER = new RamUsageTester.Filter() {
@Override
public boolean accept(Field field) {
if (field.getDeclaringClass().equals(OrdinalMap.class) && field.getName().equals("owner")) {
return false;
private static final Field ORDINAL_MAP_OWNER_FIELD;
static {
try {
ORDINAL_MAP_OWNER_FIELD = OrdinalMap.class.getDeclaredField("owner");
} catch (Exception e) {
throw new Error();
}
}
private static final RamUsageTester.Accumulator ORDINAL_MAP_ACCUMULATOR = new RamUsageTester.Accumulator() {
public long accumulateObject(Object o, long shallowSize, java.util.Map<Field,Object> fieldValues, java.util.Collection<Object> queue) {
if (o == LongValues.IDENTITY) {
return 0L;
}
return true;
if (o instanceof OrdinalMap) {
fieldValues = new HashMap<>(fieldValues);
fieldValues.remove(ORDINAL_MAP_OWNER_FIELD);
}
return super.accumulateObject(o, shallowSize, fieldValues, queue);
}
public boolean accept(Object o) {
return o != LongValues.IDENTITY;
}
};
public void testRamBytesUsed() throws IOException {
@ -77,12 +88,12 @@ public class TestOrdinalMap extends LuceneTestCase {
SortedDocValues sdv = ar.getSortedDocValues("sdv");
if (sdv instanceof MultiSortedDocValues) {
OrdinalMap map = ((MultiSortedDocValues) sdv).mapping;
assertEquals(RamUsageTester.sizeOf(map, ORDINAL_MAP_FILTER), map.ramBytesUsed());
assertEquals(RamUsageTester.sizeOf(map, ORDINAL_MAP_ACCUMULATOR), map.ramBytesUsed());
}
SortedSetDocValues ssdv = ar.getSortedSetDocValues("ssdv");
if (ssdv instanceof MultiSortedSetDocValues) {
OrdinalMap map = ((MultiSortedSetDocValues) ssdv).mapping;
assertEquals(RamUsageTester.sizeOf(map, ORDINAL_MAP_FILTER), map.ramBytesUsed());
assertEquals(RamUsageTester.sizeOf(map, ORDINAL_MAP_ACCUMULATOR), map.ramBytesUsed());
}
iw.close();
r.close();

View File

@ -27,6 +27,7 @@ import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.MockDirectoryWrapper;
import org.apache.lucene.util.PagedBytes.Reader;
import org.junit.Ignore;
public class TestPagedBytes extends LuceneTestCase {
@ -185,4 +186,17 @@ public class TestPagedBytes extends LuceneTestCase {
dir.close();
}
public void testRamBytesUsed() {
final int blockBits = TestUtil.nextInt(random(), 4, 22);
PagedBytes b = new PagedBytes(blockBits);
final int totalBytes = random().nextInt(10000);
for (long pointer = 0; pointer < totalBytes; ) {
BytesRef bytes = new BytesRef(TestUtil.randomSimpleString(random(), 10));
pointer = b.copyUsingLengthPrefix(bytes);
}
assertEquals(RamUsageTester.sizeOf(b), b.ramBytesUsed());
final PagedBytes.Reader reader = b.freeze(random().nextBoolean());
assertEquals(RamUsageTester.sizeOf(reader), reader.ramBytesUsed());
}
}

View File

@ -18,24 +18,106 @@ package org.apache.lucene.index;
*/
import java.io.IOException;
import java.lang.reflect.Field;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.IdentityHashMap;
import java.util.Map;
import java.util.Set;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.document.Document;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.CloseableThreadLocal;
import org.apache.lucene.util.InfoStream;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.RamUsageTester;
import org.apache.lucene.util.packed.PackedInts;
/**
* Common tests to all index formats.
*/
abstract class BaseIndexFileFormatTestCase extends LuceneTestCase {
// metadata or Directory-level objects
private static final Set<Class<?>> EXCLUDED_CLASSES = Collections.newSetFromMap(new IdentityHashMap<Class<?>,Boolean>());
// Notorious singletons
private static final Set<Object> EXCLUDED_OBJECTS = Collections.newSetFromMap(new IdentityHashMap<Object,Boolean>());
static {
// Directory objects, don't take into account eg. the NIO buffers
EXCLUDED_CLASSES.add(Directory.class);
EXCLUDED_CLASSES.add(IndexInput.class);
// used for thread management, not by the index
EXCLUDED_CLASSES.add(CloseableThreadLocal.class);
// don't follow references to the top-level reader
EXCLUDED_CLASSES.add(IndexReader.class);
EXCLUDED_CLASSES.add(IndexReaderContext.class);
// usually small but can bump memory usage for
// memory-efficient things like stored fields
EXCLUDED_CLASSES.add(FieldInfos.class);
EXCLUDED_CLASSES.add(SegmentInfo.class);
EXCLUDED_CLASSES.add(SegmentCommitInfo.class);
EXCLUDED_CLASSES.add(FieldInfo.class);
// singletons
EXCLUDED_CLASSES.add(Codec.class);
EXCLUDED_CLASSES.add(Enum.class);
for (PackedInts.Format format : PackedInts.Format.values()) {
for (int i = 1; i <= 64; ++i) {
if (format.isSupported(i)) {
EXCLUDED_OBJECTS.add(PackedInts.getDecoder(format, PackedInts.VERSION_CURRENT, i));
}
}
}
}
static class Accumulator extends RamUsageTester.Accumulator {
private final Object root;
Accumulator(Object root) {
this.root = root;
}
public long accumulateObject(Object o, long shallowSize, java.util.Map<Field, Object> fieldValues, java.util.Collection<Object> queue) {
if (EXCLUDED_OBJECTS.contains(o)) {
return 0L;
}
for (Class<?> clazz = o.getClass(); clazz != null; clazz = clazz.getSuperclass()) {
if (EXCLUDED_CLASSES.contains(clazz) && o != root) {
return 0;
}
}
// we have no way to estimate the size of these things in codecs although
// something like a Collections.newSetFromMap(new HashMap<>()) uses quite
// some memory... So for now the test ignores the overhead of such
// collections but can we do better?
if (o instanceof Collection) {
Collection<?> coll = (Collection<?>) o;
queue.addAll((Collection<?>) o);
return (long) coll.size() * RamUsageEstimator.NUM_BYTES_OBJECT_REF;
} else if (o instanceof Map) {
final Map<?, ?> map = (Map<?,?>) o;
queue.addAll(map.keySet());
queue.addAll(map.values());
return 2L * map.size() * RamUsageEstimator.NUM_BYTES_OBJECT_REF;
}
return super.accumulateObject(o, shallowSize, fieldValues, queue);
}
};
/** Returns the codec to run tests against */
protected abstract Codec getCodec();
@ -73,11 +155,11 @@ abstract class BaseIndexFileFormatTestCase extends LuceneTestCase {
* comparing indices that store the same content.
*/
protected Collection<String> excludedExtensionsFromByteCounts() {
return new HashSet<String>(Arrays.asList(new String[] {
return new HashSet<String>(Arrays.asList(new String[] {
// segment infos store various pieces of information that don't solely depend
// on the content of the index in the diagnostics (such as a timestamp) so we
// exclude this file from the bytes counts
"si",
"si",
// lock files are 0 bytes (one directory in the test could be RAMDir, the other FSDir)
"lock" }));
}
@ -118,4 +200,38 @@ abstract class BaseIndexFileFormatTestCase extends LuceneTestCase {
dir2.close();
}
/** Test the accuracy of the ramBytesUsed estimations. */
public void testRamBytesUsed() throws IOException {
Directory dir = newDirectory();
IndexWriterConfig cfg = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
IndexWriter w = new IndexWriter(dir, cfg);
// we need to index enough documents so that constant overhead doesn't dominate
final int numDocs = atLeast(10000);
for (int i = 0; i < numDocs; ++i) {
Document d = new Document();
addRandomFields(d);
w.addDocument(d);
}
w.forceMerge(1);
w.commit();
w.close();
IndexReader reader = DirectoryReader.open(dir);
for (AtomicReaderContext context : reader.leaves()) {
final AtomicReader r = context.reader();
// beware of lazy-loaded stuff
new SimpleMergedSegmentWarmer(InfoStream.NO_OUTPUT).warm(r);
final long actualBytes = RamUsageTester.sizeOf(r, new Accumulator(r));
final long expectedBytes = ((SegmentReader) r).ramBytesUsed();
final long absoluteError = actualBytes - expectedBytes;
final double relativeError = (double) absoluteError / actualBytes;
final String message = "Actual RAM usage " + actualBytes + ", but got " + expectedBytes + ", " + relativeError + "% error";
assertTrue(message, Math.abs(relativeError) < 0.20d || Math.abs(absoluteError) < 1000);
}
reader.close();
dir.close();
}
}

View File

@ -238,7 +238,8 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
@Override
protected void addRandomFields(Document doc) {
// TODO: improve
doc.add(new TextField("foobar", "boo", Field.Store.NO));
doc.add(new TextField("foobar", TestUtil.randomSimpleString(random()), Field.Store.NO));
}
@Override

View File

@ -20,40 +20,40 @@ package org.apache.lucene.util;
import java.lang.reflect.Array;
import java.lang.reflect.Field;
import java.lang.reflect.Modifier;
import java.util.AbstractList;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.IdentityHashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.NoSuchElementException;
/** Crawls object graph to collect RAM usage for testing */
public final class RamUsageTester {
/**
* A {@link Filter} that accepts all fields.
*/
private static final Filter DEFAULT_FILTER = new Filter() {
@Override
public boolean accept(Field field) {
return true;
/** An accumulator of object references. This class allows for customizing RAM usage estimation. */
public static class Accumulator {
/** Accumulate transitive references for the provided fields of the given
* object into <code>queue</code> and return the shallow size of this object. */
public long accumulateObject(Object o, long shallowSize, Map<Field, Object> fieldValues, Collection<Object> queue) {
for (Object value : fieldValues.values()) {
queue.add(value);
}
return shallowSize;
}
public boolean accept(Object o) {
return true;
/** Accumulate transitive references for the provided values of the given
* array into <code>queue</code> and return the shallow size of this array. */
public long accumulateArray(Object array, long shallowSize, List<Object> values, Collection<Object> queue) {
queue.addAll(values);
return shallowSize;
}
};
/** A filter that allows to decide on what to take into account when measuring RAM usage. */
public static interface Filter {
/** Whether the provided field should be taken into account when measuring RAM usage. */
boolean accept(Field field);
/** Whether the provided field value should be taken into account when measuring RAM usage. */
boolean accept(Object o);
}
/**
@ -65,13 +65,13 @@ public final class RamUsageTester {
* (it isn't side-effect free). After the method exits, this memory
* should be GCed.</p>
*/
public static long sizeOf(Object obj, Filter filter) {
return measureObjectSize(obj, filter);
public static long sizeOf(Object obj, Accumulator accumulator) {
return measureObjectSize(obj, accumulator);
}
/** Same as calling <code>sizeOf(obj, DEFAULT_FILTER)</code>. */
public static long sizeOf(Object obj) {
return sizeOf(obj, DEFAULT_FILTER);
return sizeOf(obj, new Accumulator());
}
/**
@ -89,7 +89,7 @@ public final class RamUsageTester {
* or complex graphs (a max. recursion depth on my machine was ~5000 objects linked in a chain
* so not too much).
*/
private static long measureObjectSize(Object root, Filter filter) {
private static long measureObjectSize(Object root, Accumulator accumulator) {
// Objects seen so far.
final IdentityHashSet<Object> seen = new IdentityHashSet<>();
// Class cache with reference Field and precalculated shallow size.
@ -114,25 +114,28 @@ public final class RamUsageTester {
* Consider an array, possibly of primitive types. Push any of its references to
* the processing stack and accumulate this array's shallow size.
*/
long size = RamUsageEstimator.NUM_BYTES_ARRAY_HEADER;
final long shallowSize = RamUsageEstimator.shallowSizeOf(ob);
final int len = Array.getLength(ob);
if (len > 0) {
Class<?> componentClazz = obClazz.getComponentType();
if (componentClazz.isPrimitive()) {
size += (long) len * RamUsageEstimator.shallowSizeOfInstance(componentClazz);
} else {
size += (long) RamUsageEstimator.NUM_BYTES_OBJECT_REF * len;
final List<Object> values;
Class<?> componentClazz = obClazz.getComponentType();
if (componentClazz.isPrimitive()) {
values = Collections.emptyList();
} else {
values = new AbstractList<Object>() {
// Push refs for traversal later.
for (int i = len; --i >= 0 ;) {
final Object o = Array.get(ob, i);
if (o != null && !seen.contains(o) && filter.accept(o)) {
stack.add(o);
@Override
public Object get(int index) {
return Array.get(ob, index);
}
@Override
public int size() {
return len;
}
}
};
}
}
totalSize += RamUsageEstimator.alignObjectSize(size);
totalSize += accumulator.accumulateArray(ob, shallowSize, values, stack);
} else {
/*
* Consider an object. Push any references it has to the processing stack
@ -144,17 +147,12 @@ public final class RamUsageTester {
classCache.put(obClazz, cachedInfo = createCacheEntry(obClazz));
}
Map<Field, Object> fieldValues = new HashMap<>();
for (Field f : cachedInfo.referenceFields) {
if (filter.accept(f)) {
// Fast path to eliminate redundancies.
final Object o = f.get(ob);
if (o != null && !seen.contains(o) && filter.accept(o)) {
stack.add(o);
}
}
fieldValues.put(f, f.get(ob));
}
totalSize += cachedInfo.alignedShallowInstanceSize;
totalSize += accumulator.accumulateObject(ob, cachedInfo.alignedShallowInstanceSize, fieldValues, stack);
} catch (IllegalAccessException e) {
// this should never happen as we enabled setAccessible().
throw new RuntimeException("Reflective field access failed?", e);