LUCENE-5700: Add oal.util.Accountable and make all classes that can compute their memory usage implement it.

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1598470 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Adrien Grand 2014-05-30 07:01:31 +00:00
parent 75e8b848e8
commit add1e44fd7
61 changed files with 254 additions and 217 deletions

View File

@ -171,6 +171,9 @@ API Changes
* LUCENE-5706: Removed the option to unset a DocValues field through DocValues
updates. (Shai Erera)
* LUCENE-5700: Added oal.util.Accountable that is now implemented by all
classes whose memory usage can be estimated. (Robert Muir, Adrien Grand)
Optimizations
* LUCENE-5603: hunspell stemmer more efficiently strips prefixes

View File

@ -164,7 +164,7 @@ public class TokenInfoDictionaryBuilder {
final FST<Long> fst = fstBuilder.finish();
System.out.print(" " + fst.getNodeCount() + " nodes, " + fst.getArcCount() + " arcs, " + fst.sizeInBytes() + " bytes... ");
System.out.print(" " + fst.getNodeCount() + " nodes, " + fst.getArcCount() + " arcs, " + fst.ramBytesUsed() + " bytes... ");
dictionary.setFST(fst);
System.out.println(" done");

View File

@ -24,6 +24,7 @@ import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.PagedBytes;
@ -223,7 +224,7 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
return true;
}
private final class FieldIndexData {
private final class FieldIndexData implements Accountable {
// where this field's terms begin in the packed byte[]
// data
final long termBytesStart;
@ -264,7 +265,7 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
}
}
/** Returns approximate RAM bytes used */
@Override
public long ramBytesUsed() {
return ((termOffsets!=null)? termOffsets.ramBytesUsed() : 0) +
((termsDictOffsets!=null)? termsDictOffsets.ramBytesUsed() : 0);

View File

@ -17,11 +17,12 @@ package org.apache.lucene.codecs.blockterms;
* limitations under the License.
*/
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.util.BytesRef;
import java.io.IOException;
import java.io.Closeable;
import java.io.IOException;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.BytesRef;
// TODO
@ -38,7 +39,7 @@ import java.io.Closeable;
* text.
* @lucene.experimental */
public abstract class TermsIndexReaderBase implements Closeable {
public abstract class TermsIndexReaderBase implements Closeable, Accountable {
public abstract FieldIndexEnum getFieldEnum(FieldInfo fieldInfo);
@ -71,6 +72,4 @@ public abstract class TermsIndexReaderBase implements Closeable {
public abstract long ord();
}
/** Returns approximate RAM bytes used */
public abstract long ramBytesUsed();
}

View File

@ -18,9 +18,6 @@ package org.apache.lucene.codecs.blockterms;
*/
import java.io.IOException;
import java.io.FileOutputStream; // for toDot
import java.io.OutputStreamWriter; // for toDot
import java.io.Writer; // for toDot
import java.util.HashMap;
import org.apache.lucene.codecs.CodecUtil;
@ -31,12 +28,12 @@ import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.fst.BytesRefFSTEnum;
import org.apache.lucene.util.fst.FST;
import org.apache.lucene.util.fst.PositiveIntOutputs;
import org.apache.lucene.util.fst.Util; // for toDot
/** See {@link VariableGapTermsIndexWriter}
*
@ -156,7 +153,7 @@ public class VariableGapTermsIndexReader extends TermsIndexReaderBase {
return false;
}
private final class FieldIndexData {
private final class FieldIndexData implements Accountable {
private final FST<Long> fst;
public FieldIndexData(IndexInput in, FieldInfo fieldInfo, long indexStart) throws IOException {
@ -174,9 +171,9 @@ public class VariableGapTermsIndexReader extends TermsIndexReaderBase {
*/
}
/** Returns approximate RAM bytes used */
@Override
public long ramBytesUsed() {
return fst == null ? 0 : fst.sizeInBytes();
return fst == null ? 0 : fst.ramBytesUsed();
}
}

View File

@ -20,6 +20,7 @@ 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.BytesRef;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.RamUsageEstimator;
@ -46,7 +47,7 @@ import org.apache.lucene.util.RamUsageEstimator;
* </p>
* @lucene.experimental
*/
public class FuzzySet {
public class FuzzySet implements Accountable {
public static final int VERSION_SPI = 1; // HashFunction used to be loaded through a SPI
public static final int VERSION_START = VERSION_SPI;
@ -304,6 +305,7 @@ public class FuzzySet {
return (float) numBitsSet / (float) bloomSize;
}
@Override
public long ramBytesUsed() {
return RamUsageEstimator.sizeOf(filter.getBits());
}

View File

@ -29,8 +29,8 @@ import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat; // javadocs
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.OrdTermState;
import org.apache.lucene.index.SegmentReadState;
@ -40,6 +40,7 @@ import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
@ -166,13 +167,10 @@ public final class DirectPostingsFormat extends PostingsFormat {
}
}
private final static class DirectField extends Terms {
private final static class DirectField extends Terms implements Accountable {
private static abstract class TermAndSkip {
private static abstract class TermAndSkip implements Accountable {
public int[] skips;
/** Returns the approximate number of RAM bytes used */
public abstract long ramBytesUsed();
}
private static final class LowFreqTerm extends TermAndSkip {
@ -494,7 +492,7 @@ public final class DirectPostingsFormat extends PostingsFormat {
assert skipOffset == skipCount;
}
/** Returns approximate RAM bytes used */
@Override
public long ramBytesUsed() {
long sizeInBytes = 0;
sizeInBytes += ((termBytes!=null) ? RamUsageEstimator.sizeOf(termBytes) : 0);

View File

@ -830,7 +830,7 @@ public class FSTOrdTermsReader extends FieldsProducer {
long ramBytesUsed = 0;
for (TermsReader r : fields.values()) {
if (r.index != null) {
ramBytesUsed += r.index.sizeInBytes();
ramBytesUsed += r.index.ramBytesUsed();
ramBytesUsed += RamUsageEstimator.sizeOf(r.metaBytesBlock);
ramBytesUsed += RamUsageEstimator.sizeOf(r.metaLongsBlock);
ramBytesUsed += RamUsageEstimator.sizeOf(r.skipInfo);

View File

@ -734,7 +734,7 @@ public class FSTTermsReader extends FieldsProducer {
public long ramBytesUsed() {
long ramBytesUsed = 0;
for (TermsReader r : fields.values()) {
ramBytesUsed += r.dict == null ? 0 : r.dict.sizeInBytes();
ramBytesUsed += r.dict == null ? 0 : r.dict.ramBytesUsed();
}
return ramBytesUsed;
}

View File

@ -330,7 +330,7 @@ class MemoryDocValuesProducer extends DocValuesProducer {
if (instance == null) {
data.seek(entry.offset);
instance = new FST<>(data, PositiveIntOutputs.getSingleton());
ramBytesUsed.addAndGet(instance.sizeInBytes());
ramBytesUsed.addAndGet(instance.ramBytesUsed());
fstInstances.put(field.number, instance);
}
}
@ -405,7 +405,7 @@ class MemoryDocValuesProducer extends DocValuesProducer {
if (instance == null) {
data.seek(entry.offset);
instance = new FST<>(data, PositiveIntOutputs.getSingleton());
ramBytesUsed.addAndGet(instance.sizeInBytes());
ramBytesUsed.addAndGet(instance.ramBytesUsed());
fstInstances.put(field.number, instance);
}
}

View File

@ -31,8 +31,8 @@ import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.TermStats;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.IndexFileNames;
@ -46,6 +46,7 @@ import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
@ -877,7 +878,7 @@ public final class MemoryPostingsFormat extends PostingsFormat {
}
}
private final static class TermsReader extends Terms {
private final static class TermsReader extends Terms implements Accountable {
private final long sumTotalTermFreq;
private final long sumDocFreq;
@ -947,8 +948,9 @@ public final class MemoryPostingsFormat extends PostingsFormat {
return field.hasPayloads();
}
@Override
public long ramBytesUsed() {
return ((fst!=null) ? fst.sizeInBytes() : 0);
return ((fst!=null) ? fst.ramBytesUsed() : 0);
}
}

View File

@ -17,6 +17,16 @@ package org.apache.lucene.codecs.simpletext;
* limitations under the License.
*/
import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.DOC;
import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.END;
import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.END_OFFSET;
import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.FIELD;
import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.FREQ;
import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.PAYLOAD;
import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.POS;
import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.START_OFFSET;
import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.TERM;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.Collections;
@ -37,6 +47,7 @@ import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.BufferedChecksumIndexInput;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
@ -53,16 +64,6 @@ import org.apache.lucene.util.fst.PairOutputs;
import org.apache.lucene.util.fst.PositiveIntOutputs;
import org.apache.lucene.util.fst.Util;
import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.END;
import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.FIELD;
import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.TERM;
import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.DOC;
import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.FREQ;
import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.POS;
import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.START_OFFSET;
import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.END_OFFSET;
import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.PAYLOAD;
class SimpleTextFieldsReader extends FieldsProducer {
private final TreeMap<String,Long> fields;
private final IndexInput in;
@ -496,7 +497,7 @@ class SimpleTextFieldsReader extends FieldsProducer {
}
}
private class SimpleTextTerms extends Terms {
private class SimpleTextTerms extends Terms implements Accountable {
private final long termsStart;
private final FieldInfo fieldInfo;
private final int maxDoc;
@ -578,9 +579,9 @@ class SimpleTextFieldsReader extends FieldsProducer {
//System.out.println("FST " + fst.sizeInBytes());
}
/** Returns approximate RAM bytes used */
@Override
public long ramBytesUsed() {
return (fst!=null) ? fst.sizeInBytes() : 0;
return (fst!=null) ? fst.ramBytesUsed() : 0;
}
@Override

View File

@ -25,6 +25,7 @@ import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Bits;
/** Abstract API that produces numeric, binary and
@ -32,7 +33,7 @@ import org.apache.lucene.util.Bits;
*
* @lucene.experimental
*/
public abstract class DocValuesProducer implements Closeable {
public abstract class DocValuesProducer implements Closeable, Accountable {
/** Sole constructor. (For invocation by subclass
* constructors, typically implicit.) */
@ -64,9 +65,6 @@ 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();
/**
* Checks consistency of this producer
* <p>

View File

@ -21,6 +21,7 @@ import java.io.Closeable;
import java.io.IOException;
import org.apache.lucene.index.Fields;
import org.apache.lucene.util.Accountable;
/** Abstract API that produces terms, doc, freq, prox, offset and
* payloads postings.
@ -28,7 +29,7 @@ import org.apache.lucene.index.Fields;
* @lucene.experimental
*/
public abstract class FieldsProducer extends Fields implements Closeable {
public abstract class FieldsProducer extends Fields implements Closeable, Accountable {
/** Sole constructor. (For invocation by subclass
* constructors, typically implicit.) */
protected FieldsProducer() {
@ -37,9 +38,6 @@ public abstract class FieldsProducer extends Fields implements Closeable {
@Override
public abstract void close() throws IOException;
/** Returns approximate RAM bytes used */
public abstract long ramBytesUsed();
/**
* Checks consistency of this reader.
* <p>

View File

@ -17,14 +17,15 @@ package org.apache.lucene.codecs;
* limitations under the License.
*/
import java.io.IOException;
import java.io.Closeable;
import java.io.IOException;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Bits;
/** The core terms dictionaries (BlockTermsReader,
@ -42,7 +43,7 @@ import org.apache.lucene.util.Bits;
// TODO: find a better name; this defines the API that the
// terms dict impls use to talk to a postings impl.
// TermsDict + PostingsReader/WriterBase == PostingsConsumer/Producer
public abstract class PostingsReaderBase implements Closeable {
public abstract class PostingsReaderBase implements Closeable, Accountable {
/** Sole constructor. (For invocation by subclass
* constructors, typically implicit.) */
@ -71,9 +72,6 @@ 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();
/**
* Checks consistency of this reader.
* <p>

View File

@ -20,6 +20,7 @@ import java.io.Closeable;
import java.io.IOException;
import org.apache.lucene.index.StoredFieldVisitor;
import org.apache.lucene.util.Accountable;
/**
* Codec API for reading stored fields.
@ -29,7 +30,7 @@ import org.apache.lucene.index.StoredFieldVisitor;
* clones of any IndexInputs used, etc), and {@link #close()}
* @lucene.experimental
*/
public abstract class StoredFieldsReader implements Cloneable, Closeable {
public abstract class StoredFieldsReader implements Cloneable, Closeable, Accountable {
/** Sole constructor. (For invocation by subclass
* constructors, typically implicit.) */
protected StoredFieldsReader() {
@ -41,9 +42,6 @@ public abstract class StoredFieldsReader implements Cloneable, Closeable {
@Override
public abstract StoredFieldsReader clone();
/** Returns approximate RAM bytes used */
public abstract long ramBytesUsed();
/**
* Checks consistency of this reader.
* <p>

View File

@ -23,13 +23,14 @@ import java.io.IOException;
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute; // javadocs
import org.apache.lucene.index.DocsAndPositionsEnum; // javadocs
import org.apache.lucene.index.Fields;
import org.apache.lucene.util.Accountable;
/**
* Codec API for reading term vectors:
*
* @lucene.experimental
*/
public abstract class TermVectorsReader implements Cloneable, Closeable {
public abstract class TermVectorsReader implements Cloneable, Closeable, Accountable {
/** Sole constructor. (For invocation by subclass
* constructors, typically implicit.) */
@ -42,9 +43,6 @@ 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();
/**
* Checks consistency of this reader.
* <p>

View File

@ -19,12 +19,13 @@ package org.apache.lucene.codecs.blocktree;
import java.io.IOException;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
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.automaton.CompiledAutomaton;
import org.apache.lucene.util.fst.ByteSequenceOutputs;
@ -32,7 +33,7 @@ import org.apache.lucene.util.fst.FST;
/** BlockTree's implementation of {@link Terms}. */
// public for CheckIndex:
public final class FieldReader extends Terms {
public final class FieldReader extends Terms implements Accountable {
final long numTerms;
final FieldInfo fieldInfo;
final long sumTotalTermFreq;
@ -169,8 +170,8 @@ public final class FieldReader extends Terms {
return new IntersectTermsEnum(this, compiled, startTerm);
}
/** Returns approximate RAM bytes used */
@Override
public long ramBytesUsed() {
return ((index!=null)? index.sizeInBytes() : 0);
return ((index!=null)? index.ramBytesUsed() : 0);
}
}

View File

@ -126,7 +126,7 @@ final class SegmentTermsEnum extends TermsEnum {
if (fr.index != null) {
stats.indexNodeCount = fr.index.getNodeCount();
stats.indexArcCount = fr.index.getArcCount();
stats.indexNumBytes = fr.index.sizeInBytes();
stats.indexNumBytes = fr.index.ramBytesUsed();
}
currentFrame = staticFrame;

View File

@ -25,6 +25,7 @@ import java.util.Arrays;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.packed.PackedInts;
@ -33,7 +34,7 @@ import org.apache.lucene.util.packed.PackedInts;
* Random-access reader for {@link CompressingStoredFieldsIndexWriter}.
* @lucene.internal
*/
public final class CompressingStoredFieldsIndexReader implements Cloneable {
public final class CompressingStoredFieldsIndexReader implements Cloneable, Accountable {
final int maxDoc;
final int[] docBases;
@ -160,7 +161,8 @@ public final class CompressingStoredFieldsIndexReader implements Cloneable {
return this;
}
long ramBytesUsed() {
@Override
public long ramBytesUsed() {
long res = 0;
for(PackedInts.Reader r : docBasesDeltas) {

View File

@ -319,7 +319,7 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
if (instance == null) {
data.seek(entry.offset);
instance = new FST<>(data, PositiveIntOutputs.getSingleton());
ramBytesUsed.addAndGet(instance.sizeInBytes());
ramBytesUsed.addAndGet(instance.ramBytesUsed());
fstInstances.put(field.number, instance);
}
}
@ -394,7 +394,7 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
if (instance == null) {
data.seek(entry.offset);
instance = new FST<>(data, PositiveIntOutputs.getSingleton());
ramBytesUsed.addAndGet(instance.sizeInBytes());
ramBytesUsed.addAndGet(instance.ramBytesUsed());
fstInstances.put(field.number, instance);
}
}

View File

@ -112,7 +112,7 @@ class FrozenBufferedUpdates {
}
binaryDVUpdates = allBinaryUpdates.toArray(new BinaryDocValuesUpdate[allBinaryUpdates.size()]);
bytesUsed = (int) terms.getSizeInBytes() + queries.length * BYTES_PER_DEL_QUERY
bytesUsed = (int) terms.ramBytesUsed() + queries.length * BYTES_PER_DEL_QUERY
+ numericUpdatesSize + numericDVUpdates.length * RamUsageEstimator.NUM_BYTES_OBJECT_REF
+ binaryUpdatesSize + binaryDVUpdates.length * RamUsageEstimator.NUM_BYTES_OBJECT_REF;

View File

@ -22,6 +22,7 @@ import java.util.List;
import org.apache.lucene.index.MultiTermsEnum.TermsEnumIndex;
import org.apache.lucene.index.MultiTermsEnum.TermsEnumWithSlice;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.packed.AppendingPackedLongBuffer;
@ -318,7 +319,7 @@ public class MultiDocValues {
/** maps per-segment ordinals to/from global ordinal space */
// TODO: use more efficient packed ints structures?
// TODO: pull this out? its pretty generic (maps between N ord()-enabled TermsEnums)
public static class OrdinalMap {
public static class OrdinalMap implements Accountable {
// 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
@ -413,9 +414,7 @@ public class MultiDocValues {
return globalOrdDeltas.size();
}
/**
* Returns total byte size used by this ordinal map.
*/
@Override
public long ramBytesUsed() {
long size = globalOrdDeltas.ramBytesUsed() + firstSegments.ramBytesUsed();
for (int i = 0; i < ordDeltas.length; i++) {

View File

@ -24,22 +24,23 @@ import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.RAMFile;
import org.apache.lucene.store.RAMInputStream;
import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.BytesRef;
/**
* Prefix codes term instances (prefixes are shared)
* @lucene.experimental
*/
class PrefixCodedTerms implements Iterable<Term> {
class PrefixCodedTerms implements Iterable<Term>, Accountable {
final RAMFile buffer;
private PrefixCodedTerms(RAMFile buffer) {
this.buffer = buffer;
}
/** @return size in bytes */
public long getSizeInBytes() {
return buffer.getSizeInBytes();
@Override
public long ramBytesUsed() {
return buffer.ramBytesUsed();
}
/** @return iterator over the bytes */

View File

@ -17,6 +17,14 @@ package org.apache.lucene.index;
* limitations under the License.
*/
import java.io.IOException;
import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedHashSet;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.codecs.FieldsProducer;
@ -28,20 +36,13 @@ import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
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 java.io.IOException;
import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedHashSet;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
/** Holds core readers that are shared (unchanged) when
* SegmentReader is cloned or reopened */
final class SegmentCoreReaders {
final class SegmentCoreReaders implements Accountable {
// Counts how many other readers share the core objects
// (freqStream, proxStream, tis, etc.) of this reader;
@ -210,7 +211,7 @@ final class SegmentCoreReaders {
coreClosedListeners.remove(listener);
}
/** Returns approximate RAM bytes used */
@Override
public long ramBytesUsed() {
return ((normsProducer!=null) ? normsProducer.ramBytesUsed() : 0) +
((fields!=null) ? fields.ramBytesUsed() : 0) +

View File

@ -37,6 +37,7 @@ import org.apache.lucene.search.CachingWrapperFilter;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.CloseableThreadLocal;
import org.apache.lucene.util.IOUtils;
@ -49,7 +50,7 @@ import org.apache.lucene.util.Version;
* may share the same core data.
* @lucene.experimental
*/
public final class SegmentReader extends AtomicReader {
public final class SegmentReader extends AtomicReader implements Accountable {
private final SegmentCommitInfo si;
private final Bits liveDocs;
@ -570,7 +571,7 @@ public final class SegmentReader extends AtomicReader {
core.removeCoreClosedListener(listener);
}
/** Returns approximate RAM Bytes used */
@Override
public long ramBytesUsed() {
ensureOpen();
long ramBytesUsed = 0;

View File

@ -26,6 +26,7 @@ import java.util.WeakHashMap;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.WAH8DocIdSet;
@ -35,7 +36,7 @@ import org.apache.lucene.util.WAH8DocIdSet;
* filters to simply filter, and then wrap with this class
* to add caching.
*/
public class CachingWrapperFilter extends Filter {
public class CachingWrapperFilter extends Filter implements Accountable {
private final Filter filter;
private final Map<Object,DocIdSet> cache = Collections.synchronizedMap(new WeakHashMap<Object,DocIdSet>());
@ -149,8 +150,8 @@ public class CachingWrapperFilter extends Filter {
}
};
/** Returns total byte size used by cached filters. */
public long sizeInBytes() {
@Override
public long ramBytesUsed() {
// Sync only to pull the current set of values:
List<DocIdSet> docIdSets;

View File

@ -148,8 +148,8 @@ public class NRTCachingDirectory extends Directory {
/** Returns how many bytes are being used by the
* RAMDirectory cache */
public long sizeInBytes() {
return cache.sizeInBytes();
public long cacheRamBytesUsed() {
return cache.ramBytesUsed();
}
@Override
@ -256,7 +256,7 @@ public class NRTCachingDirectory extends Directory {
bytes = context.flushInfo.estimatedSegmentSize;
}
return !name.equals(IndexFileNames.SEGMENTS_GEN) && (bytes <= maxMergeSizeBytes) && (bytes + cache.sizeInBytes()) <= maxCachedBytes;
return !name.equals(IndexFileNames.SEGMENTS_GEN) && (bytes <= maxMergeSizeBytes) && (bytes + cache.ramBytesUsed()) <= maxCachedBytes;
}
private final Object uncacheLock = new Object();

View File

@ -27,6 +27,8 @@ import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.util.Accountable;
/**
* A memory-resident {@link Directory} implementation. Locking
@ -45,7 +47,7 @@ import java.util.concurrent.atomic.AtomicLong;
* implementation working directly on the file system cache of the
* operating system, so copying data to Java heap space is not useful.
*/
public class RAMDirectory extends BaseDirectory {
public class RAMDirectory extends BaseDirectory implements Accountable {
protected final Map<String,RAMFile> fileMap = new ConcurrentHashMap<>();
protected final AtomicLong sizeInBytes = new AtomicLong();
@ -140,7 +142,8 @@ public class RAMDirectory extends BaseDirectory {
* Return total size in bytes of all files in this directory. This is
* currently quantized to RAMOutputStream.BUFFER_SIZE.
*/
public final long sizeInBytes() {
@Override
public final long ramBytesUsed() {
ensureOpen();
return sizeInBytes.get();
}

View File

@ -19,10 +19,12 @@ package org.apache.lucene.store;
import java.util.ArrayList;
import org.apache.lucene.util.Accountable;
/**
* Represents a file in RAM as a list of byte[] buffers.
* @lucene.internal */
public class RAMFile {
public class RAMFile implements Accountable {
protected ArrayList<byte[]> buffers = new ArrayList<>();
long length;
RAMDirectory directory;
@ -75,7 +77,8 @@ public class RAMFile {
return new byte[size];
}
public synchronized long getSizeInBytes() {
@Override
public synchronized long ramBytesUsed() {
return sizeInBytes;
}

View File

@ -21,12 +21,14 @@ import java.io.IOException;
import java.util.zip.CRC32;
import java.util.zip.Checksum;
import org.apache.lucene.util.Accountable;
/**
* A memory-resident {@link IndexOutput} implementation.
*
* @lucene.internal
*/
public class RAMOutputStream extends IndexOutput {
public class RAMOutputStream extends IndexOutput implements Accountable {
static final int BUFFER_SIZE = 1024;
private RAMFile file;
@ -176,7 +178,8 @@ public class RAMOutputStream extends IndexOutput {
}
/** Returns byte usage of all buffers. */
public long sizeInBytes() {
@Override
public long ramBytesUsed() {
return (long) file.numBuffers() * (long) BUFFER_SIZE;
}

View File

@ -0,0 +1,32 @@
package org.apache.lucene.util;
/*
* 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.
*/
/**
* An object whose RAM usage can be computed.
*
* @lucene.internal
*/
public interface Accountable {
/**
* Return the memory usage of this object in bytes. Negative values are illegal.
*/
long ramBytesUsed();
}

View File

@ -33,7 +33,7 @@ import org.apache.lucene.util.packed.PackedInts;
* <p>On the contrary to the original PFOR paper, exceptions are encoded with
* FOR instead of Simple16.</p>
*/
public final class PForDeltaDocIdSet extends DocIdSet {
public final class PForDeltaDocIdSet extends DocIdSet implements Accountable {
static final int BLOCK_SIZE = 128;
static final int MAX_EXCEPTIONS = 24; // no more than 24 exceptions per block
@ -511,7 +511,7 @@ public final class PForDeltaDocIdSet extends DocIdSet {
return cardinality;
}
/** Return the memory usage of this instance. */
@Override
public long ramBytesUsed() {
return RamUsageEstimator.alignObjectSize(3 * RamUsageEstimator.NUM_BYTES_OBJECT_REF) + docIDs.ramBytesUsed() + offsets.ramBytesUsed();
}

View File

@ -34,7 +34,7 @@ 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 {
public final class PagedBytes implements Accountable {
private final List<byte[]> blocks = new ArrayList<>();
// TODO: these are unused?
private final List<Integer> blockEnd = new ArrayList<>();
@ -53,7 +53,7 @@ public final class PagedBytes {
* PagedBytes.
*
* @see #freeze */
public final static class Reader {
public final static class Reader implements Accountable {
private final byte[][] blocks;
private final int[] blockEnds;
private final int blockBits;
@ -130,7 +130,7 @@ public final class PagedBytes {
}
}
/** Returns approximate RAM bytes used */
@Override
public long ramBytesUsed() {
return ((blocks!=null) ? (blockSize * blocks.length) : 0);
}
@ -229,7 +229,7 @@ public final class PagedBytes {
}
}
/** Return approx RAM usage in bytes. */
@Override
public long ramBytesUsed() {
return (blocks.size() + (currentBlock != null ? 1 : 0)) * bytesUsedPerBlock;
}

View File

@ -74,7 +74,7 @@ import org.apache.lucene.util.packed.PackedInts;
* the next doc ID by reading at most 2 dirty words.</p>
* @lucene.experimental
*/
public final class WAH8DocIdSet extends DocIdSet {
public final class WAH8DocIdSet extends DocIdSet implements Accountable {
// Minimum index interval, intervals below this value can't guarantee anymore
// that this set implementation won't be significantly larger than a FixedBitSet
@ -736,7 +736,7 @@ public final class WAH8DocIdSet extends DocIdSet {
return cardinality;
}
/** Return the memory usage of this class in bytes. */
@Override
public long ramBytesUsed() {
return RamUsageEstimator.alignObjectSize(3 * RamUsageEstimator.NUM_BYTES_OBJECT_REF + 2 * RamUsageEstimator.NUM_BYTES_INT)
+ RamUsageEstimator.sizeOf(data)

View File

@ -515,8 +515,8 @@ public class Builder<T> {
boolean isCompiled();
}
public long fstSizeInBytes() {
return fst.sizeInBytes();
public long fstRamBytesUsed() {
return fst.ramBytesUsed();
}
static final class CompiledNode implements Node {

View File

@ -25,7 +25,6 @@ import java.io.FileOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.charset.StandardCharsets;
import java.util.HashMap;
import java.util.Map;
@ -36,6 +35,7 @@ import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.InputStreamDataInput;
import org.apache.lucene.store.OutputStreamDataOutput;
import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Constants;
import org.apache.lucene.util.IOUtils;
@ -44,8 +44,6 @@ import org.apache.lucene.util.PriorityQueue;
import org.apache.lucene.util.fst.Builder.UnCompiledNode;
import org.apache.lucene.util.packed.GrowableWriter;
import org.apache.lucene.util.packed.PackedInts;
//import java.io.Writer;
//import java.io.OutputStreamWriter;
// TODO: break this into WritableFST and ReadOnlyFST.. then
// we can have subclasses of ReadOnlyFST to handle the
@ -70,7 +68,7 @@ import org.apache.lucene.util.packed.PackedInts;
*
* @lucene.experimental
*/
public final class FST<T> {
public final class FST<T> implements Accountable {
/** Specifies allowed range of each int input label for
* this FST. */
public static enum INPUT_TYPE {BYTE1, BYTE2, BYTE4};
@ -392,8 +390,8 @@ public final class FST<T> {
return inputType;
}
/** Returns bytes used to represent the FST */
public long sizeInBytes() {
@Override
public long ramBytesUsed() {
long size = bytes.getPosition();
if (packed) {
size += nodeRefToAddress.ramBytesUsed();

View File

@ -17,6 +17,7 @@ package org.apache.lucene.util.packed;
* 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;
@ -26,7 +27,7 @@ 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 {
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
@ -188,7 +189,7 @@ abstract class AbstractAppendingLongBuffer extends LongValues {
+ RamUsageEstimator.NUM_BYTES_LONG; // valuesBytes
}
/** Return the number of bytes used by this instance. */
@Override
public long ramBytesUsed() {
// TODO: this is called per-doc-per-norms/dv-field, can we optimize this?
long bytesUsed = RamUsageEstimator.alignObjectSize(baseRamBytesUsed())

View File

@ -29,13 +29,14 @@ import static org.apache.lucene.util.packed.PackedInts.numBlocks;
import java.io.IOException;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.LongValues;
/**
* Provides random access to a stream written with {@link BlockPackedWriter}.
* @lucene.internal
*/
public final class BlockPackedReader extends LongValues {
public final class BlockPackedReader extends LongValues implements Accountable {
private final int blockShift, blockMask;
private final long valueCount;
@ -86,7 +87,7 @@ public final class BlockPackedReader extends LongValues {
return (minValues == null ? 0 : minValues[block]) + subReaders[block].get(idx);
}
/** Returns approximate RAM bytes used */
@Override
public long ramBytesUsed() {
long size = 0;
for (PackedInts.Reader reader : subReaders) {

View File

@ -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.Accountable;
import org.apache.lucene.util.LongValues;
import org.apache.lucene.util.RamUsageEstimator;
@ -34,7 +35,7 @@ import org.apache.lucene.util.RamUsageEstimator;
* {@link MonotonicBlockPackedWriter}.
* @lucene.internal
*/
public final class MonotonicBlockPackedReader extends LongValues {
public final class MonotonicBlockPackedReader extends LongValues implements Accountable {
private final int blockShift, blockMask;
private final long valueCount;
@ -86,7 +87,7 @@ public final class MonotonicBlockPackedReader extends LongValues {
return valueCount;
}
/** Returns the approximate RAM bytes used */
@Override
public long ramBytesUsed() {
long sizeInBytes = 0;
sizeInBytes += RamUsageEstimator.sizeOf(minValues);

View File

@ -25,6 +25,7 @@ import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.LongsRef;
import org.apache.lucene.util.RamUsageEstimator;
@ -452,7 +453,7 @@ public class PackedInts {
* A read-only random access array of positive integers.
* @lucene.internal
*/
public static abstract class Reader extends NumericDocValues {
public static abstract class Reader extends NumericDocValues implements Accountable {
/**
* Bulk get: read at least one and at most <code>len</code> longs starting
@ -484,11 +485,6 @@ public class PackedInts {
*/
public abstract int size();
/**
* Return the in-memory size in bytes.
*/
public abstract long ramBytesUsed();
/**
* Expert: if the bit-width of this reader matches one of
* java's native types, returns the underlying array

View File

@ -302,7 +302,7 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
CachingWrapperFilter filter = new CachingWrapperFilter(startFilter);
docs = searcher.search(new MatchAllDocsQuery(), filter, 1);
assertTrue(filter.sizeInBytes() > 0);
assertTrue(filter.ramBytesUsed() > 0);
assertEquals("[query + filter] Should find a hit...", 1, docs.totalHits);

View File

@ -70,7 +70,7 @@ public class Test2BFST extends LuceneTestCase {
b.add(input2, NO_OUTPUT);
count++;
if (count % 100000 == 0) {
System.out.println(count + ": " + b.fstSizeInBytes() + " bytes; " + b.getTotStateCount() + " nodes");
System.out.println(count + ": " + b.fstRamBytesUsed() + " bytes; " + b.getTotStateCount() + " nodes");
}
if (b.getTotStateCount() > Integer.MAX_VALUE + 100L * 1024 * 1024) {
break;
@ -81,7 +81,7 @@ public class Test2BFST extends LuceneTestCase {
FST<Object> fst = b.finish();
for(int verify=0;verify<2;verify++) {
System.out.println("\nTEST: now verify [fst size=" + fst.sizeInBytes() + "; nodeCount=" + fst.getNodeCount() + "; arcCount=" + fst.getArcCount() + "]");
System.out.println("\nTEST: now verify [fst size=" + fst.ramBytesUsed() + "; nodeCount=" + fst.getNodeCount() + "; arcCount=" + fst.getArcCount() + "]");
Arrays.fill(ints2, 0);
r = new Random(seed);
@ -151,9 +151,9 @@ public class Test2BFST extends LuceneTestCase {
b.add(input, BytesRef.deepCopyOf(output));
count++;
if (count % 1000000 == 0) {
System.out.println(count + "...: " + b.fstSizeInBytes() + " bytes");
System.out.println(count + "...: " + b.fstRamBytesUsed() + " bytes");
}
if (b.fstSizeInBytes() > LIMIT) {
if (b.fstRamBytesUsed() > LIMIT) {
break;
}
nextInput(r, ints);
@ -162,7 +162,7 @@ public class Test2BFST extends LuceneTestCase {
FST<BytesRef> fst = b.finish();
for(int verify=0;verify<2;verify++) {
System.out.println("\nTEST: now verify [fst size=" + fst.sizeInBytes() + "; nodeCount=" + fst.getNodeCount() + "; arcCount=" + fst.getArcCount() + "]");
System.out.println("\nTEST: now verify [fst size=" + fst.ramBytesUsed() + "; nodeCount=" + fst.getNodeCount() + "; arcCount=" + fst.getArcCount() + "]");
r = new Random(seed);
Arrays.fill(ints, 0);
@ -228,9 +228,9 @@ public class Test2BFST extends LuceneTestCase {
output += 1+r.nextInt(10);
count++;
if (count % 1000000 == 0) {
System.out.println(count + "...: " + b.fstSizeInBytes() + " bytes");
System.out.println(count + "...: " + b.fstRamBytesUsed() + " bytes");
}
if (b.fstSizeInBytes() > LIMIT) {
if (b.fstRamBytesUsed() > LIMIT) {
break;
}
nextInput(r, ints);
@ -240,7 +240,7 @@ public class Test2BFST extends LuceneTestCase {
for(int verify=0;verify<2;verify++) {
System.out.println("\nTEST: now verify [fst size=" + fst.sizeInBytes() + "; nodeCount=" + fst.getNodeCount() + "; arcCount=" + fst.getArcCount() + "]");
System.out.println("\nTEST: now verify [fst size=" + fst.ramBytesUsed() + "; nodeCount=" + fst.getNodeCount() + "; arcCount=" + fst.getArcCount() + "]");
Arrays.fill(ints, 0);

View File

@ -380,7 +380,7 @@ public class TestFSTs extends LuceneTestCase {
}
FST<Long> fst = builder.finish();
if (VERBOSE) {
System.out.println("FST: " + docCount + " docs; " + ord + " terms; " + fst.getNodeCount() + " nodes; " + fst.getArcCount() + " arcs;" + " " + fst.sizeInBytes() + " bytes");
System.out.println("FST: " + docCount + " docs; " + ord + " terms; " + fst.getNodeCount() + " nodes; " + fst.getArcCount() + " arcs;" + " " + fst.ramBytesUsed() + " bytes");
}
if (ord > 0) {
@ -516,7 +516,7 @@ public class TestFSTs extends LuceneTestCase {
return;
}
System.out.println(ord + " terms; " + fst.getNodeCount() + " nodes; " + fst.getArcCount() + " arcs; " + fst.getArcWithOutputCount() + " arcs w/ output; tot size " + fst.sizeInBytes());
System.out.println(ord + " terms; " + fst.getNodeCount() + " nodes; " + fst.getArcCount() + " arcs; " + fst.getArcWithOutputCount() + " arcs w/ output; tot size " + fst.ramBytesUsed());
if (fst.getNodeCount() < 100) {
Writer w = new OutputStreamWriter(new FileOutputStream("out.dot"), StandardCharsets.UTF_8);
Util.toDot(fst, w, false, false);

View File

@ -24,6 +24,7 @@ import java.util.WeakHashMap;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.RamUsageEstimator;
@ -53,7 +54,7 @@ import org.apache.lucene.util.RamUsageEstimator;
* for all facet implementations (the cache is per-instance,
* not static).
*/
public class CachedOrdinalsReader extends OrdinalsReader {
public class CachedOrdinalsReader extends OrdinalsReader implements Accountable {
private final OrdinalsReader source;
@ -94,7 +95,7 @@ public class CachedOrdinalsReader extends OrdinalsReader {
}
/** Holds the cached ordinals in two parallel {@code int[]} arrays. */
public static final class CachedOrds {
public static final class CachedOrds implements Accountable {
/** Index into {@link #ordinals} for each document. */
public final int[] offsets;
@ -137,7 +138,7 @@ public class CachedOrdinalsReader extends OrdinalsReader {
}
}
/** Returns number of bytes used by this cache entry */
@Override
public long ramBytesUsed() {
long mem = RamUsageEstimator.shallowSizeOf(this) + RamUsageEstimator.sizeOf(offsets);
if (offsets != ordinals) {
@ -147,7 +148,7 @@ public class CachedOrdinalsReader extends OrdinalsReader {
}
}
/** How many bytes is this cache using? */
@Override
public synchronized long ramBytesUsed() {
long bytes = 0;
for(CachedOrds ords : ordsCache.values()) {

View File

@ -19,21 +19,20 @@ package org.apache.lucene.codecs.idversion;
import java.io.IOException;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
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.automaton.CompiledAutomaton;
import org.apache.lucene.util.fst.ByteSequenceOutputs;
import org.apache.lucene.util.fst.FST;
import org.apache.lucene.util.fst.PairOutputs.Pair;
/** BlockTree's implementation of {@link Terms}. */
// public for CheckIndex:
final class VersionFieldReader extends Terms {
final class VersionFieldReader extends Terms implements Accountable {
final long numTerms;
final FieldInfo fieldInfo;
final long sumTotalTermFreq;
@ -156,8 +155,8 @@ final class VersionFieldReader extends Terms {
return docCount;
}
/** Returns approximate RAM bytes used */
@Override
public long ramBytesUsed() {
return ((index!=null)? index.sizeInBytes() : 0);
return ((index!=null)? index.ramBytesUsed() : 0);
}
}

View File

@ -29,6 +29,7 @@ import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.InputStreamDataInput;
import org.apache.lucene.store.OutputStreamDataOutput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.PriorityQueue;
@ -37,7 +38,7 @@ import org.apache.lucene.util.PriorityQueue;
* Simple Lookup interface for {@link CharSequence} suggestions.
* @lucene.experimental
*/
public abstract class Lookup {
public abstract class Lookup implements Accountable {
/**
* Result of a lookup.
@ -267,10 +268,4 @@ public abstract class Lookup {
*/
public abstract boolean load(DataInput input) throws IOException;
/**
* Get the size of the underlying lookup implementation in memory
* @return ram size of the lookup implementation in bytes
*/
public abstract long sizeInBytes();
}

View File

@ -647,7 +647,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
}
@Override
public long sizeInBytes() {
public long ramBytesUsed() {
long mem = RamUsageEstimator.shallowSizeOf(this);
try {
if (searcherMgr != null) {

View File

@ -251,8 +251,8 @@ public class AnalyzingSuggester extends Lookup {
/** Returns byte size of the underlying FST. */
@Override
public long sizeInBytes() {
return fst == null ? 0 : fst.sizeInBytes();
public long ramBytesUsed() {
return fst == null ? 0 : fst.ramBytesUsed();
}
private void copyDestTransitions(State from, State to, List<Transition> transitions) {

View File

@ -203,11 +203,11 @@ public class FreeTextSuggester extends Lookup {
/** Returns byte size of the underlying FST. */
@Override
public long sizeInBytes() {
public long ramBytesUsed() {
if (fst == null) {
return 0;
}
return fst.sizeInBytes();
return fst.ramBytesUsed();
}
private static class AnalyzingComparator implements Comparator<BytesRef> {

View File

@ -31,12 +31,13 @@ import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.ByteArrayDataOutput;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.util.Accountable;
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.OfflineSorter.SortInfo;
import org.apache.lucene.util.OfflineSorter;
import org.apache.lucene.util.OfflineSorter.SortInfo;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.UnicodeUtil;
import org.apache.lucene.util.fst.FST;
@ -66,7 +67,7 @@ import org.apache.lucene.util.fst.NoOutputs;
* @see FSTCompletion
* @lucene.experimental
*/
public class FSTCompletionLookup extends Lookup {
public class FSTCompletionLookup extends Lookup implements Accountable {
/**
* An invalid bucket count if we're creating an object
* of this class from an existing FST.
@ -298,14 +299,14 @@ public class FSTCompletionLookup extends Lookup {
}
@Override
public long sizeInBytes() {
public long ramBytesUsed() {
long mem = RamUsageEstimator.shallowSizeOf(this) + RamUsageEstimator.shallowSizeOf(normalCompletion) + RamUsageEstimator.shallowSizeOf(higherWeightsCompletion);
if (normalCompletion != null) {
mem += normalCompletion.getFST().sizeInBytes();
mem += normalCompletion.getFST().ramBytesUsed();
}
if (higherWeightsCompletion != null && (normalCompletion == null || normalCompletion.getFST() != higherWeightsCompletion.getFST())) {
// the fst should be shared between the 2 completion instances, don't count it twice
mem += higherWeightsCompletion.getFST().sizeInBytes();
mem += higherWeightsCompletion.getFST().ramBytesUsed();
}
return mem;
}

View File

@ -295,8 +295,8 @@ public class WFSTCompletionLookup extends Lookup {
/** Returns byte size of the underlying FST. */
@Override
public long sizeInBytes() {
return (fst == null) ? 0 : fst.sizeInBytes();
public long ramBytesUsed() {
return (fst == null) ? 0 : fst.ramBytesUsed();
}
@Override

View File

@ -27,6 +27,7 @@ import org.apache.lucene.search.suggest.Lookup;
import org.apache.lucene.search.suggest.jaspell.JaspellTernarySearchTrie.TSTNode;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CharsRef;
import org.apache.lucene.util.UnicodeUtil;
@ -37,7 +38,7 @@ import org.apache.lucene.util.UnicodeUtil;
*
* @see JaspellTernarySearchTrie
*/
public class JaspellLookup extends Lookup {
public class JaspellLookup extends Lookup implements Accountable {
JaspellTernarySearchTrie trie = new JaspellTernarySearchTrie();
private boolean usePrefix = true;
private int editDistance = 2;
@ -201,10 +202,9 @@ public class JaspellLookup extends Lookup {
return true;
}
/** Returns byte size of the underlying TST. */
@Override
public long sizeInBytes() {
return trie.sizeInBytes();
public long ramBytesUsed() {
return trie.ramBytesUsed();
}
@Override

View File

@ -39,6 +39,7 @@ import java.util.Locale;
import java.util.Vector;
import java.util.zip.GZIPInputStream;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.RamUsageEstimator;
@ -62,12 +63,12 @@ import org.apache.lucene.util.RamUsageEstimator;
* Sedgewick (Addison-Wesley, 1998) provides yet another view of ternary search
* trees.
*/
public class JaspellTernarySearchTrie {
public class JaspellTernarySearchTrie implements Accountable {
/**
* An inner class of Ternary Search Trie that represents a node in the trie.
*/
protected final class TSTNode {
protected final class TSTNode implements Accountable {
/** Index values for accessing relatives array. */
protected final static int PARENT = 0, LOKID = 1, EQKID = 2, HIKID = 3;
@ -94,12 +95,12 @@ public class JaspellTernarySearchTrie {
relatives[PARENT] = parent;
}
/** Return an approximate memory usage for this node and its sub-nodes. */
public long sizeInBytes() {
@Override
public long ramBytesUsed() {
long mem = RamUsageEstimator.shallowSizeOf(this) + RamUsageEstimator.shallowSizeOf(relatives);
for (TSTNode node : relatives) {
if (node != null) {
mem += node.sizeInBytes();
mem += node.ramBytesUsed();
}
}
return mem;
@ -884,12 +885,12 @@ public class JaspellTernarySearchTrie {
sortKeysNumReturnValues, sortKeysResult);
}
/** Return an approximate memory usage for this trie. */
public long sizeInBytes() {
@Override
public long ramBytesUsed() {
long mem = RamUsageEstimator.shallowSizeOf(this);
final TSTNode root = getRoot();
if (root != null) {
mem += root.sizeInBytes();
mem += root.ramBytesUsed();
}
return mem;
}

View File

@ -222,7 +222,7 @@ public class TSTLookup extends Lookup {
/** Returns byte size of the underlying TST */
@Override
public long sizeInBytes() {
public long ramBytesUsed() {
long mem = RamUsageEstimator.shallowSizeOf(this);
if (root != null) {
mem += root.sizeInBytes();

View File

@ -145,7 +145,7 @@ public class LookupBenchmarkTest extends LuceneTestCase {
System.err.println("-- RAM consumption");
for (Class<? extends Lookup> cls : benchmarkClasses) {
Lookup lookup = buildLookup(cls, dictionaryInput);
long sizeInBytes = lookup.sizeInBytes();
long sizeInBytes = lookup.ramBytesUsed();
System.err.println(
String.format(Locale.ROOT, "%-15s size[B]:%,13d",
lookup.getClass().getSimpleName(),

View File

@ -184,7 +184,7 @@ public class TestFreeTextSuggester extends LuceneTestCase {
}
});
if (VERBOSE) {
System.out.println(sug.sizeInBytes() + " bytes");
System.out.println(sug.ramBytesUsed() + " bytes");
List<LookupResult> results = sug.lookup("general r", 10);
System.out.println("results:");

View File

@ -45,6 +45,7 @@ import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.FixedBitSet;
@ -99,7 +100,7 @@ public final class RAMOnlyPostingsFormat extends PostingsFormat {
public void checkIntegrity() throws IOException {}
}
static class RAMField extends Terms {
static class RAMField extends Terms implements Accountable {
final String field;
final SortedMap<String,RAMTerm> termToDocs = new TreeMap<>();
long sumTotalTermFreq;
@ -112,7 +113,7 @@ public final class RAMOnlyPostingsFormat extends PostingsFormat {
this.info = info;
}
/** Returns approximate RAM bytes used */
@Override
public long ramBytesUsed() {
long sizeInBytes = 0;
for(RAMTerm term : termToDocs.values()) {
@ -167,7 +168,7 @@ public final class RAMOnlyPostingsFormat extends PostingsFormat {
}
}
static class RAMTerm {
static class RAMTerm implements Accountable {
final String term;
long totalTermFreq;
final List<RAMDoc> docs = new ArrayList<>();
@ -175,7 +176,7 @@ public final class RAMOnlyPostingsFormat extends PostingsFormat {
this.term = term;
}
/** Returns approximate RAM bytes used */
@Override
public long ramBytesUsed() {
long sizeInBytes = 0;
for(RAMDoc rDoc : docs) {
@ -185,7 +186,7 @@ public final class RAMOnlyPostingsFormat extends PostingsFormat {
}
}
static class RAMDoc {
static class RAMDoc implements Accountable {
final int docID;
final int[] positions;
byte[][] payloads;
@ -195,7 +196,7 @@ public final class RAMOnlyPostingsFormat extends PostingsFormat {
positions = new int[freq];
}
/** Returns approximate RAM bytes used */
@Override
public long ramBytesUsed() {
long sizeInBytes = 0;
sizeInBytes += (positions!=null) ? RamUsageEstimator.sizeOf(positions) : 0;

View File

@ -219,7 +219,7 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
public synchronized final long sizeInBytes() throws IOException {
if (in instanceof RAMDirectory)
return ((RAMDirectory) in).sizeInBytes();
return ((RAMDirectory) in).ramBytesUsed();
else {
// hack
long size = 0;
@ -607,7 +607,7 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
return sizeInBytes();
long size = 0;
for(final RAMFile file: ((RAMDirectory)in).fileMap.values()) {
size += file.getSizeInBytes();
size += file.ramBytesUsed();
}
return size;
}

View File

@ -32,6 +32,7 @@ import java.util.concurrent.ConcurrentHashMap;
import org.apache.lucene.search.suggest.Lookup;
import org.apache.lucene.search.suggest.Lookup.LookupResult;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CharsRef;
import org.apache.solr.common.params.CommonParams;
@ -56,7 +57,7 @@ import org.slf4j.LoggerFactory;
* Responsible for routing commands and queries to the appropriate {@link SolrSuggester}
* and for initializing them as specified by SolrConfig
*/
public class SuggestComponent extends SearchComponent implements SolrCoreAware, SuggesterParams {
public class SuggestComponent extends SearchComponent implements SolrCoreAware, SuggesterParams, Accountable {
private static final Logger LOG = LoggerFactory.getLogger(SuggestComponent.class);
/** Name used to identify whether the user query concerns this component */
@ -326,7 +327,7 @@ public class SuggestComponent extends SearchComponent implements SolrCoreAware,
@Override
public NamedList getStatistics() {
NamedList<String> stats = new SimpleOrderedMap<>();
stats.add("totalSizeInBytes", String.valueOf(sizeInBytes()));
stats.add("totalSizeInBytes", String.valueOf(ramBytesUsed()));
for (Map.Entry<String, SolrSuggester> entry : suggesters.entrySet()) {
SolrSuggester suggester = entry.getValue();
stats.add(entry.getKey(), suggester.toString());
@ -334,11 +335,11 @@ public class SuggestComponent extends SearchComponent implements SolrCoreAware,
return stats;
}
/** Returns the total size of all the suggester */
public long sizeInBytes() {
@Override
public long ramBytesUsed() {
long sizeInBytes = 0;
for (SolrSuggester suggester : suggesters.values()) {
sizeInBytes += suggester.sizeInBytes();
sizeInBytes += suggester.ramBytesUsed();
}
return sizeInBytes;
}

View File

@ -25,8 +25,9 @@ import java.io.IOException;
import java.util.List;
import org.apache.lucene.search.spell.Dictionary;
import org.apache.lucene.search.suggest.Lookup.LookupResult;
import org.apache.lucene.search.suggest.Lookup;
import org.apache.lucene.search.suggest.Lookup.LookupResult;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.IOUtils;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.core.CloseHook;
@ -41,7 +42,7 @@ import org.slf4j.LoggerFactory;
* Interacts (query/build/reload) with Lucene Suggesters through {@link Lookup} and
* {@link Dictionary}
* */
public class SolrSuggester {
public class SolrSuggester implements Accountable {
private static final Logger LOG = LoggerFactory.getLogger(SolrSuggester.class);
/** Name used when an unnamed suggester config is passed */
@ -205,9 +206,9 @@ public class SolrSuggester {
return name;
}
/** Returns the size of the in-memory data structure used by the underlying lookup implementation */
public long sizeInBytes() {
return lookup.sizeInBytes();
@Override
public long ramBytesUsed() {
return lookup.ramBytesUsed();
}
@Override
@ -217,7 +218,7 @@ public class SolrSuggester {
+ "storeDir=" + ((storeDir == null) ? "" : storeDir.getAbsoluteFile()) + ", "
+ "lookupImpl=" + lookupImpl + ", "
+ "dictionaryImpl=" + dictionaryImpl + ", "
+ "sizeInBytes=" + ((lookup!=null) ? String.valueOf(sizeInBytes()) : "0") + " ]";
+ "sizeInBytes=" + ((lookup!=null) ? String.valueOf(ramBytesUsed()) : "0") + " ]";
}
}