mirror of https://github.com/apache/lucene.git
LUCENE-5197: Added SegmentReader.ramBytesUsed
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1521267 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
61d684fe7f
commit
7ccb123261
|
@ -107,6 +107,9 @@ New features
|
|||
* LUCENE-5186: Added CachingWrapperFilter.getFilter in order to be able to get
|
||||
the wrapped filter. (Trejkaz via Adrien Grand)
|
||||
|
||||
* LUCENE-5197: Added SegmentReader.ramBytesUsed to return approximate heap RAM
|
||||
used by index datastructures. (Areek Zillur via Robert Muir)
|
||||
|
||||
Bug Fixes
|
||||
|
||||
* LUCENE-5116: IndexWriter.addIndexes(IndexReader...) should drop empty (or all
|
||||
|
|
|
@ -862,4 +862,11 @@ public class BlockTermsReader extends FieldsProducer {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
long sizeInBytes = (postingsReader!=null) ? postingsReader.ramBytesUsed() : 0;
|
||||
sizeInBytes += (indexReader!=null) ? indexReader.ramBytesUsed() : 0;
|
||||
return sizeInBytes;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -256,6 +256,12 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
|
|||
clone.close();
|
||||
}
|
||||
}
|
||||
|
||||
/** Returns approximate RAM bytes used */
|
||||
public long ramBytesUsed() {
|
||||
return ((termOffsets!=null)? termOffsets.ramBytesUsed() : 0) +
|
||||
((termsDictOffsets!=null)? termsDictOffsets.ramBytesUsed() : 0);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -271,4 +277,15 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
|
|||
dirOffset = input.readLong();
|
||||
input.seek(dirOffset);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
long sizeInBytes = ((termBytes!=null) ? termBytes.ramBytesUsed() : 0) +
|
||||
((termBytesReader!=null)? termBytesReader.ramBytesUsed() : 0);
|
||||
|
||||
for(FieldIndexData entry : fields.values()) {
|
||||
sizeInBytes += entry.ramBytesUsed();
|
||||
}
|
||||
return sizeInBytes;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -70,4 +70,7 @@ public abstract class TermsIndexReaderBase implements Closeable {
|
|||
/** Only implemented if {@link TermsIndexReaderBase#supportsOrd()} returns true. */
|
||||
public abstract long ord();
|
||||
}
|
||||
|
||||
/** Returns approximate RAM bytes used */
|
||||
public abstract long ramBytesUsed();
|
||||
}
|
||||
|
|
|
@ -169,6 +169,11 @@ public class VariableGapTermsIndexReader extends TermsIndexReaderBase {
|
|||
w.close();
|
||||
*/
|
||||
}
|
||||
|
||||
/** Returns approximate RAM bytes used */
|
||||
public long ramBytesUsed() {
|
||||
return fst == null ? 0 : fst.sizeInBytes();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -191,4 +196,13 @@ public class VariableGapTermsIndexReader extends TermsIndexReaderBase {
|
|||
}
|
||||
input.seek(dirOffset);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
long sizeInBytes = 0;
|
||||
for(FieldIndexData entry : fields.values()) {
|
||||
sizeInBytes += entry.ramBytesUsed();
|
||||
}
|
||||
return sizeInBytes;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -48,6 +48,7 @@ import org.apache.lucene.store.IndexOutput;
|
|||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.automaton.CompiledAutomaton;
|
||||
|
||||
/**
|
||||
|
@ -390,6 +391,16 @@ public final class BloomFilteringPostingsFormat extends PostingsFormat {
|
|||
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
long sizeInBytes = ((delegateFieldsProducer!=null) ? delegateFieldsProducer.ramBytesUsed() : 0);
|
||||
for(Map.Entry<String,FuzzySet> entry: bloomsByFieldName.entrySet()) {
|
||||
sizeInBytes += entry.getKey().length() * RamUsageEstimator.NUM_BYTES_CHAR;
|
||||
sizeInBytes += entry.getValue().ramBytesUsed();
|
||||
}
|
||||
return sizeInBytes;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -22,6 +22,7 @@ import org.apache.lucene.store.DataInput;
|
|||
import org.apache.lucene.store.DataOutput;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.FixedBitSet;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
|
@ -302,4 +303,8 @@ public class FuzzySet {
|
|||
int numBitsSet = filter.cardinality();
|
||||
return (float) numBitsSet / (float) bloomSize;
|
||||
}
|
||||
}
|
||||
|
||||
public long ramBytesUsed() {
|
||||
return RamUsageEstimator.sizeOf(filter.getBits());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -44,6 +44,7 @@ import org.apache.lucene.store.RAMOutputStream;
|
|||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.automaton.CompiledAutomaton;
|
||||
import org.apache.lucene.util.automaton.RunAutomaton;
|
||||
import org.apache.lucene.util.automaton.Transition;
|
||||
|
@ -147,12 +148,25 @@ public final class DirectPostingsFormat extends PostingsFormat {
|
|||
@Override
|
||||
public void close() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
long sizeInBytes = 0;
|
||||
for(Map.Entry<String,DirectField> entry: fields.entrySet()) {
|
||||
sizeInBytes += entry.getKey().length() * RamUsageEstimator.NUM_BYTES_CHAR;
|
||||
sizeInBytes += entry.getValue().ramBytesUsed();
|
||||
}
|
||||
return sizeInBytes;
|
||||
}
|
||||
}
|
||||
|
||||
private final static class DirectField extends Terms {
|
||||
|
||||
private static abstract class TermAndSkip {
|
||||
public int[] skips;
|
||||
|
||||
/** Returns the approximate number of RAM bytes used */
|
||||
public abstract long ramBytesUsed();
|
||||
}
|
||||
|
||||
private static final class LowFreqTerm extends TermAndSkip {
|
||||
|
@ -167,6 +181,12 @@ public final class DirectPostingsFormat extends PostingsFormat {
|
|||
this.docFreq = docFreq;
|
||||
this.totalTermFreq = totalTermFreq;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return ((postings!=null) ? RamUsageEstimator.sizeOf(postings) : 0) +
|
||||
((payloads!=null) ? RamUsageEstimator.sizeOf(payloads) : 0);
|
||||
}
|
||||
}
|
||||
|
||||
// TODO: maybe specialize into prx/no-prx/no-frq cases?
|
||||
|
@ -184,6 +204,31 @@ public final class DirectPostingsFormat extends PostingsFormat {
|
|||
this.payloads = payloads;
|
||||
this.totalTermFreq = totalTermFreq;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
long sizeInBytes = 0;
|
||||
sizeInBytes += (docIDs!=null)? RamUsageEstimator.sizeOf(docIDs) : 0;
|
||||
sizeInBytes += (freqs!=null)? RamUsageEstimator.sizeOf(freqs) : 0;
|
||||
|
||||
if(positions != null) {
|
||||
for(int[] position : positions) {
|
||||
sizeInBytes += (position!=null) ? RamUsageEstimator.sizeOf(position) : 0;
|
||||
}
|
||||
}
|
||||
|
||||
if (payloads != null) {
|
||||
for(byte[][] payload : payloads) {
|
||||
if(payload != null) {
|
||||
for(byte[] pload : payload) {
|
||||
sizeInBytes += (pload!=null) ? RamUsageEstimator.sizeOf(pload) : 0;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return sizeInBytes;
|
||||
}
|
||||
}
|
||||
|
||||
private final byte[] termBytes;
|
||||
|
@ -444,6 +489,24 @@ public final class DirectPostingsFormat extends PostingsFormat {
|
|||
assert skipOffset == skipCount;
|
||||
}
|
||||
|
||||
/** Returns approximate RAM bytes used */
|
||||
public long ramBytesUsed() {
|
||||
long sizeInBytes = 0;
|
||||
sizeInBytes += ((termBytes!=null) ? RamUsageEstimator.sizeOf(termBytes) : 0);
|
||||
sizeInBytes += ((termOffsets!=null) ? RamUsageEstimator.sizeOf(termOffsets) : 0);
|
||||
sizeInBytes += ((skips!=null) ? RamUsageEstimator.sizeOf(skips) : 0);
|
||||
sizeInBytes += ((skipOffsets!=null) ? RamUsageEstimator.sizeOf(skipOffsets) : 0);
|
||||
sizeInBytes += ((sameCounts!=null) ? RamUsageEstimator.sizeOf(sameCounts) : 0);
|
||||
|
||||
if(terms!=null) {
|
||||
for(TermAndSkip termAndSkip : terms) {
|
||||
sizeInBytes += (termAndSkip!=null) ? termAndSkip.ramBytesUsed() : 0;
|
||||
}
|
||||
}
|
||||
|
||||
return sizeInBytes;
|
||||
}
|
||||
|
||||
// Compares in unicode (UTF8) order:
|
||||
int compare(int ord, BytesRef other) {
|
||||
final byte[] otherBytes = other.bytes;
|
||||
|
|
|
@ -60,6 +60,7 @@ import org.apache.lucene.codecs.BlockTermState;
|
|||
import org.apache.lucene.codecs.FieldsProducer;
|
||||
import org.apache.lucene.codecs.PostingsReaderBase;
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.memory.FSTTermsReader.TermsReader;
|
||||
|
||||
/**
|
||||
* FST-based terms dictionary reader.
|
||||
|
@ -817,4 +818,19 @@ public class FSTOrdTermsReader extends FieldsProducer {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
long ramBytesUsed = 0;
|
||||
for (TermsReader r : fields.values()) {
|
||||
if (r.index != null) {
|
||||
ramBytesUsed += r.index.sizeInBytes();
|
||||
ramBytesUsed += RamUsageEstimator.sizeOf(r.metaBytesBlock);
|
||||
ramBytesUsed += RamUsageEstimator.sizeOf(r.metaLongsBlock);
|
||||
ramBytesUsed += RamUsageEstimator.sizeOf(r.skipInfo);
|
||||
ramBytesUsed += RamUsageEstimator.sizeOf(r.statsBlock);
|
||||
}
|
||||
}
|
||||
return ramBytesUsed;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -729,4 +729,13 @@ public class FSTTermsReader extends FieldsProducer {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
long ramBytesUsed = 0;
|
||||
for (TermsReader r : fields.values()) {
|
||||
ramBytesUsed += r.dict == null ? 0 : r.dict.sizeInBytes();
|
||||
}
|
||||
return ramBytesUsed;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -44,6 +44,7 @@ import org.apache.lucene.util.FixedBitSet;
|
|||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.IntsRef;
|
||||
import org.apache.lucene.util.PagedBytes;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.fst.BytesRefFSTEnum;
|
||||
import org.apache.lucene.util.fst.BytesRefFSTEnum.InputOutput;
|
||||
import org.apache.lucene.util.fst.FST;
|
||||
|
@ -202,6 +203,12 @@ class MemoryDocValuesProducer extends DocValuesProducer {
|
|||
return instance;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
// TODO: optimize me
|
||||
return RamUsageEstimator.sizeOf(this);
|
||||
}
|
||||
|
||||
private NumericDocValues loadNumeric(FieldInfo field) throws IOException {
|
||||
NumericEntry entry = numerics.get(field.number);
|
||||
data.seek(entry.offset + entry.missingBytes);
|
||||
|
|
|
@ -21,6 +21,7 @@ import java.io.IOException;
|
|||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.SortedMap;
|
||||
import java.util.TreeMap;
|
||||
|
||||
|
@ -49,6 +50,7 @@ import org.apache.lucene.util.ArrayUtil;
|
|||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IntsRef;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.fst.Builder;
|
||||
import org.apache.lucene.util.fst.ByteSequenceOutputs;
|
||||
import org.apache.lucene.util.fst.BytesRefFSTEnum;
|
||||
|
@ -843,6 +845,10 @@ public final class MemoryPostingsFormat extends PostingsFormat {
|
|||
public boolean hasPayloads() {
|
||||
return field.hasPayloads();
|
||||
}
|
||||
|
||||
public long ramBytesUsed() {
|
||||
return ((fst!=null) ? fst.sizeInBytes() : 0);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -889,6 +895,16 @@ public final class MemoryPostingsFormat extends PostingsFormat {
|
|||
termsReader.fst = null;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
long sizeInBytes = 0;
|
||||
for(Map.Entry<String,TermsReader> entry: fields.entrySet()) {
|
||||
sizeInBytes += (entry.getKey().length() * RamUsageEstimator.NUM_BYTES_CHAR);
|
||||
sizeInBytes += entry.getValue().ramBytesUsed();
|
||||
}
|
||||
return sizeInBytes;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
|
@ -648,4 +648,9 @@ public class PulsingPostingsReader extends PostingsReaderBase {
|
|||
// we don't want to copy any stuff over to another docsenum ever!
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return ((wrappedPostingsReader!=null) ? wrappedPostingsReader.ramBytesUsed(): 0);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -701,4 +701,9 @@ public class SepPostingsReader extends PostingsReaderBase {
|
|||
return docFreq;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -62,6 +62,7 @@ class SimpleTextDocValuesReader extends DocValuesProducer {
|
|||
boolean fixedLength;
|
||||
long minValue;
|
||||
long numValues;
|
||||
|
||||
};
|
||||
|
||||
final int maxDoc;
|
||||
|
@ -464,4 +465,9 @@ class SimpleTextDocValuesReader extends DocValuesProducer {
|
|||
private String stripPrefix(BytesRef prefix) throws IOException {
|
||||
return new String(scratch.bytes, scratch.offset + prefix.length, scratch.length - prefix.length, "UTF-8");
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -574,6 +574,11 @@ class SimpleTextFieldsReader extends FieldsProducer {
|
|||
*/
|
||||
//System.out.println("FST " + fst.sizeInBytes());
|
||||
}
|
||||
|
||||
/** Returns approximate RAM bytes used */
|
||||
public long ramBytesUsed() {
|
||||
return (fst!=null) ? fst.sizeInBytes() : 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermsEnum iterator(TermsEnum reuse) throws IOException {
|
||||
|
@ -630,7 +635,7 @@ class SimpleTextFieldsReader extends FieldsProducer {
|
|||
return Collections.unmodifiableSet(fields.keySet()).iterator();
|
||||
}
|
||||
|
||||
private final Map<String,Terms> termsCache = new HashMap<String,Terms>();
|
||||
private final Map<String,SimpleTextTerms> termsCache = new HashMap<String,SimpleTextTerms>();
|
||||
|
||||
@Override
|
||||
synchronized public Terms terms(String field) throws IOException {
|
||||
|
@ -641,7 +646,7 @@ class SimpleTextFieldsReader extends FieldsProducer {
|
|||
return null;
|
||||
} else {
|
||||
terms = new SimpleTextTerms(field, fp);
|
||||
termsCache.put(field, terms);
|
||||
termsCache.put(field, (SimpleTextTerms) terms);
|
||||
}
|
||||
}
|
||||
return terms;
|
||||
|
@ -656,4 +661,13 @@ class SimpleTextFieldsReader extends FieldsProducer {
|
|||
public void close() throws IOException {
|
||||
in.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
long sizeInBytes = 0;
|
||||
for(SimpleTextTerms simpleTextTerms : termsCache.values()) {
|
||||
sizeInBytes += (simpleTextTerms!=null) ? simpleTextTerms.ramBytesUsed() : 0;
|
||||
}
|
||||
return sizeInBytes;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -192,4 +192,9 @@ public class SimpleTextStoredFieldsReader extends StoredFieldsReader {
|
|||
return a.length == b.length - bOffset &&
|
||||
ArrayUtil.equals(a.bytes, a.offset, b.bytes, b.offset + bOffset, b.length - bOffset);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -538,4 +538,9 @@ public class SimpleTextTermVectorsReader extends TermVectorsReader {
|
|||
return 1;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -550,6 +550,11 @@ public class BlockTreeTermsReader extends FieldsProducer {
|
|||
return new IntersectEnum(compiled, startTerm);
|
||||
}
|
||||
|
||||
/** Returns approximate RAM bytes used */
|
||||
public long ramBytesUsed() {
|
||||
return ((index!=null)? index.sizeInBytes() : 0);
|
||||
}
|
||||
|
||||
// NOTE: cannot seek!
|
||||
private final class IntersectEnum extends TermsEnum {
|
||||
private final IndexInput in;
|
||||
|
@ -2974,4 +2979,13 @@ public class BlockTreeTermsReader extends FieldsProducer {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
long sizeInByes = ((postingsReader!=null) ? postingsReader.ramBytesUsed() : 0);
|
||||
for(FieldReader reader : fields.values()) {
|
||||
sizeInByes += reader.ramBytesUsed();
|
||||
}
|
||||
return sizeInByes;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -64,6 +64,9 @@ public abstract class DocValuesProducer implements Closeable {
|
|||
* used by a single thread. */
|
||||
public abstract Bits getDocsWithField(FieldInfo field) throws IOException;
|
||||
|
||||
/** Returns approximate RAM bytes used */
|
||||
public abstract long ramBytesUsed();
|
||||
|
||||
/**
|
||||
* A simple implementation of {@link DocValuesProducer#getDocsWithField} that
|
||||
* returns {@code true} if a document has an ordinal >= 0
|
||||
|
|
|
@ -36,4 +36,7 @@ public abstract class FieldsProducer extends Fields implements Closeable {
|
|||
|
||||
@Override
|
||||
public abstract void close() throws IOException;
|
||||
|
||||
/** Returns approximate RAM bytes used */
|
||||
public abstract long ramBytesUsed();
|
||||
}
|
||||
|
|
|
@ -69,6 +69,9 @@ public abstract class PostingsReaderBase implements Closeable {
|
|||
public abstract DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, BlockTermState state, Bits skipDocs, DocsAndPositionsEnum reuse,
|
||||
int flags) throws IOException;
|
||||
|
||||
/** Returns approximate RAM bytes used */
|
||||
public abstract long ramBytesUsed();
|
||||
|
||||
@Override
|
||||
public abstract void close() throws IOException;
|
||||
}
|
||||
|
|
|
@ -40,4 +40,7 @@ public abstract class StoredFieldsReader implements Cloneable, Closeable {
|
|||
|
||||
@Override
|
||||
public abstract StoredFieldsReader clone();
|
||||
|
||||
/** Returns approximate RAM bytes used */
|
||||
public abstract long ramBytesUsed();
|
||||
}
|
||||
|
|
|
@ -42,6 +42,9 @@ public abstract class TermVectorsReader implements Cloneable, Closeable {
|
|||
* available from the {@link DocsAndPositionsEnum}. */
|
||||
public abstract Fields get(int doc) throws IOException;
|
||||
|
||||
/** Returns approximate RAM bytes used */
|
||||
public abstract long ramBytesUsed();
|
||||
|
||||
/** Create a clone that one caller at a time may use to
|
||||
* read term vectors. */
|
||||
@Override
|
||||
|
|
|
@ -24,6 +24,7 @@ import org.apache.lucene.index.CorruptIndexException;
|
|||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
/**
|
||||
|
@ -160,5 +161,23 @@ public final class CompressingStoredFieldsIndexReader implements Cloneable {
|
|||
public CompressingStoredFieldsIndexReader clone() {
|
||||
return this;
|
||||
}
|
||||
|
||||
long ramBytesUsed() {
|
||||
long res = 0;
|
||||
|
||||
for(PackedInts.Reader r : docBasesDeltas) {
|
||||
res += r.ramBytesUsed();
|
||||
}
|
||||
for(PackedInts.Reader r : startPointersDeltas) {
|
||||
res += r.ramBytesUsed();
|
||||
}
|
||||
|
||||
res += RamUsageEstimator.sizeOf(docBases);
|
||||
res += RamUsageEstimator.sizeOf(startPointers);
|
||||
res += RamUsageEstimator.sizeOf(avgChunkDocs);
|
||||
res += RamUsageEstimator.sizeOf(avgChunkSizes);
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -497,4 +497,9 @@ public final class CompressingStoredFieldsReader extends StoredFieldsReader {
|
|||
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return indexReader.ramBytesUsed();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -57,6 +57,7 @@ import org.apache.lucene.util.Bits;
|
|||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.LongsRef;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.packed.BlockPackedReaderIterator;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
|
@ -1041,4 +1042,9 @@ public final class CompressingTermVectorsReader extends TermVectorsReader implem
|
|||
return sum;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return indexReader.ramBytesUsed();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -39,6 +39,7 @@ import org.apache.lucene.util.Bits;
|
|||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.PagedBytes;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
/**
|
||||
|
@ -630,4 +631,9 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
|
|||
public void close() throws IOException {
|
||||
dir.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return RamUsageEstimator.sizeOf(this);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1162,4 +1162,10 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
|
|||
return limit;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -244,4 +244,9 @@ public final class Lucene40StoredFieldsReader extends StoredFieldsReader impleme
|
|||
|
||||
return fieldsStream;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -763,5 +763,10 @@ public class Lucene40TermVectorsReader extends TermVectorsReader implements Clos
|
|||
|
||||
return new Lucene40TermVectorsReader(fieldInfos, cloneTvx, cloneTvd, cloneTvf, size, numTotalDocs);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -1541,4 +1541,10 @@ public final class Lucene41PostingsReader extends PostingsReaderBase {
|
|||
return docFreq;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -43,6 +43,7 @@ import org.apache.lucene.util.BytesRef;
|
|||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.IntsRef;
|
||||
import org.apache.lucene.util.PagedBytes;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.fst.BytesRefFSTEnum;
|
||||
import org.apache.lucene.util.fst.BytesRefFSTEnum.InputOutput;
|
||||
import org.apache.lucene.util.fst.FST;
|
||||
|
@ -188,6 +189,11 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
|
|||
return instance;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return RamUsageEstimator.sizeOf(this);
|
||||
}
|
||||
|
||||
private NumericDocValues loadNumeric(FieldInfo field) throws IOException {
|
||||
NumericEntry entry = numerics.get(field.number);
|
||||
data.seek(entry.offset);
|
||||
|
|
|
@ -20,7 +20,6 @@ package org.apache.lucene.codecs.lucene45;
|
|||
import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer.DELTA_COMPRESSED;
|
||||
import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer.GCD_COMPRESSED;
|
||||
import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer.TABLE_COMPRESSED;
|
||||
|
||||
import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer.BINARY_FIXED_UNCOMPRESSED;
|
||||
import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer.BINARY_VARIABLE_UNCOMPRESSED;
|
||||
import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer.BINARY_PREFIX_COMPRESSED;
|
||||
|
@ -244,6 +243,18 @@ public class Lucene45DocValuesProducer extends DocValuesProducer implements Clos
|
|||
return getNumeric(entry);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
long sizeInBytes = 0;
|
||||
for(MonotonicBlockPackedReader monotonicBlockPackedReader: addressInstances.values()) {
|
||||
sizeInBytes += Integer.SIZE + monotonicBlockPackedReader.ramBytesUsed();
|
||||
}
|
||||
for(MonotonicBlockPackedReader monotonicBlockPackedReader: ordIndexInstances.values()) {
|
||||
sizeInBytes += Integer.SIZE + monotonicBlockPackedReader.ramBytesUsed();
|
||||
}
|
||||
return sizeInBytes;
|
||||
}
|
||||
|
||||
LongNumericDocValues getNumeric(NumericEntry entry) throws IOException {
|
||||
final IndexInput data = this.data.clone();
|
||||
data.seek(entry.offset);
|
||||
|
|
|
@ -39,6 +39,7 @@ import org.apache.lucene.index.SortedSetDocValues;
|
|||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
/**
|
||||
* Enables per field docvalues support.
|
||||
|
@ -282,6 +283,16 @@ public abstract class PerFieldDocValuesFormat extends DocValuesFormat {
|
|||
public DocValuesProducer clone() {
|
||||
return new FieldsReader(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
long size = 0;
|
||||
for (Map.Entry<String,DocValuesProducer> entry : formats.entrySet()) {
|
||||
size += (entry.getKey().length() * RamUsageEstimator.NUM_BYTES_CHAR) +
|
||||
entry.getValue().ramBytesUsed();
|
||||
}
|
||||
return size;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -35,6 +35,7 @@ import org.apache.lucene.index.SegmentReadState;
|
|||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
/**
|
||||
* Enables per field postings support.
|
||||
|
@ -225,6 +226,16 @@ public abstract class PerFieldPostingsFormat extends PostingsFormat {
|
|||
public void close() throws IOException {
|
||||
IOUtils.close(formats.values());
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
long sizeInBytes = 0;
|
||||
for(Map.Entry<String,FieldsProducer> entry: formats.entrySet()) {
|
||||
sizeInBytes += entry.getKey().length() * RamUsageEstimator.NUM_BYTES_CHAR;
|
||||
sizeInBytes += entry.getValue().ramBytesUsed();
|
||||
}
|
||||
return sizeInBytes;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -354,6 +354,14 @@ final class SegmentCoreReaders {
|
|||
coreClosedListeners.remove(listener);
|
||||
}
|
||||
|
||||
/** Returns approximate RAM bytes used */
|
||||
public long ramBytesUsed() {
|
||||
return ((dvProducer!=null) ? dvProducer.ramBytesUsed() : 0) +
|
||||
((normsProducer!=null) ? normsProducer.ramBytesUsed() : 0) +
|
||||
((fields!=null) ? fields.ramBytesUsed() : 0) +
|
||||
((fieldsReaderOrig!=null)? fieldsReaderOrig.ramBytesUsed() : 0) +
|
||||
((termVectorsReaderOrig!=null) ? termVectorsReaderOrig.ramBytesUsed() : 0);
|
||||
}
|
||||
@Override
|
||||
public String toString() {
|
||||
return "SegmentCoreReader(owner=" + owner + ")";
|
||||
|
|
|
@ -283,4 +283,10 @@ public final class SegmentReader extends AtomicReader {
|
|||
ensureOpen();
|
||||
core.removeCoreClosedListener(listener);
|
||||
}
|
||||
|
||||
/** Returns approximate RAM Bytes used */
|
||||
public long ramBytesUsed() {
|
||||
ensureOpen();
|
||||
return (core!=null) ? core.ramBytesUsed() : 0;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -126,6 +126,11 @@ public final class PagedBytes {
|
|||
assert b.length > 0;
|
||||
}
|
||||
}
|
||||
|
||||
/** Returns approximate RAM bytes used */
|
||||
public long ramBytesUsed() {
|
||||
return ((blocks!=null) ? (blockSize * blocks.length) : 0);
|
||||
}
|
||||
}
|
||||
|
||||
/** 1<<blockBits must be bigger than biggest single
|
||||
|
|
|
@ -85,4 +85,12 @@ public final class BlockPackedReader {
|
|||
return (minValues == null ? 0 : minValues[block]) + subReaders[block].get(idx);
|
||||
}
|
||||
|
||||
/** Returns approximate RAM bytes used */
|
||||
public long ramBytesUsed() {
|
||||
long size = 0;
|
||||
for (PackedInts.Reader reader : subReaders) {
|
||||
size += reader.ramBytesUsed();
|
||||
}
|
||||
return size;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -26,6 +26,7 @@ import static org.apache.lucene.util.packed.PackedInts.numBlocks;
|
|||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
/**
|
||||
* Provides random access to a stream written with
|
||||
|
@ -83,5 +84,16 @@ public final class MonotonicBlockPackedReader {
|
|||
public long size() {
|
||||
return valueCount;
|
||||
}
|
||||
|
||||
/** Returns the approximate RAM bytes used */
|
||||
public long ramBytesUsed() {
|
||||
long sizeInBytes = 0;
|
||||
sizeInBytes += RamUsageEstimator.sizeOf(minValues);
|
||||
sizeInBytes += RamUsageEstimator.sizeOf(averages);
|
||||
for(PackedInts.Reader reader: subReaders) {
|
||||
sizeInBytes += reader.ramBytesUsed();
|
||||
}
|
||||
return sizeInBytes;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -22,6 +22,7 @@ import java.io.IOException;
|
|||
import org.apache.lucene.index.BinaryDocValues;
|
||||
import org.apache.lucene.store.DataInput;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
class Facet42BinaryDocValues extends BinaryDocValues {
|
||||
|
@ -44,4 +45,9 @@ class Facet42BinaryDocValues extends BinaryDocValues {
|
|||
ret.length = (int) (addresses.get(docID+1)-start);
|
||||
}
|
||||
|
||||
/** Returns approximate RAM bytes used */
|
||||
public long ramBytesUsed() {
|
||||
return RamUsageEstimator.sizeOf(bytes) + addresses.ramBytesUsed();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -91,4 +91,13 @@ class Facet42DocValuesProducer extends DocValuesProducer {
|
|||
@Override
|
||||
public void close() throws IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
long size = 0;
|
||||
for (Facet42BinaryDocValues entry: fields.values()) {
|
||||
size += entry.ramBytesUsed() + Integer.SIZE;
|
||||
}
|
||||
return size;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -37,6 +37,7 @@ import org.apache.lucene.util.Bits;
|
|||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.FixedBitSet;
|
||||
import org.apache.lucene.util.OpenBitSet;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
/**
|
||||
* Just like {@link Lucene45DocValuesFormat} but with additional asserts.
|
||||
|
@ -301,5 +302,10 @@ public class AssertingDocValuesFormat extends DocValuesFormat {
|
|||
public void close() throws IOException {
|
||||
in.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return in.ramBytesUsed();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -86,6 +86,11 @@ public final class AssertingPostingsFormat extends PostingsFormat {
|
|||
public int size() {
|
||||
return in.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return in.ramBytesUsed();
|
||||
}
|
||||
}
|
||||
|
||||
static class AssertingFieldsConsumer extends FieldsConsumer {
|
||||
|
|
|
@ -71,6 +71,11 @@ public class AssertingStoredFieldsFormat extends StoredFieldsFormat {
|
|||
public StoredFieldsReader clone() {
|
||||
return new AssertingStoredFieldsReader(in.clone(), maxDoc);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return in.ramBytesUsed();
|
||||
}
|
||||
}
|
||||
|
||||
enum Status {
|
||||
|
|
|
@ -71,6 +71,11 @@ public class AssertingTermVectorsFormat extends TermVectorsFormat {
|
|||
public TermVectorsReader clone() {
|
||||
return new AssertingTermVectorsReader(in.clone());
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return in.ramBytesUsed();
|
||||
}
|
||||
}
|
||||
|
||||
enum Status {
|
||||
|
|
|
@ -50,6 +50,7 @@ import org.apache.lucene.store.IndexOutput;
|
|||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
/** Stores all postings data in RAM, but writes a small
|
||||
* token (header + single int) to identify which "slot" the
|
||||
|
@ -120,6 +121,15 @@ public final class RAMOnlyPostingsFormat extends PostingsFormat {
|
|||
@Override
|
||||
public void close() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
long sizeInBytes = 0;
|
||||
for(RAMField field : fieldToTerms.values()) {
|
||||
sizeInBytes += field.ramBytesUsed();
|
||||
}
|
||||
return sizeInBytes;
|
||||
}
|
||||
}
|
||||
|
||||
static class RAMField extends Terms {
|
||||
|
@ -135,6 +145,15 @@ public final class RAMOnlyPostingsFormat extends PostingsFormat {
|
|||
this.info = info;
|
||||
}
|
||||
|
||||
/** Returns approximate RAM bytes used */
|
||||
public long ramBytesUsed() {
|
||||
long sizeInBytes = 0;
|
||||
for(RAMTerm term : termToDocs.values()) {
|
||||
sizeInBytes += term.ramBytesUsed();
|
||||
}
|
||||
return sizeInBytes;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long size() {
|
||||
return termToDocs.size();
|
||||
|
@ -188,6 +207,15 @@ public final class RAMOnlyPostingsFormat extends PostingsFormat {
|
|||
public RAMTerm(String term) {
|
||||
this.term = term;
|
||||
}
|
||||
|
||||
/** Returns approximate RAM bytes used */
|
||||
public long ramBytesUsed() {
|
||||
long sizeInBytes = 0;
|
||||
for(RAMDoc rDoc : docs) {
|
||||
sizeInBytes += rDoc.ramBytesUsed();
|
||||
}
|
||||
return sizeInBytes;
|
||||
}
|
||||
}
|
||||
|
||||
static class RAMDoc {
|
||||
|
@ -199,6 +227,19 @@ public final class RAMOnlyPostingsFormat extends PostingsFormat {
|
|||
this.docID = docID;
|
||||
positions = new int[freq];
|
||||
}
|
||||
|
||||
/** Returns approximate RAM bytes used */
|
||||
public long ramBytesUsed() {
|
||||
long sizeInBytes = 0;
|
||||
sizeInBytes += (positions!=null) ? RamUsageEstimator.sizeOf(positions) : 0;
|
||||
|
||||
if (payloads != null) {
|
||||
for(byte[] payload: payloads) {
|
||||
sizeInBytes += (payload!=null) ? RamUsageEstimator.sizeOf(payload) : 0;
|
||||
}
|
||||
}
|
||||
return sizeInBytes;
|
||||
}
|
||||
}
|
||||
|
||||
// Classes for writing to the postings state
|
||||
|
|
Loading…
Reference in New Issue