LUCENE-5949: Add Accountable.getChildResources

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1625275 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Robert Muir 2014-09-16 13:26:55 +00:00
parent deeb456560
commit e73e624af5
100 changed files with 2164 additions and 479 deletions

View File

@ -130,6 +130,8 @@ New Features
* LUCENE-5820: SuggestStopFilter should have a factory.
(Varun Thacker via Steve Rowe)
* LUCENE-5949: Add Accountable.getChildResources(). (Robert Muir)
API Changes:
* LUCENE-5900: Deprecated more constructors taking Version in *InfixSuggester and

View File

@ -37,6 +37,8 @@ import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
@ -57,12 +59,11 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
private static final String segmentSuffix = "dv";
// ram instances we have already loaded
private final Map<Integer,NumericDocValues> numericInstances =
new HashMap<>();
private final Map<Integer,BinaryDocValues> binaryInstances =
new HashMap<>();
private final Map<Integer,SortedDocValues> sortedInstances =
new HashMap<>();
private final Map<String,NumericDocValues> numericInstances = new HashMap<>();
private final Map<String,BinaryDocValues> binaryInstances = new HashMap<>();
private final Map<String,SortedDocValues> sortedInstances = new HashMap<>();
private final Map<String,Accountable> instanceInfo = new HashMap<>();
private final AtomicLong ramBytesUsed;
@ -75,7 +76,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
@Override
public synchronized NumericDocValues getNumeric(FieldInfo field) throws IOException {
NumericDocValues instance = numericInstances.get(field.number);
NumericDocValues instance = numericInstances.get(field.name);
if (instance == null) {
String fileName = IndexFileNames.segmentFileName(state.segmentInfo.name + "_" + Integer.toString(field.number), segmentSuffix, "dat");
IndexInput input = dir.openInput(fileName, state.context);
@ -115,7 +116,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
IOUtils.closeWhileHandlingException(input);
}
}
numericInstances.put(field.number, instance);
numericInstances.put(field.name, instance);
}
return instance;
}
@ -131,7 +132,9 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
for (int i = 0; i < values.length; i++) {
values[i] = input.readLong();
}
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(values));
long bytesUsed = RamUsageEstimator.sizeOf(values);
instanceInfo.put(field.name, Accountables.namedAccountable("long array", bytesUsed));
ramBytesUsed.addAndGet(bytesUsed);
return new NumericDocValues() {
@Override
public long get(int docID) {
@ -142,6 +145,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
final long minValue = input.readLong();
final long defaultValue = input.readLong();
final PackedInts.Reader reader = PackedInts.getReader(input);
instanceInfo.put(field.name, reader);
ramBytesUsed.addAndGet(reader.ramBytesUsed());
return new NumericDocValues() {
@Override
@ -170,7 +174,9 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
int maxDoc = state.segmentInfo.getDocCount();
final byte values[] = new byte[maxDoc];
input.readBytes(values, 0, values.length);
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(values));
long bytesUsed = RamUsageEstimator.sizeOf(values);
instanceInfo.put(field.name, Accountables.namedAccountable("byte array", bytesUsed));
ramBytesUsed.addAndGet(bytesUsed);
return new NumericDocValues() {
@Override
public long get(int docID) {
@ -192,7 +198,9 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
for (int i = 0; i < values.length; i++) {
values[i] = input.readShort();
}
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(values));
long bytesUsed = RamUsageEstimator.sizeOf(values);
instanceInfo.put(field.name, Accountables.namedAccountable("short array", bytesUsed));
ramBytesUsed.addAndGet(bytesUsed);
return new NumericDocValues() {
@Override
public long get(int docID) {
@ -214,7 +222,9 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
for (int i = 0; i < values.length; i++) {
values[i] = input.readInt();
}
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(values));
long bytesUsed = RamUsageEstimator.sizeOf(values);
instanceInfo.put(field.name, Accountables.namedAccountable("int array", bytesUsed));
ramBytesUsed.addAndGet(bytesUsed);
return new NumericDocValues() {
@Override
public long get(int docID) {
@ -236,7 +246,9 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
for (int i = 0; i < values.length; i++) {
values[i] = input.readLong();
}
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(values));
long bytesUsed = RamUsageEstimator.sizeOf(values);
instanceInfo.put(field.name, Accountables.namedAccountable("long array", bytesUsed));
ramBytesUsed.addAndGet(bytesUsed);
return new NumericDocValues() {
@Override
public long get(int docID) {
@ -258,7 +270,9 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
for (int i = 0; i < values.length; i++) {
values[i] = input.readInt();
}
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(values));
long bytesUsed = RamUsageEstimator.sizeOf(values);
instanceInfo.put(field.name, Accountables.namedAccountable("float array", bytesUsed));
ramBytesUsed.addAndGet(bytesUsed);
return new NumericDocValues() {
@Override
public long get(int docID) {
@ -280,7 +294,9 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
for (int i = 0; i < values.length; i++) {
values[i] = input.readLong();
}
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(values));
long bytesUsed = RamUsageEstimator.sizeOf(values);
instanceInfo.put(field.name, Accountables.namedAccountable("double array", bytesUsed));
ramBytesUsed.addAndGet(bytesUsed);
return new NumericDocValues() {
@Override
public long get(int docID) {
@ -291,7 +307,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
@Override
public synchronized BinaryDocValues getBinary(FieldInfo field) throws IOException {
BinaryDocValues instance = binaryInstances.get(field.number);
BinaryDocValues instance = binaryInstances.get(field.name);
if (instance == null) {
switch(LegacyDocValuesType.valueOf(field.getAttribute(legacyKey))) {
case BYTES_FIXED_STRAIGHT:
@ -309,7 +325,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
default:
throw new AssertionError();
}
binaryInstances.put(field.number, instance);
binaryInstances.put(field.name, instance);
}
return instance;
}
@ -329,6 +345,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
CodecUtil.checkEOF(input);
success = true;
ramBytesUsed.addAndGet(bytesReader.ramBytesUsed());
instanceInfo.put(field.name, bytesReader);
return new BinaryDocValues() {
@Override
@ -370,7 +387,9 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
CodecUtil.checkEOF(data);
CodecUtil.checkEOF(index);
success = true;
ramBytesUsed.addAndGet(bytesReader.ramBytesUsed() + reader.ramBytesUsed());
long bytesUsed = bytesReader.ramBytesUsed() + reader.ramBytesUsed();
ramBytesUsed.addAndGet(bytesUsed);
instanceInfo.put(field.name, Accountables.namedAccountable("variable straight", bytesUsed));
return new BinaryDocValues() {
@Override
public BytesRef get(int docID) {
@ -414,7 +433,9 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
final PackedInts.Reader reader = PackedInts.getReader(index);
CodecUtil.checkEOF(data);
CodecUtil.checkEOF(index);
ramBytesUsed.addAndGet(bytesReader.ramBytesUsed() + reader.ramBytesUsed());
long bytesUsed = bytesReader.ramBytesUsed() + reader.ramBytesUsed();
ramBytesUsed.addAndGet(bytesUsed);
instanceInfo.put(field.name, Accountables.namedAccountable("fixed deref", bytesUsed));
success = true;
return new BinaryDocValues() {
@Override
@ -457,7 +478,9 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
final PackedInts.Reader reader = PackedInts.getReader(index);
CodecUtil.checkEOF(data);
CodecUtil.checkEOF(index);
ramBytesUsed.addAndGet(bytesReader.ramBytesUsed() + reader.ramBytesUsed());
long bytesUsed = bytesReader.ramBytesUsed() + reader.ramBytesUsed();
ramBytesUsed.addAndGet(bytesUsed);
instanceInfo.put(field.name, Accountables.namedAccountable("variable deref", bytesUsed));
success = true;
return new BinaryDocValues() {
@ -490,7 +513,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
@Override
public synchronized SortedDocValues getSorted(FieldInfo field) throws IOException {
SortedDocValues instance = sortedInstances.get(field.number);
SortedDocValues instance = sortedInstances.get(field.name);
if (instance == null) {
String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name + "_" + Integer.toString(field.number), segmentSuffix, "dat");
String indexName = IndexFileNames.segmentFileName(state.segmentInfo.name + "_" + Integer.toString(field.number), segmentSuffix, "idx");
@ -520,7 +543,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
IOUtils.closeWhileHandlingException(data, index);
}
}
sortedInstances.put(field.number, instance);
sortedInstances.put(field.name, instance);
}
return instance;
}
@ -540,7 +563,9 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
bytes.copy(data, fixedLength * (long) valueCount);
final PagedBytes.Reader bytesReader = bytes.freeze(true);
final PackedInts.Reader reader = PackedInts.getReader(index);
ramBytesUsed.addAndGet(bytesReader.ramBytesUsed() + reader.ramBytesUsed());
long bytesUsed = bytesReader.ramBytesUsed() + reader.ramBytesUsed();
ramBytesUsed.addAndGet(bytesUsed);
instanceInfo.put(field.name, Accountables.namedAccountable("fixed sorted", bytesUsed));
return correctBuggyOrds(new SortedDocValues() {
@Override
@ -578,7 +603,9 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
final PackedInts.Reader ordsReader = PackedInts.getReader(index);
final int valueCount = addressReader.size() - 1;
ramBytesUsed.addAndGet(bytesReader.ramBytesUsed() + addressReader.ramBytesUsed() + ordsReader.ramBytesUsed());
long bytesUsed = bytesReader.ramBytesUsed() + addressReader.ramBytesUsed() + ordsReader.ramBytesUsed();
ramBytesUsed.addAndGet(bytesUsed);
instanceInfo.put(field.name, Accountables.namedAccountable("var sorted", bytesUsed));
return correctBuggyOrds(new SortedDocValues() {
@Override
@ -654,8 +681,18 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
public long ramBytesUsed() {
return ramBytesUsed.get();
}
@Override
public synchronized Iterable<? extends Accountable> getChildResources() {
return Accountables.namedAccountables("field", instanceInfo);
}
@Override
public void checkIntegrity() throws IOException {
}
@Override
public String toString() {
return getClass().getSimpleName();
}
}

View File

@ -23,6 +23,7 @@ import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.util.Accountable;
/**
* Reads 4.0/4.1 norms.
@ -51,9 +52,19 @@ class Lucene40NormsReader extends NormsProducer {
public long ramBytesUsed() {
return impl.ramBytesUsed();
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return impl.getChildResources();
}
@Override
public void checkIntegrity() throws IOException {
impl.checkIntegrity();
}
@Override
public String toString() {
return getClass().getSimpleName() + "(" + impl + ")";
}
}

View File

@ -19,6 +19,7 @@ package org.apache.lucene.codecs.lucene40;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.codecs.CodecUtil;
@ -35,6 +36,7 @@ import org.apache.lucene.store.DataInput;
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.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
@ -1163,8 +1165,17 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
public long ramBytesUsed() {
return 0;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
@Override
public void checkIntegrity() throws IOException {}
@Override
public String toString() {
return getClass().getSimpleName() + "(positions=" + (proxIn != null) + ")";
}
}

View File

@ -31,11 +31,13 @@ import org.apache.lucene.store.AlreadyClosedException;
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.IOUtils;
import org.apache.lucene.util.RamUsageEstimator;
import java.io.Closeable;
import java.nio.charset.StandardCharsets;
import java.util.Collections;
/**
* Class responsible for access to stored document fields.
@ -258,7 +260,17 @@ public final class Lucene40StoredFieldsReader extends StoredFieldsReader impleme
public long ramBytesUsed() {
return RAM_BYTES_USED;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
@Override
public void checkIntegrity() throws IOException {}
@Override
public String toString() {
return getClass().getSimpleName();
}
}

View File

@ -19,6 +19,7 @@ package org.apache.lucene.codecs.lucene40;
import java.io.Closeable;
import java.io.IOException;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.Map;
@ -26,6 +27,7 @@ import java.util.NoSuchElementException;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo;
@ -38,6 +40,7 @@ import org.apache.lucene.index.TermsEnum;
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.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
@ -121,8 +124,12 @@ public class Lucene40TermVectorsReader extends TermVectorsReader implements Clos
assert HEADER_LENGTH_INDEX == tvx.getFilePointer();
assert HEADER_LENGTH_DOCS == tvd.getFilePointer();
assert HEADER_LENGTH_FIELDS == tvf.getFilePointer();
assert tvxVersion == tvdVersion;
assert tvxVersion == tvfVersion;
if (tvxVersion != tvdVersion) {
throw new CorruptIndexException("version mismatch: tvx=" + tvxVersion + " != tvd=" + tvdVersion + " (resource=" + tvd + ")");
}
if (tvxVersion != tvfVersion) {
throw new CorruptIndexException("version mismatch: tvx=" + tvxVersion + " != tvf=" + tvfVersion + " (resource=" + tvf + ")");
}
numTotalDocs = (int) (tvx.length()-HEADER_LENGTH_INDEX >> 4);
@ -705,8 +712,18 @@ public class Lucene40TermVectorsReader extends TermVectorsReader implements Clos
public long ramBytesUsed() {
return 0;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
@Override
public void checkIntegrity() throws IOException {}
@Override
public String toString() {
return getClass().getSimpleName();
}
}

View File

@ -18,7 +18,10 @@ package org.apache.lucene.codecs.lucene42;
*/
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicLong;
@ -41,6 +44,8 @@ import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
@ -65,19 +70,21 @@ import org.apache.lucene.util.packed.PackedInts;
*/
class Lucene42DocValuesProducer extends DocValuesProducer {
// metadata maps (just file pointers and minimal stuff)
private final Map<Integer,NumericEntry> numerics;
private final Map<Integer,BinaryEntry> binaries;
private final Map<Integer,FSTEntry> fsts;
private final Map<String,NumericEntry> numerics;
private final Map<String,BinaryEntry> binaries;
private final Map<String,FSTEntry> fsts;
private final IndexInput data;
private final int version;
private final int numEntries;
// ram instances we have already loaded
private final Map<Integer,NumericDocValues> numericInstances =
new HashMap<>();
private final Map<Integer,BinaryDocValues> binaryInstances =
new HashMap<>();
private final Map<Integer,FST<Long>> fstInstances =
new HashMap<>();
private final Map<String,NumericDocValues> numericInstances = new HashMap<>();
private final Map<String,BinaryDocValues> binaryInstances = new HashMap<>();
private final Map<String,FST<Long>> fstInstances = new HashMap<>();
private final Map<String,Accountable> numericInfo = new HashMap<>();
private final Map<String,Accountable> binaryInfo = new HashMap<>();
private final Map<String,Accountable> addressInfo = new HashMap<>();
private final int maxDoc;
private final AtomicLong ramBytesUsed;
@ -112,7 +119,7 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
numerics = new HashMap<>();
binaries = new HashMap<>();
fsts = new HashMap<>();
readFields(in, state.fieldInfos);
numEntries = readFields(in, state.fieldInfos);
if (version >= VERSION_CHECKSUM) {
CodecUtil.checkFooter(in);
@ -156,10 +163,13 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
}
}
private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
private int readFields(IndexInput meta, FieldInfos infos) throws IOException {
int numEntries = 0;
int fieldNumber = meta.readVInt();
while (fieldNumber != -1) {
if (infos.fieldInfo(fieldNumber) == null) {
numEntries++;
FieldInfo info = infos.fieldInfo(fieldNumber);
if (info == null) {
// trickier to validate more: because we re-use for norms, because we use multiple entries
// for "composite" types like sortedset, etc.
throw new CorruptIndexException("Invalid field number: " + fieldNumber + " (resource=" + meta + ")");
@ -181,7 +191,7 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
if (entry.format != UNCOMPRESSED) {
entry.packedIntsVersion = meta.readVInt();
}
numerics.put(fieldNumber, entry);
numerics.put(info.name, entry);
} else if (fieldType == BYTES) {
BinaryEntry entry = new BinaryEntry();
entry.offset = meta.readLong();
@ -192,25 +202,26 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
entry.packedIntsVersion = meta.readVInt();
entry.blockSize = meta.readVInt();
}
binaries.put(fieldNumber, entry);
binaries.put(info.name, entry);
} else if (fieldType == FST) {
FSTEntry entry = new FSTEntry();
entry.offset = meta.readLong();
entry.numOrds = meta.readVLong();
fsts.put(fieldNumber, entry);
fsts.put(info.name, entry);
} else {
throw new CorruptIndexException("invalid entry type: " + fieldType + ", input=" + meta);
}
fieldNumber = meta.readVInt();
}
return numEntries;
}
@Override
public synchronized NumericDocValues getNumeric(FieldInfo field) throws IOException {
NumericDocValues instance = numericInstances.get(field.number);
NumericDocValues instance = numericInstances.get(field.name);
if (instance == null) {
instance = loadNumeric(field);
numericInstances.put(field.number, instance);
numericInstances.put(field.name, instance);
}
return instance;
}
@ -220,6 +231,16 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
return ramBytesUsed.get();
}
@Override
public synchronized Iterable<? extends Accountable> getChildResources() {
List<Accountable> resources = new ArrayList<>();
resources.addAll(Accountables.namedAccountables("numeric field", numericInfo));
resources.addAll(Accountables.namedAccountables("binary field", binaryInfo));
resources.addAll(Accountables.namedAccountables("addresses field", addressInfo));
resources.addAll(Accountables.namedAccountables("terms dict field", fstInstances));
return Collections.unmodifiableList(resources);
}
@Override
public void checkIntegrity() throws IOException {
if (version >= VERSION_CHECKSUM) {
@ -227,8 +248,13 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
}
}
@Override
public String toString() {
return getClass().getSimpleName() + "(entries=" + numEntries + ")";
}
private NumericDocValues loadNumeric(FieldInfo field) throws IOException {
NumericEntry entry = numerics.get(field.number);
NumericEntry entry = numerics.get(field.name);
data.seek(entry.offset);
switch (entry.format) {
case TABLE_COMPRESSED:
@ -244,6 +270,7 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
final int bitsPerValue = data.readVInt();
final PackedInts.Reader ordsReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatID), entry.packedIntsVersion, maxDoc, bitsPerValue);
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(decode) + ordsReader.ramBytesUsed());
numericInfo.put(field.name, ordsReader);
return new NumericDocValues() {
@Override
public long get(int docID) {
@ -254,11 +281,13 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
final int blockSize = data.readVInt();
final BlockPackedReader reader = new BlockPackedReader(data, entry.packedIntsVersion, blockSize, maxDoc, false);
ramBytesUsed.addAndGet(reader.ramBytesUsed());
numericInfo.put(field.name, reader);
return reader;
case UNCOMPRESSED:
final byte bytes[] = new byte[maxDoc];
data.readBytes(bytes, 0, bytes.length);
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(bytes));
numericInfo.put(field.name, Accountables.namedAccountable("byte array", maxDoc));
return new NumericDocValues() {
@Override
public long get(int docID) {
@ -271,6 +300,7 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
final int quotientBlockSize = data.readVInt();
final BlockPackedReader quotientReader = new BlockPackedReader(data, entry.packedIntsVersion, quotientBlockSize, maxDoc, false);
ramBytesUsed.addAndGet(quotientReader.ramBytesUsed());
numericInfo.put(field.name, quotientReader);
return new NumericDocValues() {
@Override
public long get(int docID) {
@ -284,20 +314,21 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
@Override
public synchronized BinaryDocValues getBinary(FieldInfo field) throws IOException {
BinaryDocValues instance = binaryInstances.get(field.number);
BinaryDocValues instance = binaryInstances.get(field.name);
if (instance == null) {
instance = loadBinary(field);
binaryInstances.put(field.number, instance);
binaryInstances.put(field.name, instance);
}
return instance;
}
private BinaryDocValues loadBinary(FieldInfo field) throws IOException {
BinaryEntry entry = binaries.get(field.number);
BinaryEntry entry = binaries.get(field.name);
data.seek(entry.offset);
PagedBytes bytes = new PagedBytes(16);
bytes.copy(data, entry.numBytes);
final PagedBytes.Reader bytesReader = bytes.freeze(true);
binaryInfo.put(field.name, bytesReader);
if (entry.minLength == entry.maxLength) {
final int fixedLength = entry.minLength;
ramBytesUsed.addAndGet(bytesReader.ramBytesUsed());
@ -311,6 +342,7 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
};
} else {
final MonotonicBlockPackedReader addresses = MonotonicBlockPackedReader.of(data, entry.packedIntsVersion, entry.blockSize, maxDoc, false);
addressInfo.put(field.name, addresses);
ramBytesUsed.addAndGet(bytesReader.ramBytesUsed() + addresses.ramBytesUsed());
return new BinaryDocValues() {
@ -328,15 +360,15 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
@Override
public SortedDocValues getSorted(FieldInfo field) throws IOException {
final FSTEntry entry = fsts.get(field.number);
final FSTEntry entry = fsts.get(field.name);
FST<Long> instance;
synchronized(this) {
instance = fstInstances.get(field.number);
instance = fstInstances.get(field.name);
if (instance == null) {
data.seek(entry.offset);
instance = new FST<>(data, PositiveIntOutputs.getSingleton());
ramBytesUsed.addAndGet(instance.ramBytesUsed());
fstInstances.put(field.number, instance);
fstInstances.put(field.name, instance);
}
}
final NumericDocValues docToOrd = getNumeric(field);
@ -402,18 +434,18 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
@Override
public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
final FSTEntry entry = fsts.get(field.number);
final FSTEntry entry = fsts.get(field.name);
if (entry.numOrds == 0) {
return DocValues.emptySortedSet(); // empty FST!
}
FST<Long> instance;
synchronized(this) {
instance = fstInstances.get(field.number);
instance = fstInstances.get(field.name);
if (instance == null) {
data.seek(entry.offset);
instance = new FST<>(data, PositiveIntOutputs.getSingleton());
ramBytesUsed.addAndGet(instance.ramBytesUsed());
fstInstances.put(field.number, instance);
fstInstances.put(field.name, instance);
}
}
final BinaryDocValues docToOrds = getBinary(field);

View File

@ -18,11 +18,13 @@ package org.apache.lucene.codecs.lucene42;
*/
import java.io.IOException;
import java.util.Collections;
import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.util.Accountable;
/**
* Reads 4.2-4.8 norms.
@ -51,9 +53,19 @@ class Lucene42NormsProducer extends NormsProducer {
public long ramBytesUsed() {
return impl.ramBytesUsed();
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return impl.getChildResources();
}
@Override
public void close() throws IOException {
impl.close();
}
@Override
public String toString() {
return getClass().getSimpleName() + "(" + impl + ")";
}
}

View File

@ -29,7 +29,10 @@ import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesFormat.VERSION_
import java.io.Closeable; // javadocs
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicLong;
@ -53,6 +56,8 @@ import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.index.TermsEnum.SeekStatus;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
@ -75,6 +80,7 @@ class Lucene45DocValuesProducer extends DocValuesProducer implements Closeable {
private final IndexInput data;
private final int maxDoc;
private final int version;
private final int numFields;
// We need this for pre-4.9 indexes which recorded multiple fields' DocValues
// updates under the same generation, and therefore the passed FieldInfos may
@ -107,7 +113,7 @@ class Lucene45DocValuesProducer extends DocValuesProducer implements Closeable {
ordIndexes = new HashMap<>();
binaries = new HashMap<>();
sortedSets = new HashMap<>();
readFields(in, state.fieldInfos);
numFields = readFields(in, state.fieldInfos);
if (version >= Lucene45DocValuesFormat.VERSION_CHECKSUM) {
CodecUtil.checkFooter(in);
@ -204,9 +210,11 @@ class Lucene45DocValuesProducer extends DocValuesProducer implements Closeable {
ordIndexes.put(fieldNumber, n2);
}
private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
private int readFields(IndexInput meta, FieldInfos infos) throws IOException {
int numFields = 0;
int fieldNumber = meta.readVInt();
while (fieldNumber != -1) {
numFields++;
if ((lenientFieldInfoCheck && fieldNumber < 0) || (!lenientFieldInfoCheck && infos.fieldInfo(fieldNumber) == null)) {
// trickier to validate more: because we re-use for norms, because we use multiple entries
// for "composite" types like sortedset, etc.
@ -241,6 +249,7 @@ class Lucene45DocValuesProducer extends DocValuesProducer implements Closeable {
}
fieldNumber = meta.readVInt();
}
return numFields;
}
static NumericEntry readNumericEntry(IndexInput meta) throws IOException {
@ -329,12 +338,25 @@ class Lucene45DocValuesProducer extends DocValuesProducer implements Closeable {
return ramBytesUsed.get();
}
@Override
public synchronized Iterable<? extends Accountable> getChildResources() {
List<Accountable> resources = new ArrayList<>();
resources.addAll(Accountables.namedAccountables("addresses field number", addressInstances));
resources.addAll(Accountables.namedAccountables("ord index field number", ordIndexInstances));
return Collections.unmodifiableList(resources);
}
@Override
public void checkIntegrity() throws IOException {
if (version >= Lucene45DocValuesFormat.VERSION_CHECKSUM) {
CodecUtil.checksumEntireFile(data);
}
}
@Override
public String toString() {
return getClass().getSimpleName() + "(fields=" + numFields + ")";
}
LongValues getNumeric(NumericEntry entry) throws IOException {
final IndexInput data = this.data.clone();
@ -411,18 +433,16 @@ class Lucene45DocValuesProducer extends DocValuesProducer implements Closeable {
/** returns an address instance for variable-length binary values.
* @lucene.internal */
protected MonotonicBlockPackedReader getAddressInstance(IndexInput data, FieldInfo field, BinaryEntry bytes) throws IOException {
protected synchronized MonotonicBlockPackedReader getAddressInstance(IndexInput data, FieldInfo field, BinaryEntry bytes) throws IOException {
final MonotonicBlockPackedReader addresses;
synchronized (addressInstances) {
MonotonicBlockPackedReader addrInstance = addressInstances.get(field.number);
if (addrInstance == null) {
data.seek(bytes.addressesOffset);
addrInstance = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, bytes.count, false);
addressInstances.put(field.number, addrInstance);
ramBytesUsed.addAndGet(addrInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
}
addresses = addrInstance;
MonotonicBlockPackedReader addrInstance = addressInstances.get(field.number);
if (addrInstance == null) {
data.seek(bytes.addressesOffset);
addrInstance = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, bytes.count, false);
addressInstances.put(field.number, addrInstance);
ramBytesUsed.addAndGet(addrInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
}
addresses = addrInstance;
return addresses;
}
@ -453,25 +473,23 @@ class Lucene45DocValuesProducer extends DocValuesProducer implements Closeable {
/** returns an address instance for prefix-compressed binary values.
* @lucene.internal */
protected MonotonicBlockPackedReader getIntervalInstance(IndexInput data, FieldInfo field, BinaryEntry bytes) throws IOException {
protected synchronized MonotonicBlockPackedReader getIntervalInstance(IndexInput data, FieldInfo field, BinaryEntry bytes) throws IOException {
final MonotonicBlockPackedReader addresses;
final long interval = bytes.addressInterval;
synchronized (addressInstances) {
MonotonicBlockPackedReader addrInstance = addressInstances.get(field.number);
if (addrInstance == null) {
data.seek(bytes.addressesOffset);
final long size;
if (bytes.count % interval == 0) {
size = bytes.count / interval;
} else {
size = 1L + bytes.count / interval;
}
addrInstance = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
addressInstances.put(field.number, addrInstance);
ramBytesUsed.addAndGet(addrInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
MonotonicBlockPackedReader addrInstance = addressInstances.get(field.number);
if (addrInstance == null) {
data.seek(bytes.addressesOffset);
final long size;
if (bytes.count % interval == 0) {
size = bytes.count / interval;
} else {
size = 1L + bytes.count / interval;
}
addresses = addrInstance;
addrInstance = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
addressInstances.put(field.number, addrInstance);
ramBytesUsed.addAndGet(addrInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
}
addresses = addrInstance;
return addresses;
}
@ -532,18 +550,16 @@ class Lucene45DocValuesProducer extends DocValuesProducer implements Closeable {
/** returns an address instance for sortedset ordinal lists
* @lucene.internal */
protected MonotonicBlockPackedReader getOrdIndexInstance(IndexInput data, FieldInfo field, NumericEntry entry) throws IOException {
protected synchronized MonotonicBlockPackedReader getOrdIndexInstance(IndexInput data, FieldInfo field, NumericEntry entry) throws IOException {
final MonotonicBlockPackedReader ordIndex;
synchronized (ordIndexInstances) {
MonotonicBlockPackedReader ordIndexInstance = ordIndexInstances.get(field.number);
if (ordIndexInstance == null) {
data.seek(entry.offset);
ordIndexInstance = MonotonicBlockPackedReader.of(data, entry.packedIntsVersion, entry.blockSize, entry.count, false);
ordIndexInstances.put(field.number, ordIndexInstance);
ramBytesUsed.addAndGet(ordIndexInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
}
ordIndex = ordIndexInstance;
MonotonicBlockPackedReader ordIndexInstance = ordIndexInstances.get(field.number);
if (ordIndexInstance == null) {
data.seek(entry.offset);
ordIndexInstance = MonotonicBlockPackedReader.of(data, entry.packedIntsVersion, entry.blockSize, entry.count, false);
ordIndexInstances.put(field.number, ordIndexInstance);
ramBytesUsed.addAndGet(ordIndexInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
}
ordIndex = ordIndexInstance;
return ordIndex;
}

View File

@ -29,7 +29,10 @@ import static org.apache.lucene.codecs.lucene49.Lucene49DocValuesConsumer.TABLE_
import java.io.Closeable; // javadocs
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicLong;
@ -54,6 +57,8 @@ import org.apache.lucene.index.TermsEnum.SeekStatus;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.RandomAccessInput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
@ -64,20 +69,21 @@ import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
/** reader for {@link Lucene49DocValuesFormat} */
class Lucene49DocValuesProducer extends DocValuesProducer implements Closeable {
private final Map<Integer,NumericEntry> numerics;
private final Map<Integer,BinaryEntry> binaries;
private final Map<Integer,SortedSetEntry> sortedSets;
private final Map<Integer,SortedSetEntry> sortedNumerics;
private final Map<Integer,NumericEntry> ords;
private final Map<Integer,NumericEntry> ordIndexes;
private final Map<String,NumericEntry> numerics;
private final Map<String,BinaryEntry> binaries;
private final Map<String,SortedSetEntry> sortedSets;
private final Map<String,SortedSetEntry> sortedNumerics;
private final Map<String,NumericEntry> ords;
private final Map<String,NumericEntry> ordIndexes;
private final AtomicLong ramBytesUsed;
private final IndexInput data;
private final int numFields;
private final int maxDoc;
private final int version;
// memory-resident structures
private final Map<Integer,MonotonicBlockPackedReader> addressInstances = new HashMap<>();
private final Map<Integer,MonotonicBlockPackedReader> ordIndexInstances = new HashMap<>();
private final Map<String,MonotonicBlockPackedReader> addressInstances = new HashMap<>();
private final Map<String,MonotonicBlockPackedReader> ordIndexInstances = new HashMap<>();
/** expert: instantiates a new reader */
Lucene49DocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
@ -96,7 +102,7 @@ class Lucene49DocValuesProducer extends DocValuesProducer implements Closeable {
binaries = new HashMap<>();
sortedSets = new HashMap<>();
sortedNumerics = new HashMap<>();
readFields(in, state.fieldInfos);
numFields = readFields(in, state.fieldInfos);
CodecUtil.checkFooter(in);
success = true;
@ -135,108 +141,110 @@ class Lucene49DocValuesProducer extends DocValuesProducer implements Closeable {
ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOfInstance(getClass()));
}
private void readSortedField(int fieldNumber, IndexInput meta, FieldInfos infos) throws IOException {
private void readSortedField(FieldInfo info, IndexInput meta) throws IOException {
// sorted = binary + numeric
if (meta.readVInt() != fieldNumber) {
throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
if (meta.readVInt() != info.number) {
throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
}
if (meta.readByte() != Lucene49DocValuesFormat.BINARY) {
throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
}
BinaryEntry b = readBinaryEntry(meta);
binaries.put(fieldNumber, b);
binaries.put(info.name, b);
if (meta.readVInt() != fieldNumber) {
throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
if (meta.readVInt() != info.number) {
throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
}
if (meta.readByte() != Lucene49DocValuesFormat.NUMERIC) {
throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
}
NumericEntry n = readNumericEntry(meta);
ords.put(fieldNumber, n);
ords.put(info.name, n);
}
private void readSortedSetFieldWithAddresses(int fieldNumber, IndexInput meta, FieldInfos infos) throws IOException {
private void readSortedSetFieldWithAddresses(FieldInfo info, IndexInput meta) throws IOException {
// sortedset = binary + numeric (addresses) + ordIndex
if (meta.readVInt() != fieldNumber) {
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
if (meta.readVInt() != info.number) {
throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
}
if (meta.readByte() != Lucene49DocValuesFormat.BINARY) {
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
}
BinaryEntry b = readBinaryEntry(meta);
binaries.put(fieldNumber, b);
binaries.put(info.name, b);
if (meta.readVInt() != fieldNumber) {
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
if (meta.readVInt() != info.number) {
throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
}
if (meta.readByte() != Lucene49DocValuesFormat.NUMERIC) {
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
}
NumericEntry n1 = readNumericEntry(meta);
ords.put(fieldNumber, n1);
ords.put(info.name, n1);
if (meta.readVInt() != fieldNumber) {
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
if (meta.readVInt() != info.number) {
throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
}
if (meta.readByte() != Lucene49DocValuesFormat.NUMERIC) {
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
}
NumericEntry n2 = readNumericEntry(meta);
ordIndexes.put(fieldNumber, n2);
ordIndexes.put(info.name, n2);
}
private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
private int readFields(IndexInput meta, FieldInfos infos) throws IOException {
int numFields = 0;
int fieldNumber = meta.readVInt();
while (fieldNumber != -1) {
if (infos.fieldInfo(fieldNumber) == null) {
// trickier to validate more: because we re-use for norms, because we use multiple entries
// for "composite" types like sortedset, etc.
numFields++;
FieldInfo info = infos.fieldInfo(fieldNumber);
if (info == null) {
// trickier to validate more: because we use multiple entries for "composite" types like sortedset, etc.
throw new CorruptIndexException("Invalid field number: " + fieldNumber + " (resource=" + meta + ")");
}
byte type = meta.readByte();
if (type == Lucene49DocValuesFormat.NUMERIC) {
numerics.put(fieldNumber, readNumericEntry(meta));
numerics.put(info.name, readNumericEntry(meta));
} else if (type == Lucene49DocValuesFormat.BINARY) {
BinaryEntry b = readBinaryEntry(meta);
binaries.put(fieldNumber, b);
binaries.put(info.name, b);
} else if (type == Lucene49DocValuesFormat.SORTED) {
readSortedField(fieldNumber, meta, infos);
readSortedField(info, meta);
} else if (type == Lucene49DocValuesFormat.SORTED_SET) {
SortedSetEntry ss = readSortedSetEntry(meta);
sortedSets.put(fieldNumber, ss);
sortedSets.put(info.name, ss);
if (ss.format == SORTED_WITH_ADDRESSES) {
readSortedSetFieldWithAddresses(fieldNumber, meta, infos);
readSortedSetFieldWithAddresses(info, meta);
} else if (ss.format == SORTED_SINGLE_VALUED) {
if (meta.readVInt() != fieldNumber) {
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
}
if (meta.readByte() != Lucene49DocValuesFormat.SORTED) {
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
}
readSortedField(fieldNumber, meta, infos);
readSortedField(info, meta);
} else {
throw new AssertionError();
}
} else if (type == Lucene49DocValuesFormat.SORTED_NUMERIC) {
SortedSetEntry ss = readSortedSetEntry(meta);
sortedNumerics.put(fieldNumber, ss);
sortedNumerics.put(info.name, ss);
if (meta.readVInt() != fieldNumber) {
throw new CorruptIndexException("sortednumeric entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
}
if (meta.readByte() != Lucene49DocValuesFormat.NUMERIC) {
throw new CorruptIndexException("sortednumeric entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
}
numerics.put(fieldNumber, readNumericEntry(meta));
numerics.put(info.name, readNumericEntry(meta));
if (ss.format == SORTED_WITH_ADDRESSES) {
if (meta.readVInt() != fieldNumber) {
throw new CorruptIndexException("sortednumeric entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
}
if (meta.readByte() != Lucene49DocValuesFormat.NUMERIC) {
throw new CorruptIndexException("sortednumeric entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
}
NumericEntry ordIndex = readNumericEntry(meta);
ordIndexes.put(fieldNumber, ordIndex);
ordIndexes.put(info.name, ordIndex);
} else if (ss.format != SORTED_SINGLE_VALUED) {
throw new AssertionError();
}
@ -245,6 +253,7 @@ class Lucene49DocValuesProducer extends DocValuesProducer implements Closeable {
}
fieldNumber = meta.readVInt();
}
return numFields;
}
static NumericEntry readNumericEntry(IndexInput meta) throws IOException {
@ -324,7 +333,7 @@ class Lucene49DocValuesProducer extends DocValuesProducer implements Closeable {
@Override
public NumericDocValues getNumeric(FieldInfo field) throws IOException {
NumericEntry entry = numerics.get(field.number);
NumericEntry entry = numerics.get(field.name);
return getNumeric(entry);
}
@ -333,10 +342,23 @@ class Lucene49DocValuesProducer extends DocValuesProducer implements Closeable {
return ramBytesUsed.get();
}
@Override
public synchronized Iterable<? extends Accountable> getChildResources() {
List<Accountable> resources = new ArrayList<>();
resources.addAll(Accountables.namedAccountables("addresses field", addressInstances));
resources.addAll(Accountables.namedAccountables("ord index field", ordIndexInstances));
return Collections.unmodifiableList(resources);
}
@Override
public void checkIntegrity() throws IOException {
CodecUtil.checksumEntireFile(data);
}
@Override
public String toString() {
return getClass().getSimpleName() + "(fields=" + numFields + ")";
}
LongValues getNumeric(NumericEntry entry) throws IOException {
RandomAccessInput slice = this.data.randomAccessSlice(entry.offset, entry.endOffset - entry.offset);
@ -376,7 +398,7 @@ class Lucene49DocValuesProducer extends DocValuesProducer implements Closeable {
@Override
public BinaryDocValues getBinary(FieldInfo field) throws IOException {
BinaryEntry bytes = binaries.get(field.number);
BinaryEntry bytes = binaries.get(field.name);
switch(bytes.format) {
case BINARY_FIXED_UNCOMPRESSED:
return getFixedBinary(field, bytes);
@ -415,18 +437,16 @@ class Lucene49DocValuesProducer extends DocValuesProducer implements Closeable {
}
/** returns an address instance for variable-length binary values. */
private MonotonicBlockPackedReader getAddressInstance(IndexInput data, FieldInfo field, BinaryEntry bytes) throws IOException {
private synchronized MonotonicBlockPackedReader getAddressInstance(IndexInput data, FieldInfo field, BinaryEntry bytes) throws IOException {
final MonotonicBlockPackedReader addresses;
synchronized (addressInstances) {
MonotonicBlockPackedReader addrInstance = addressInstances.get(field.number);
if (addrInstance == null) {
data.seek(bytes.addressesOffset);
addrInstance = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, bytes.count+1, false);
addressInstances.put(field.number, addrInstance);
ramBytesUsed.addAndGet(addrInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
}
addresses = addrInstance;
MonotonicBlockPackedReader addrInstance = addressInstances.get(field.name);
if (addrInstance == null) {
data.seek(bytes.addressesOffset);
addrInstance = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, bytes.count+1, false);
addressInstances.put(field.name, addrInstance);
ramBytesUsed.addAndGet(addrInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
}
addresses = addrInstance;
return addresses;
}
@ -456,25 +476,23 @@ class Lucene49DocValuesProducer extends DocValuesProducer implements Closeable {
}
/** returns an address instance for prefix-compressed binary values. */
private MonotonicBlockPackedReader getIntervalInstance(IndexInput data, FieldInfo field, BinaryEntry bytes) throws IOException {
private synchronized MonotonicBlockPackedReader getIntervalInstance(IndexInput data, FieldInfo field, BinaryEntry bytes) throws IOException {
final MonotonicBlockPackedReader addresses;
final long interval = bytes.addressInterval;
synchronized (addressInstances) {
MonotonicBlockPackedReader addrInstance = addressInstances.get(field.number);
if (addrInstance == null) {
data.seek(bytes.addressesOffset);
final long size;
if (bytes.count % interval == 0) {
size = bytes.count / interval;
} else {
size = 1L + bytes.count / interval;
}
addrInstance = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
addressInstances.put(field.number, addrInstance);
ramBytesUsed.addAndGet(addrInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
MonotonicBlockPackedReader addrInstance = addressInstances.get(field.name);
if (addrInstance == null) {
data.seek(bytes.addressesOffset);
final long size;
if (bytes.count % interval == 0) {
size = bytes.count / interval;
} else {
size = 1L + bytes.count / interval;
}
addresses = addrInstance;
addrInstance = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
addressInstances.put(field.name, addrInstance);
ramBytesUsed.addAndGet(addrInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
}
addresses = addrInstance;
return addresses;
}
@ -489,9 +507,9 @@ class Lucene49DocValuesProducer extends DocValuesProducer implements Closeable {
@Override
public SortedDocValues getSorted(FieldInfo field) throws IOException {
final int valueCount = (int) binaries.get(field.number).count;
final int valueCount = (int) binaries.get(field.name).count;
final BinaryDocValues binary = getBinary(field);
NumericEntry entry = ords.get(field.number);
NumericEntry entry = ords.get(field.name);
final LongValues ordinals = getNumeric(entry);
return new SortedDocValues() {
@ -532,32 +550,30 @@ class Lucene49DocValuesProducer extends DocValuesProducer implements Closeable {
}
/** returns an address instance for sortedset ordinal lists */
private MonotonicBlockPackedReader getOrdIndexInstance(IndexInput data, FieldInfo field, NumericEntry entry) throws IOException {
private synchronized MonotonicBlockPackedReader getOrdIndexInstance(IndexInput data, FieldInfo field, NumericEntry entry) throws IOException {
final MonotonicBlockPackedReader ordIndex;
synchronized (ordIndexInstances) {
MonotonicBlockPackedReader ordIndexInstance = ordIndexInstances.get(field.number);
if (ordIndexInstance == null) {
data.seek(entry.offset);
ordIndexInstance = MonotonicBlockPackedReader.of(data, entry.packedIntsVersion, entry.blockSize, entry.count+1, false);
ordIndexInstances.put(field.number, ordIndexInstance);
ramBytesUsed.addAndGet(ordIndexInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
}
ordIndex = ordIndexInstance;
MonotonicBlockPackedReader ordIndexInstance = ordIndexInstances.get(field.name);
if (ordIndexInstance == null) {
data.seek(entry.offset);
ordIndexInstance = MonotonicBlockPackedReader.of(data, entry.packedIntsVersion, entry.blockSize, entry.count+1, false);
ordIndexInstances.put(field.name, ordIndexInstance);
ramBytesUsed.addAndGet(ordIndexInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
}
ordIndex = ordIndexInstance;
return ordIndex;
}
@Override
public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException {
SortedSetEntry ss = sortedNumerics.get(field.number);
NumericEntry numericEntry = numerics.get(field.number);
SortedSetEntry ss = sortedNumerics.get(field.name);
NumericEntry numericEntry = numerics.get(field.name);
final LongValues values = getNumeric(numericEntry);
if (ss.format == SORTED_SINGLE_VALUED) {
final Bits docsWithField = getMissingBits(numericEntry.missingOffset);
return DocValues.singleton(values, docsWithField);
} else if (ss.format == SORTED_WITH_ADDRESSES) {
final IndexInput data = this.data.clone();
final MonotonicBlockPackedReader ordIndex = getOrdIndexInstance(data, field, ordIndexes.get(field.number));
final MonotonicBlockPackedReader ordIndex = getOrdIndexInstance(data, field, ordIndexes.get(field.name));
return new SortedNumericDocValues() {
long startOffset;
@ -586,7 +602,7 @@ class Lucene49DocValuesProducer extends DocValuesProducer implements Closeable {
@Override
public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
SortedSetEntry ss = sortedSets.get(field.number);
SortedSetEntry ss = sortedSets.get(field.name);
if (ss.format == SORTED_SINGLE_VALUED) {
final SortedDocValues values = getSorted(field);
return DocValues.singleton(values);
@ -595,12 +611,12 @@ class Lucene49DocValuesProducer extends DocValuesProducer implements Closeable {
}
final IndexInput data = this.data.clone();
final long valueCount = binaries.get(field.number).count;
final long valueCount = binaries.get(field.name).count;
// we keep the byte[]s and list of ords on disk, these could be large
final LongBinaryDocValues binary = (LongBinaryDocValues) getBinary(field);
final LongValues ordinals = getNumeric(ords.get(field.number));
final LongValues ordinals = getNumeric(ords.get(field.name));
// but the addresses to the ord stream are in RAM
final MonotonicBlockPackedReader ordIndex = getOrdIndexInstance(data, field, ordIndexes.get(field.number));
final MonotonicBlockPackedReader ordIndex = getOrdIndexInstance(data, field, ordIndexes.get(field.name));
return new RandomAccessOrds() {
long startOffset;
@ -698,10 +714,10 @@ class Lucene49DocValuesProducer extends DocValuesProducer implements Closeable {
case SORTED:
return DocValues.docsWithValue(getSorted(field), maxDoc);
case BINARY:
BinaryEntry be = binaries.get(field.number);
BinaryEntry be = binaries.get(field.name);
return getMissingBits(be.missingOffset);
case NUMERIC:
NumericEntry ne = numerics.get(field.number);
NumericEntry ne = numerics.get(field.name);
return getMissingBits(ne.missingOffset);
default:
throw new AssertionError();

View File

@ -18,8 +18,10 @@ package org.apache.lucene.codecs.blockterms;
*/
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.TreeMap;
import org.apache.lucene.codecs.BlockTermState;
@ -42,6 +44,7 @@ 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.Accountables;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
@ -263,6 +266,11 @@ public class BlockTermsReader extends FieldsProducer {
public long ramBytesUsed() {
return FIELD_READER_RAM_BYTES_USED;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
@Override
public TermsEnum iterator(TermsEnum reuse) throws IOException {
@ -877,6 +885,23 @@ public class BlockTermsReader extends FieldsProducer {
}
return ramBytesUsed;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
List<Accountable> resources = new ArrayList<>();
if (indexReader != null) {
resources.add(Accountables.namedAccountable("term index", indexReader));
}
if (postingsReader != null) {
resources.add(Accountables.namedAccountable("delegate", postingsReader));
}
return Collections.unmodifiableList(resources);
}
@Override
public String toString() {
return getClass().getSimpleName() + "(index=" + indexReader + ",delegate=" + postingsReader + ")";
}
@Override
public void checkIntegrity() throws IOException {

View File

@ -25,13 +25,16 @@ 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.Accountables;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.PagedBytes;
import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Comparator;
import java.util.List;
import java.io.IOException;
import org.apache.lucene.index.IndexFileNames;
@ -61,7 +64,7 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
// all fields share this single logical byte[]
private final PagedBytes.Reader termBytesReader;
final HashMap<FieldInfo,FieldIndexData> fields = new HashMap<>();
final HashMap<String,FieldIndexData> fields = new HashMap<>();
// start of the field info data
private long dirOffset;
@ -115,7 +118,7 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
throw new CorruptIndexException("invalid packedIndexStart: " + packedIndexStart + " indexStart: " + indexStart + "numIndexTerms: " + numIndexTerms + " (resource=" + in + ")");
}
final FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
FieldIndexData previous = fields.put(fieldInfo, new FieldIndexData(in, termBytes, indexStart, termsStart, packedIndexStart, packedOffsetsStart, numIndexTerms));
FieldIndexData previous = fields.put(fieldInfo.name, new FieldIndexData(in, termBytes, indexStart, termsStart, packedIndexStart, packedOffsetsStart, numIndexTerms));
if (previous != null) {
throw new CorruptIndexException("duplicate field: " + fieldInfo.name + " (resource=" + in + ")");
}
@ -270,11 +273,28 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
return ((termOffsets!=null)? termOffsets.ramBytesUsed() : 0) +
((termsDictOffsets!=null)? termsDictOffsets.ramBytesUsed() : 0);
}
@Override
public Iterable<? extends Accountable> getChildResources() {
List<Accountable> resources = new ArrayList<>();
if (termOffsets != null) {
resources.add(Accountables.namedAccountable("term lengths", termOffsets));
}
if (termsDictOffsets != null) {
resources.add(Accountables.namedAccountable("offsets", termsDictOffsets));
}
return resources;
}
@Override
public String toString() {
return "FixedGapTermIndex(indexterms=" + numIndexTerms + ")";
}
}
@Override
public FieldIndexEnum getFieldEnum(FieldInfo fieldInfo) {
return new IndexEnum(fields.get(fieldInfo));
return new IndexEnum(fields.get(fieldInfo.name));
}
@Override
@ -298,4 +318,14 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
}
return sizeInBytes;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Accountables.namedAccountables("field", fields);
}
@Override
public String toString() {
return getClass().getSimpleName() + "(fields=" + fields.size() + ",interval=" + indexInterval + ")";
}
}

View File

@ -18,6 +18,7 @@ package org.apache.lucene.codecs.blockterms;
*/
import java.io.IOException;
import java.util.Collections;
import java.util.HashMap;
import org.apache.lucene.codecs.CodecUtil;
@ -29,6 +30,7 @@ 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.Accountables;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.fst.BytesRefFSTEnum;
@ -42,7 +44,7 @@ public class VariableGapTermsIndexReader extends TermsIndexReaderBase {
private final PositiveIntOutputs fstOutputs = PositiveIntOutputs.getSingleton();
final HashMap<FieldInfo,FieldIndexData> fields = new HashMap<>();
final HashMap<String,FieldIndexData> fields = new HashMap<>();
// start of the field info data
private long dirOffset;
@ -76,7 +78,7 @@ public class VariableGapTermsIndexReader extends TermsIndexReaderBase {
final int field = in.readVInt();
final long indexStart = in.readVLong();
final FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
FieldIndexData previous = fields.put(fieldInfo, new FieldIndexData(in, fieldInfo, indexStart));
FieldIndexData previous = fields.put(fieldInfo.name, new FieldIndexData(in, fieldInfo, indexStart));
if (previous != null) {
throw new CorruptIndexException("duplicate field: " + fieldInfo.name + " (resource=" + in + ")");
}
@ -175,11 +177,25 @@ public class VariableGapTermsIndexReader extends TermsIndexReaderBase {
public long ramBytesUsed() {
return fst == null ? 0 : fst.ramBytesUsed();
}
@Override
public Iterable<? extends Accountable> getChildResources() {
if (fst == null) {
return Collections.emptyList();
} else {
return Collections.singletonList(Accountables.namedAccountable("index data", fst));
}
}
@Override
public String toString() {
return "VarGapTermIndex";
}
}
@Override
public FieldIndexEnum getFieldEnum(FieldInfo fieldInfo) {
final FieldIndexData fieldData = fields.get(fieldInfo);
final FieldIndexData fieldData = fields.get(fieldInfo.name);
if (fieldData.fst == null) {
return null;
} else {
@ -209,4 +225,14 @@ public class VariableGapTermsIndexReader extends TermsIndexReaderBase {
}
return sizeInBytes;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Accountables.namedAccountables("field", fields);
}
@Override
public String toString() {
return getClass().getSimpleName() + "(fields=" + fields.size() + ")";
}
}

View File

@ -18,8 +18,10 @@ package org.apache.lucene.codecs.blocktreeords;
*/
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.TreeMap;
import org.apache.lucene.codecs.CodecUtil;
@ -36,6 +38,8 @@ import org.apache.lucene.index.Terms;
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.Accountables;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
@ -235,11 +239,19 @@ public final class OrdsBlockTreeTermsReader extends FieldsProducer {
@Override
public long ramBytesUsed() {
long sizeInByes = ((postingsReader!=null) ? postingsReader.ramBytesUsed() : 0);
long sizeInBytes = postingsReader.ramBytesUsed();
for (OrdsFieldReader reader : fields.values()) {
sizeInByes += reader.ramBytesUsed();
sizeInBytes += reader.ramBytesUsed();
}
return sizeInByes;
return sizeInBytes;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
List<Accountable> resources = new ArrayList<>();
resources.addAll(Accountables.namedAccountables("field", fields));
resources.add(Accountables.namedAccountable("delegate", postingsReader));
return Collections.unmodifiableList(resources);
}
@Override
@ -250,4 +262,9 @@ public final class OrdsBlockTreeTermsReader extends FieldsProducer {
// postings
postingsReader.checkIntegrity();
}
@Override
public String toString() {
return getClass().getSimpleName() + "(fields=" + fields.size() + ",delegate=" + postingsReader.toString() + ")";
}
}

View File

@ -18,6 +18,7 @@ package org.apache.lucene.codecs.blocktreeords;
*/
import java.io.IOException;
import java.util.Collections;
import org.apache.lucene.codecs.blocktreeords.FSTOrdsOutputs.Output;
import org.apache.lucene.index.FieldInfo.IndexOptions;
@ -27,6 +28,7 @@ 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.Accountables;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.automaton.CompiledAutomaton;
import org.apache.lucene.util.fst.FST;
@ -170,4 +172,18 @@ final class OrdsFieldReader extends Terms implements Accountable {
public long ramBytesUsed() {
return ((index!=null)? index.ramBytesUsed() : 0);
}
@Override
public Iterable<? extends Accountable> getChildResources() {
if (index == null) {
return Collections.emptyList();
} else {
return Collections.singleton(Accountables.namedAccountable("term index", index));
}
}
@Override
public String toString() {
return "OrdsBlockTreeTerms(terms=" + numTerms + ",postings=" + sumDocFreq + ",positions=" + sumTotalTermFreq + ",docs=" + docCount + ")";
}
}

View File

@ -19,6 +19,7 @@ package org.apache.lucene.codecs.bloom;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
@ -42,6 +43,8 @@ import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
@ -408,10 +411,25 @@ public final class BloomFilteringPostingsFormat extends PostingsFormat {
return sizeInBytes;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
List<Accountable> resources = new ArrayList<>();
resources.addAll(Accountables.namedAccountables("field", bloomsByFieldName));
if (delegateFieldsProducer != null) {
resources.add(Accountables.namedAccountable("delegate", delegateFieldsProducer));
}
return Collections.unmodifiableList(resources);
}
@Override
public void checkIntegrity() throws IOException {
delegateFieldsProducer.checkIntegrity();
}
@Override
public String toString() {
return getClass().getSimpleName() + "(fields=" + bloomsByFieldName.size() + ",delegate=" + delegateFieldsProducer + ")";
}
}
class BloomFilteredFieldsConsumer extends FieldsConsumer {

View File

@ -17,6 +17,7 @@ package org.apache.lucene.codecs.bloom;
* limitations under the License.
*/
import java.io.IOException;
import java.util.Collections;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
@ -309,4 +310,14 @@ public class FuzzySet implements Accountable {
public long ramBytesUsed() {
return RamUsageEstimator.sizeOf(filter.getBits());
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
@Override
public String toString() {
return getClass().getSimpleName() + "(hash=" + hashFunction + ")";
}
}

View File

@ -98,5 +98,9 @@ public final class MurmurHash2 extends HashFunction{
public final int hash(BytesRef br) {
return hash32(br.bytes, br.offset, br.length);
}
@Override
public String toString() {
return getClass().getSimpleName();
}
}

View File

@ -18,7 +18,10 @@ package org.apache.lucene.codecs.memory;
*/
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicLong;
@ -28,6 +31,7 @@ import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.RandomAccessOrds;
@ -37,6 +41,8 @@ import org.apache.lucene.index.SortedNumericDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.FixedBitSet;
@ -49,25 +55,22 @@ import org.apache.lucene.util.RamUsageEstimator;
class DirectDocValuesProducer extends DocValuesProducer {
// metadata maps (just file pointers and minimal stuff)
private final Map<Integer,NumericEntry> numerics = new HashMap<>();
private final Map<Integer,BinaryEntry> binaries = new HashMap<>();
private final Map<Integer,SortedEntry> sorteds = new HashMap<>();
private final Map<Integer,SortedSetEntry> sortedSets = new HashMap<>();
private final Map<Integer,SortedNumericEntry> sortedNumerics = new HashMap<>();
private final Map<String,NumericEntry> numerics = new HashMap<>();
private final Map<String,BinaryEntry> binaries = new HashMap<>();
private final Map<String,SortedEntry> sorteds = new HashMap<>();
private final Map<String,SortedSetEntry> sortedSets = new HashMap<>();
private final Map<String,SortedNumericEntry> sortedNumerics = new HashMap<>();
private final IndexInput data;
// ram instances we have already loaded
private final Map<Integer,NumericDocValues> numericInstances =
new HashMap<>();
private final Map<Integer,BinaryRawValues> binaryInstances =
new HashMap<>();
private final Map<Integer,SortedRawValues> sortedInstances =
new HashMap<>();
private final Map<Integer,SortedSetRawValues> sortedSetInstances =
new HashMap<>();
private final Map<Integer,SortedNumericRawValues> sortedNumericInstances =
new HashMap<>();
private final Map<Integer,Bits> docsWithFieldInstances = new HashMap<>();
private final Map<String,NumericRawValues> numericInstances = new HashMap<>();
private final Map<String,BinaryRawValues> binaryInstances = new HashMap<>();
private final Map<String,SortedRawValues> sortedInstances = new HashMap<>();
private final Map<String,SortedSetRawValues> sortedSetInstances = new HashMap<>();
private final Map<String,SortedNumericRawValues> sortedNumericInstances = new HashMap<>();
private final Map<String,FixedBitSet> docsWithFieldInstances = new HashMap<>();
private final int numEntries;
private final int maxDoc;
private final AtomicLong ramBytesUsed;
@ -95,7 +98,7 @@ class DirectDocValuesProducer extends DocValuesProducer {
version = CodecUtil.checkHeader(in, metaCodec,
VERSION_START,
VERSION_CURRENT);
readFields(in);
numEntries = readFields(in, state.fieldInfos);
CodecUtil.checkFooter(in);
success = true;
@ -188,37 +191,41 @@ class DirectDocValuesProducer extends DocValuesProducer {
return entry;
}
private void readFields(IndexInput meta) throws IOException {
private int readFields(IndexInput meta, FieldInfos infos) throws IOException {
int numEntries = 0;
int fieldNumber = meta.readVInt();
while (fieldNumber != -1) {
numEntries++;
FieldInfo info = infos.fieldInfo(fieldNumber);
int fieldType = meta.readByte();
if (fieldType == NUMBER) {
numerics.put(fieldNumber, readNumericEntry(meta));
numerics.put(info.name, readNumericEntry(meta));
} else if (fieldType == BYTES) {
binaries.put(fieldNumber, readBinaryEntry(meta));
binaries.put(info.name, readBinaryEntry(meta));
} else if (fieldType == SORTED) {
SortedEntry entry = readSortedEntry(meta);
sorteds.put(fieldNumber, entry);
binaries.put(fieldNumber, entry.values);
sorteds.put(info.name, entry);
binaries.put(info.name, entry.values);
} else if (fieldType == SORTED_SET) {
SortedSetEntry entry = readSortedSetEntry(meta, false);
sortedSets.put(fieldNumber, entry);
binaries.put(fieldNumber, entry.values);
sortedSets.put(info.name, entry);
binaries.put(info.name, entry.values);
} else if (fieldType == SORTED_SET_SINGLETON) {
SortedSetEntry entry = readSortedSetEntry(meta, true);
sortedSets.put(fieldNumber, entry);
binaries.put(fieldNumber, entry.values);
sortedSets.put(info.name, entry);
binaries.put(info.name, entry.values);
} else if (fieldType == SORTED_NUMERIC) {
SortedNumericEntry entry = readSortedNumericEntry(meta, false);
sortedNumerics.put(fieldNumber, entry);
sortedNumerics.put(info.name, entry);
} else if (fieldType == SORTED_NUMERIC_SINGLETON) {
SortedNumericEntry entry = readSortedNumericEntry(meta, true);
sortedNumerics.put(fieldNumber, entry);
sortedNumerics.put(info.name, entry);
} else {
throw new CorruptIndexException("invalid entry type: " + fieldType + ", input=" + meta);
throw new CorruptIndexException("invalid entry type: " + fieldType + ", field= " + info.name + ", input=" + meta);
}
fieldNumber = meta.readVInt();
}
return numEntries;
}
@Override
@ -226,6 +233,23 @@ class DirectDocValuesProducer extends DocValuesProducer {
return ramBytesUsed.get();
}
@Override
public synchronized Iterable<? extends Accountable> getChildResources() {
List<Accountable> resources = new ArrayList<>();
resources.addAll(Accountables.namedAccountables("numeric field", numericInstances));
resources.addAll(Accountables.namedAccountables("binary field", binaryInstances));
resources.addAll(Accountables.namedAccountables("sorted field", sortedInstances));
resources.addAll(Accountables.namedAccountables("sorted set field", sortedSetInstances));
resources.addAll(Accountables.namedAccountables("sorted numeric field", sortedNumericInstances));
resources.addAll(Accountables.namedAccountables("missing bitset field", docsWithFieldInstances));
return Collections.unmodifiableList(resources);
}
@Override
public String toString() {
return getClass().getSimpleName() + "(entries=" + numEntries + ")";
}
@Override
public void checkIntegrity() throws IOException {
CodecUtil.checksumEntireFile(data);
@ -233,29 +257,32 @@ class DirectDocValuesProducer extends DocValuesProducer {
@Override
public synchronized NumericDocValues getNumeric(FieldInfo field) throws IOException {
NumericDocValues instance = numericInstances.get(field.number);
NumericRawValues instance = numericInstances.get(field.name);
if (instance == null) {
// Lazy load
instance = loadNumeric(numerics.get(field.number));
numericInstances.put(field.number, instance);
instance = loadNumeric(numerics.get(field.name));
numericInstances.put(field.name, instance);
}
return instance;
return instance.numerics;
}
private NumericDocValues loadNumeric(NumericEntry entry) throws IOException {
private NumericRawValues loadNumeric(NumericEntry entry) throws IOException {
NumericRawValues ret = new NumericRawValues();
data.seek(entry.offset + entry.missingBytes);
switch (entry.byteWidth) {
case 1:
{
final byte[] values = new byte[entry.count];
data.readBytes(values, 0, entry.count);
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(values));
return new NumericDocValues() {
ret.bytesUsed = RamUsageEstimator.sizeOf(values);
ramBytesUsed.addAndGet(ret.bytesUsed);
ret.numerics = new NumericDocValues() {
@Override
public long get(int idx) {
return values[idx];
}
};
return ret;
}
case 2:
@ -264,13 +291,15 @@ class DirectDocValuesProducer extends DocValuesProducer {
for(int i=0;i<entry.count;i++) {
values[i] = data.readShort();
}
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(values));
return new NumericDocValues() {
ret.bytesUsed = RamUsageEstimator.sizeOf(values);
ramBytesUsed.addAndGet(ret.bytesUsed);
ret.numerics = new NumericDocValues() {
@Override
public long get(int idx) {
return values[idx];
}
};
return ret;
}
case 4:
@ -279,13 +308,15 @@ class DirectDocValuesProducer extends DocValuesProducer {
for(int i=0;i<entry.count;i++) {
values[i] = data.readInt();
}
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(values));
return new NumericDocValues() {
ret.bytesUsed = RamUsageEstimator.sizeOf(values);
ramBytesUsed.addAndGet(ret.bytesUsed);
ret.numerics = new NumericDocValues() {
@Override
public long get(int idx) {
return values[idx];
}
};
return ret;
}
case 8:
@ -294,13 +325,15 @@ class DirectDocValuesProducer extends DocValuesProducer {
for(int i=0;i<entry.count;i++) {
values[i] = data.readLong();
}
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(values));
return new NumericDocValues() {
ret.bytesUsed = RamUsageEstimator.sizeOf(values);
ramBytesUsed.addAndGet(ret.bytesUsed);
ret.numerics = new NumericDocValues() {
@Override
public long get(int idx) {
return values[idx];
}
};
return ret;
}
default:
@ -310,11 +343,11 @@ class DirectDocValuesProducer extends DocValuesProducer {
@Override
public synchronized BinaryDocValues getBinary(FieldInfo field) throws IOException {
BinaryRawValues instance = binaryInstances.get(field.number);
BinaryRawValues instance = binaryInstances.get(field.name);
if (instance == null) {
// Lazy load
instance = loadBinary(binaries.get(field.number));
binaryInstances.put(field.number, instance);
instance = loadBinary(binaries.get(field.name));
binaryInstances.put(field.name, instance);
}
final byte[] bytes = instance.bytes;
final int[] address = instance.address;
@ -354,17 +387,17 @@ class DirectDocValuesProducer extends DocValuesProducer {
@Override
public SortedDocValues getSorted(FieldInfo field) throws IOException {
final SortedEntry entry = sorteds.get(field.number);
final SortedEntry entry = sorteds.get(field.name);
SortedRawValues instance;
synchronized (this) {
instance = sortedInstances.get(field.number);
instance = sortedInstances.get(field.name);
if (instance == null) {
// Lazy load
instance = loadSorted(field);
sortedInstances.put(field.number, instance);
sortedInstances.put(field.name, instance);
}
}
return newSortedInstance(instance.docToOrd, getBinary(field), entry.values.count);
return newSortedInstance(instance.docToOrd.numerics, getBinary(field), entry.values.count);
}
private SortedDocValues newSortedInstance(final NumericDocValues docToOrd, final BinaryDocValues values, final int count) {
@ -392,8 +425,8 @@ class DirectDocValuesProducer extends DocValuesProducer {
}
private SortedRawValues loadSorted(FieldInfo field) throws IOException {
final SortedEntry entry = sorteds.get(field.number);
final NumericDocValues docToOrd = loadNumeric(entry.docToOrd);
final SortedEntry entry = sorteds.get(field.name);
final NumericRawValues docToOrd = loadNumeric(entry.docToOrd);
final SortedRawValues values = new SortedRawValues();
values.docToOrd = docToOrd;
return values;
@ -401,21 +434,21 @@ class DirectDocValuesProducer extends DocValuesProducer {
@Override
public synchronized SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException {
SortedNumericRawValues instance = sortedNumericInstances.get(field.number);
final SortedNumericEntry entry = sortedNumerics.get(field.number);
SortedNumericRawValues instance = sortedNumericInstances.get(field.name);
final SortedNumericEntry entry = sortedNumerics.get(field.name);
if (instance == null) {
// Lazy load
instance = loadSortedNumeric(entry);
sortedNumericInstances.put(field.number, instance);
sortedNumericInstances.put(field.name, instance);
}
if (entry.docToAddress == null) {
final NumericDocValues single = instance.values;
final Bits docsWithField = getMissingBits(field.number, entry.values.missingOffset, entry.values.missingBytes);
final NumericDocValues single = instance.values.numerics;
final Bits docsWithField = getMissingBits(field, entry.values.missingOffset, entry.values.missingBytes);
return DocValues.singleton(single, docsWithField);
} else {
final NumericDocValues docToAddress = instance.docToAddress;
final NumericDocValues values = instance.values;
final NumericDocValues docToAddress = instance.docToAddress.numerics;
final NumericDocValues values = instance.values.numerics;
return new SortedNumericDocValues() {
int valueStart;
@ -451,20 +484,20 @@ class DirectDocValuesProducer extends DocValuesProducer {
@Override
public synchronized SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
SortedSetRawValues instance = sortedSetInstances.get(field.number);
final SortedSetEntry entry = sortedSets.get(field.number);
SortedSetRawValues instance = sortedSetInstances.get(field.name);
final SortedSetEntry entry = sortedSets.get(field.name);
if (instance == null) {
// Lazy load
instance = loadSortedSet(entry);
sortedSetInstances.put(field.number, instance);
sortedSetInstances.put(field.name, instance);
}
if (instance.docToOrdAddress == null) {
SortedDocValues sorted = newSortedInstance(instance.ords, getBinary(field), entry.values.count);
SortedDocValues sorted = newSortedInstance(instance.ords.numerics, getBinary(field), entry.values.count);
return DocValues.singleton(sorted);
} else {
final NumericDocValues docToOrdAddress = instance.docToOrdAddress;
final NumericDocValues ords = instance.ords;
final NumericDocValues docToOrdAddress = instance.docToOrdAddress.numerics;
final NumericDocValues ords = instance.ords.numerics;
final BinaryDocValues values = getBinary(field);
// Must make a new instance since the iterator has state:
@ -524,13 +557,13 @@ class DirectDocValuesProducer extends DocValuesProducer {
return instance;
}
private Bits getMissingBits(int fieldNumber, final long offset, final long length) throws IOException {
private Bits getMissingBits(FieldInfo field, final long offset, final long length) throws IOException {
if (offset == -1) {
return new Bits.MatchAllBits(maxDoc);
} else {
Bits instance;
FixedBitSet instance;
synchronized(this) {
instance = docsWithFieldInstances.get(fieldNumber);
instance = docsWithFieldInstances.get(field.name);
if (instance == null) {
IndexInput data = this.data.clone();
data.seek(offset);
@ -540,7 +573,7 @@ class DirectDocValuesProducer extends DocValuesProducer {
bits[i] = data.readLong();
}
instance = new FixedBitSet(bits, maxDoc);
docsWithFieldInstances.put(fieldNumber, instance);
docsWithFieldInstances.put(field.name, instance);
}
}
return instance;
@ -557,11 +590,11 @@ class DirectDocValuesProducer extends DocValuesProducer {
case SORTED:
return DocValues.docsWithValue(getSorted(field), maxDoc);
case BINARY:
BinaryEntry be = binaries.get(field.number);
return getMissingBits(field.number, be.missingOffset, be.missingBytes);
BinaryEntry be = binaries.get(field.name);
return getMissingBits(field, be.missingOffset, be.missingBytes);
case NUMERIC:
NumericEntry ne = numerics.get(field.number);
return getMissingBits(field.number, ne.missingOffset, ne.missingBytes);
NumericEntry ne = numerics.get(field.name);
return getMissingBits(field, ne.missingOffset, ne.missingBytes);
default:
throw new AssertionError();
}
@ -572,23 +605,130 @@ class DirectDocValuesProducer extends DocValuesProducer {
data.close();
}
static class BinaryRawValues {
static class BinaryRawValues implements Accountable {
byte[] bytes;
int[] address;
}
@Override
public long ramBytesUsed() {
long bytesUsed = RamUsageEstimator.sizeOf(bytes);
if (address != null) {
bytesUsed += RamUsageEstimator.sizeOf(address);
}
return bytesUsed;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
List<Accountable> resources = new ArrayList<>();
if (address != null) {
resources.add(Accountables.namedAccountable("addresses", RamUsageEstimator.sizeOf(address)));
}
resources.add(Accountables.namedAccountable("bytes", RamUsageEstimator.sizeOf(bytes)));
return Collections.unmodifiableList(resources);
}
static class SortedRawValues {
NumericDocValues docToOrd;
@Override
public String toString() {
return getClass().getSimpleName();
}
}
static class SortedNumericRawValues {
NumericDocValues docToAddress;
NumericDocValues values;
static class NumericRawValues implements Accountable {
NumericDocValues numerics;
long bytesUsed;
@Override
public long ramBytesUsed() {
return bytesUsed;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
@Override
public String toString() {
return getClass().getSimpleName();
}
}
static class SortedSetRawValues {
NumericDocValues docToOrdAddress;
NumericDocValues ords;
static class SortedRawValues implements Accountable {
NumericRawValues docToOrd;
@Override
public long ramBytesUsed() {
return docToOrd.ramBytesUsed();
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return docToOrd.getChildResources();
}
@Override
public String toString() {
return getClass().getSimpleName();
}
}
static class SortedNumericRawValues implements Accountable {
NumericRawValues docToAddress;
NumericRawValues values;
@Override
public long ramBytesUsed() {
long bytesUsed = values.ramBytesUsed();
if (docToAddress != null) {
bytesUsed += docToAddress.ramBytesUsed();
}
return bytesUsed;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
List<Accountable> resources = new ArrayList<>();
if (docToAddress != null) {
resources.add(Accountables.namedAccountable("addresses", docToAddress));
}
resources.add(Accountables.namedAccountable("values", values));
return Collections.unmodifiableList(resources);
}
@Override
public String toString() {
return getClass().getSimpleName();
}
}
static class SortedSetRawValues implements Accountable {
NumericRawValues docToOrdAddress;
NumericRawValues ords;
@Override
public long ramBytesUsed() {
long bytesUsed = ords.ramBytesUsed();
if (docToOrdAddress != null) {
bytesUsed += docToOrdAddress.ramBytesUsed();
}
return bytesUsed;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
List<Accountable> resources = new ArrayList<>();
if (docToOrdAddress != null) {
resources.add(Accountables.namedAccountable("addresses", docToOrdAddress));
}
resources.add(Accountables.namedAccountable("ordinals", ords));
return Collections.unmodifiableList(resources);
}
@Override
public String toString() {
return getClass().getSimpleName();
}
}
static class NumericEntry {

View File

@ -41,6 +41,7 @@ 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.Accountables;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
@ -159,12 +160,22 @@ public final class DirectPostingsFormat extends PostingsFormat {
}
return sizeInBytes;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Accountables.namedAccountables("field", fields);
}
@Override
public void checkIntegrity() throws IOException {
// if we read entirely into ram, we already validated.
// otherwise returned the raw postings reader
}
@Override
public String toString() {
return getClass().getSimpleName() + "(fields=" + fields.size() + ")";
}
}
private final static class DirectField extends Terms implements Accountable {
@ -197,6 +208,11 @@ public final class DirectPostingsFormat extends PostingsFormat {
((postings!=null) ? RamUsageEstimator.sizeOf(postings) : 0) +
((payloads!=null) ? RamUsageEstimator.sizeOf(payloads) : 0);
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
}
// TODO: maybe specialize into prx/no-prx/no-frq cases?
@ -245,6 +261,11 @@ public final class DirectPostingsFormat extends PostingsFormat {
return sizeInBytes;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
}
private final byte[] termBytes;
@ -522,6 +543,16 @@ public final class DirectPostingsFormat extends PostingsFormat {
return sizeInBytes;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
@Override
public String toString() {
return "DirectTerms(terms=" + terms.length + ",postings=" + sumDocFreq + ",positions=" + sumTotalTermFreq + ",docs=" + docCount + ")";
}
// Compares in unicode (UTF8) order:
int compare(int ord, BytesRef other) {

View File

@ -23,6 +23,7 @@ import java.util.ArrayList;
import java.util.BitSet;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.TreeMap;
import org.apache.lucene.index.CorruptIndexException;
@ -43,6 +44,8 @@ import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.automaton.ByteRunAutomaton;
import org.apache.lucene.util.automaton.CompiledAutomaton;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
@ -182,7 +185,7 @@ public class FSTOrdTermsReader extends FieldsProducer {
}
}
final class TermsReader extends Terms {
final class TermsReader extends Terms implements Accountable {
final FieldInfo fieldInfo;
final long numTerms;
final long sumTotalTermFreq;
@ -276,6 +279,33 @@ public class FSTOrdTermsReader extends FieldsProducer {
return new IntersectTermsEnum(compiled, startTerm);
}
@Override
public long ramBytesUsed() {
long ramBytesUsed = 0;
if (index != null) {
ramBytesUsed += index.ramBytesUsed();
ramBytesUsed += RamUsageEstimator.sizeOf(metaBytesBlock);
ramBytesUsed += RamUsageEstimator.sizeOf(metaLongsBlock);
ramBytesUsed += RamUsageEstimator.sizeOf(skipInfo);
ramBytesUsed += RamUsageEstimator.sizeOf(statsBlock);
}
return ramBytesUsed;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
if (index == null) {
return Collections.emptyList();
} else {
return Collections.singletonList(Accountables.namedAccountable("terms", index));
}
}
@Override
public String toString() {
return "FSTOrdTerms(terms=" + numTerms + ",postings=" + sumDocFreq + ",positions=" + sumTotalTermFreq + ",docs=" + docCount + ")";
}
// Only wraps common operations for PBF interact
abstract class BaseTermsEnum extends TermsEnum {
@ -834,17 +864,24 @@ public class FSTOrdTermsReader extends FieldsProducer {
public long ramBytesUsed() {
long ramBytesUsed = postingsReader.ramBytesUsed();
for (TermsReader r : fields.values()) {
if (r.index != null) {
ramBytesUsed += r.index.ramBytesUsed();
ramBytesUsed += RamUsageEstimator.sizeOf(r.metaBytesBlock);
ramBytesUsed += RamUsageEstimator.sizeOf(r.metaLongsBlock);
ramBytesUsed += RamUsageEstimator.sizeOf(r.skipInfo);
ramBytesUsed += RamUsageEstimator.sizeOf(r.statsBlock);
}
ramBytesUsed += r.ramBytesUsed();
}
return ramBytesUsed;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
List<Accountable> resources = new ArrayList<>();
resources.addAll(Accountables.namedAccountables("field", fields));
resources.add(Accountables.namedAccountable("delegate", postingsReader));
return Collections.unmodifiableList(resources);
}
@Override
public String toString() {
return getClass().getSimpleName() + "(fields=" + fields.size() + ",delegate=" + postingsReader + ")";
}
@Override
public void checkIntegrity() throws IOException {
postingsReader.checkIntegrity();

View File

@ -19,6 +19,7 @@ package org.apache.lucene.codecs.memory;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfo.IndexOptions;
@ -79,6 +80,11 @@ class FSTTermOutputs extends Outputs<FSTTermOutputs.TermData> {
return ramBytesUsed;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
// NOTE: actually, FST nodes are seldom
// identical when outputs on their arcs
// aren't NO_OUTPUTs.

View File

@ -22,6 +22,7 @@ import java.util.ArrayList;
import java.util.BitSet;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.TreeMap;
import org.apache.lucene.index.CorruptIndexException;
@ -42,6 +43,7 @@ import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.automaton.ByteRunAutomaton;
import org.apache.lucene.util.automaton.CompiledAutomaton;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
@ -190,7 +192,25 @@ public class FSTTermsReader extends FieldsProducer {
@Override
public long ramBytesUsed() {
return BASE_RAM_BYTES_USED + dict.ramBytesUsed();
long bytesUsed = BASE_RAM_BYTES_USED;
if (dict != null) {
bytesUsed += dict.ramBytesUsed();
}
return bytesUsed;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
if (dict == null) {
return Collections.emptyList();
} else {
return Collections.singletonList(Accountables.namedAccountable("terms", dict));
}
}
@Override
public String toString() {
return "FSTTerms(terms=" + numTerms + ",postings=" + sumDocFreq + ",positions=" + sumTotalTermFreq + ",docs=" + docCount + ")";
}
@Override
@ -751,6 +771,19 @@ public class FSTTermsReader extends FieldsProducer {
return ramBytesUsed;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
List<Accountable> resources = new ArrayList<>();
resources.addAll(Accountables.namedAccountables("field", fields));
resources.add(Accountables.namedAccountable("delegate", postingsReader));
return Collections.unmodifiableCollection(resources);
}
@Override
public String toString() {
return getClass().getSimpleName() + "(fields=" + fields.size() + ",delegate=" + postingsReader + ")";
}
@Override
public void checkIntegrity() throws IOException {
postingsReader.checkIntegrity();

View File

@ -18,7 +18,10 @@ package org.apache.lucene.codecs.memory;
*/
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicLong;
@ -41,6 +44,8 @@ import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
@ -67,23 +72,23 @@ import org.apache.lucene.util.packed.PackedInts;
*/
class MemoryDocValuesProducer extends DocValuesProducer {
// metadata maps (just file pointers and minimal stuff)
private final Map<Integer,NumericEntry> numerics = new HashMap<>();
private final Map<Integer,BinaryEntry> binaries = new HashMap<>();
private final Map<Integer,FSTEntry> fsts = new HashMap<>();
private final Map<Integer,SortedSetEntry> sortedSets = new HashMap<>();
private final Map<Integer,SortedNumericEntry> sortedNumerics = new HashMap<>();
private final Map<String,NumericEntry> numerics = new HashMap<>();
private final Map<String,BinaryEntry> binaries = new HashMap<>();
private final Map<String,FSTEntry> fsts = new HashMap<>();
private final Map<String,SortedSetEntry> sortedSets = new HashMap<>();
private final Map<String,SortedNumericEntry> sortedNumerics = new HashMap<>();
private final IndexInput data;
// ram instances we have already loaded
private final Map<Integer,NumericDocValues> numericInstances =
new HashMap<>();
private final Map<Integer,BytesAndAddresses> pagedBytesInstances =
new HashMap<>();
private final Map<Integer,FST<Long>> fstInstances =
new HashMap<>();
private final Map<Integer,Bits> docsWithFieldInstances = new HashMap<>();
private final Map<Integer,MonotonicBlockPackedReader> addresses = new HashMap<>();
private final Map<String,NumericDocValues> numericInstances = new HashMap<>();
private final Map<String,BytesAndAddresses> pagedBytesInstances = new HashMap<>();
private final Map<String,FST<Long>> fstInstances = new HashMap<>();
private final Map<String,FixedBitSet> docsWithFieldInstances = new HashMap<>();
private final Map<String,MonotonicBlockPackedReader> addresses = new HashMap<>();
private final Map<String,Accountable> numericInfo = new HashMap<>();
private final int numEntries;
private final int maxDoc;
private final AtomicLong ramBytesUsed;
private final int version;
@ -116,7 +121,7 @@ class MemoryDocValuesProducer extends DocValuesProducer {
version = CodecUtil.checkHeader(in, metaCodec,
VERSION_START,
VERSION_CURRENT);
readFields(in, state.fieldInfos);
numEntries = readFields(in, state.fieldInfos);
CodecUtil.checkFooter(in);
ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOfInstance(getClass()));
success = true;
@ -203,24 +208,30 @@ class MemoryDocValuesProducer extends DocValuesProducer {
return entry;
}
private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
private int readFields(IndexInput meta, FieldInfos infos) throws IOException {
int numEntries = 0;
int fieldNumber = meta.readVInt();
while (fieldNumber != -1) {
numEntries++;
FieldInfo info = infos.fieldInfo(fieldNumber);
if (info == null) {
throw new CorruptIndexException("invalid field number: " + fieldNumber + " (resource=" + meta + ")");
}
int fieldType = meta.readByte();
if (fieldType == NUMBER) {
numerics.put(fieldNumber, readNumericEntry(meta));
numerics.put(info.name, readNumericEntry(meta));
} else if (fieldType == BYTES) {
binaries.put(fieldNumber, readBinaryEntry(meta));
binaries.put(info.name, readBinaryEntry(meta));
} else if (fieldType == FST) {
fsts.put(fieldNumber,readFSTEntry(meta));
fsts.put(info.name,readFSTEntry(meta));
} else if (fieldType == SORTED_SET) {
SortedSetEntry entry = new SortedSetEntry();
entry.singleton = false;
sortedSets.put(fieldNumber, entry);
sortedSets.put(info.name, entry);
} else if (fieldType == SORTED_SET_SINGLETON) {
SortedSetEntry entry = new SortedSetEntry();
entry.singleton = true;
sortedSets.put(fieldNumber, entry);
sortedSets.put(info.name, entry);
} else if (fieldType == SORTED_NUMERIC) {
SortedNumericEntry entry = new SortedNumericEntry();
entry.singleton = false;
@ -228,24 +239,25 @@ class MemoryDocValuesProducer extends DocValuesProducer {
entry.blockSize = meta.readVInt();
entry.addressOffset = meta.readLong();
entry.valueCount = meta.readLong();
sortedNumerics.put(fieldNumber, entry);
sortedNumerics.put(info.name, entry);
} else if (fieldType == SORTED_NUMERIC_SINGLETON) {
SortedNumericEntry entry = new SortedNumericEntry();
entry.singleton = true;
sortedNumerics.put(fieldNumber, entry);
sortedNumerics.put(info.name, entry);
} else {
throw new CorruptIndexException("invalid entry type: " + fieldType + ", input=" + meta);
throw new CorruptIndexException("invalid entry type: " + fieldType + ", fieldName=" + info.name + ", input=" + meta);
}
fieldNumber = meta.readVInt();
}
return numEntries;
}
@Override
public synchronized NumericDocValues getNumeric(FieldInfo field) throws IOException {
NumericDocValues instance = numericInstances.get(field.number);
NumericDocValues instance = numericInstances.get(field.name);
if (instance == null) {
instance = loadNumeric(field);
numericInstances.put(field.number, instance);
numericInstances.put(field.name, instance);
}
return instance;
}
@ -255,13 +267,29 @@ class MemoryDocValuesProducer extends DocValuesProducer {
return ramBytesUsed.get();
}
@Override
public synchronized Iterable<? extends Accountable> getChildResources() {
List<Accountable> resources = new ArrayList<>();
resources.addAll(Accountables.namedAccountables("numeric field", numericInfo));
resources.addAll(Accountables.namedAccountables("pagedbytes field", pagedBytesInstances));
resources.addAll(Accountables.namedAccountables("term dict field", fstInstances));
resources.addAll(Accountables.namedAccountables("missing bitset field", docsWithFieldInstances));
resources.addAll(Accountables.namedAccountables("addresses field", addresses));
return Collections.unmodifiableList(resources);
}
@Override
public void checkIntegrity() throws IOException {
CodecUtil.checksumEntireFile(data);
}
@Override
public String toString() {
return getClass().getSimpleName() + "(entries=" + numEntries + ")";
}
private NumericDocValues loadNumeric(FieldInfo field) throws IOException {
NumericEntry entry = numerics.get(field.number);
NumericEntry entry = numerics.get(field.name);
data.seek(entry.offset + entry.missingBytes);
switch (entry.format) {
case TABLE_COMPRESSED:
@ -277,6 +305,7 @@ class MemoryDocValuesProducer extends DocValuesProducer {
final int bitsPerValue = data.readVInt();
final PackedInts.Reader ordsReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatID), entry.packedIntsVersion, (int)entry.count, bitsPerValue);
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(decode) + ordsReader.ramBytesUsed());
numericInfo.put(field.name, Accountables.namedAccountable("table compressed", ordsReader));
return new NumericDocValues() {
@Override
public long get(int docID) {
@ -289,6 +318,7 @@ class MemoryDocValuesProducer extends DocValuesProducer {
final int bitsPerValueDelta = data.readVInt();
final PackedInts.Reader deltaReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatIDDelta), entry.packedIntsVersion, (int)entry.count, bitsPerValueDelta);
ramBytesUsed.addAndGet(deltaReader.ramBytesUsed());
numericInfo.put(field.name, Accountables.namedAccountable("delta compressed", deltaReader));
return new NumericDocValues() {
@Override
public long get(int docID) {
@ -299,6 +329,7 @@ class MemoryDocValuesProducer extends DocValuesProducer {
final int blockSize = data.readVInt();
final BlockPackedReader reader = new BlockPackedReader(data, entry.packedIntsVersion, blockSize, entry.count, false);
ramBytesUsed.addAndGet(reader.ramBytesUsed());
numericInfo.put(field.name, Accountables.namedAccountable("block compressed", reader));
return reader;
case GCD_COMPRESSED:
final long min = data.readLong();
@ -307,6 +338,7 @@ class MemoryDocValuesProducer extends DocValuesProducer {
final int bitsPerValueGCD = data.readVInt();
final PackedInts.Reader quotientReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatIDGCD), entry.packedIntsVersion, (int)entry.count, bitsPerValueGCD);
ramBytesUsed.addAndGet(quotientReader.ramBytesUsed());
numericInfo.put(field.name, Accountables.namedAccountable("gcd compressed", quotientReader));
return new NumericDocValues() {
@Override
public long get(int docID) {
@ -320,14 +352,14 @@ class MemoryDocValuesProducer extends DocValuesProducer {
@Override
public BinaryDocValues getBinary(FieldInfo field) throws IOException {
BinaryEntry entry = binaries.get(field.number);
BinaryEntry entry = binaries.get(field.name);
BytesAndAddresses instance;
synchronized (this) {
instance = pagedBytesInstances.get(field.number);
instance = pagedBytesInstances.get(field.name);
if (instance == null) {
instance = loadBinary(field);
pagedBytesInstances.put(field.number, instance);
pagedBytesInstances.put(field.name, instance);
}
}
final PagedBytes.Reader bytesReader = instance.reader;
@ -362,7 +394,7 @@ class MemoryDocValuesProducer extends DocValuesProducer {
private BytesAndAddresses loadBinary(FieldInfo field) throws IOException {
BytesAndAddresses bytesAndAddresses = new BytesAndAddresses();
BinaryEntry entry = binaries.get(field.number);
BinaryEntry entry = binaries.get(field.name);
data.seek(entry.offset);
PagedBytes bytes = new PagedBytes(16);
bytes.copy(data, entry.numBytes);
@ -378,18 +410,18 @@ class MemoryDocValuesProducer extends DocValuesProducer {
@Override
public SortedDocValues getSorted(FieldInfo field) throws IOException {
final FSTEntry entry = fsts.get(field.number);
final FSTEntry entry = fsts.get(field.name);
if (entry.numOrds == 0) {
return DocValues.emptySorted();
}
FST<Long> instance;
synchronized(this) {
instance = fstInstances.get(field.number);
instance = fstInstances.get(field.name);
if (instance == null) {
data.seek(entry.offset);
instance = new FST<>(data, PositiveIntOutputs.getSingleton());
ramBytesUsed.addAndGet(instance.ramBytesUsed());
fstInstances.put(field.number, instance);
fstInstances.put(field.name, instance);
}
}
final NumericDocValues docToOrd = getNumeric(field);
@ -452,21 +484,21 @@ class MemoryDocValuesProducer extends DocValuesProducer {
@Override
public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException {
SortedNumericEntry entry = sortedNumerics.get(field.number);
SortedNumericEntry entry = sortedNumerics.get(field.name);
if (entry.singleton) {
NumericDocValues values = getNumeric(field);
NumericEntry ne = numerics.get(field.number);
Bits docsWithField = getMissingBits(field.number, ne.missingOffset, ne.missingBytes);
NumericEntry ne = numerics.get(field.name);
Bits docsWithField = getMissingBits(field, ne.missingOffset, ne.missingBytes);
return DocValues.singleton(values, docsWithField);
} else {
final NumericDocValues values = getNumeric(field);
final MonotonicBlockPackedReader addr;
synchronized (this) {
MonotonicBlockPackedReader res = addresses.get(field.number);
MonotonicBlockPackedReader res = addresses.get(field.name);
if (res == null) {
data.seek(entry.addressOffset);
res = MonotonicBlockPackedReader.of(data, entry.packedIntsVersion, entry.blockSize, entry.valueCount, false);
addresses.put(field.number, res);
addresses.put(field.name, res);
}
addr = res;
}
@ -520,23 +552,23 @@ class MemoryDocValuesProducer extends DocValuesProducer {
@Override
public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
SortedSetEntry sortedSetEntry = sortedSets.get(field.number);
SortedSetEntry sortedSetEntry = sortedSets.get(field.name);
if (sortedSetEntry.singleton) {
return DocValues.singleton(getSorted(field));
}
final FSTEntry entry = fsts.get(field.number);
final FSTEntry entry = fsts.get(field.name);
if (entry.numOrds == 0) {
return DocValues.emptySortedSet(); // empty FST!
}
FST<Long> instance;
synchronized(this) {
instance = fstInstances.get(field.number);
instance = fstInstances.get(field.name);
if (instance == null) {
data.seek(entry.offset);
instance = new FST<>(data, PositiveIntOutputs.getSingleton());
ramBytesUsed.addAndGet(instance.ramBytesUsed());
fstInstances.put(field.number, instance);
fstInstances.put(field.name, instance);
}
}
final BinaryDocValues docToOrds = getBinary(field);
@ -611,13 +643,13 @@ class MemoryDocValuesProducer extends DocValuesProducer {
};
}
private Bits getMissingBits(int fieldNumber, final long offset, final long length) throws IOException {
private Bits getMissingBits(FieldInfo field, final long offset, final long length) throws IOException {
if (offset == -1) {
return new Bits.MatchAllBits(maxDoc);
} else {
Bits instance;
FixedBitSet instance;
synchronized(this) {
instance = docsWithFieldInstances.get(fieldNumber);
instance = docsWithFieldInstances.get(field.name);
if (instance == null) {
IndexInput data = this.data.clone();
data.seek(offset);
@ -627,7 +659,7 @@ class MemoryDocValuesProducer extends DocValuesProducer {
bits[i] = data.readLong();
}
instance = new FixedBitSet(bits, maxDoc);
docsWithFieldInstances.put(fieldNumber, instance);
docsWithFieldInstances.put(field.name, instance);
}
}
return instance;
@ -644,11 +676,11 @@ class MemoryDocValuesProducer extends DocValuesProducer {
case SORTED:
return DocValues.docsWithValue(getSorted(field), maxDoc);
case BINARY:
BinaryEntry be = binaries.get(field.number);
return getMissingBits(field.number, be.missingOffset, be.missingBytes);
BinaryEntry be = binaries.get(field.name);
return getMissingBits(field, be.missingOffset, be.missingBytes);
case NUMERIC:
NumericEntry ne = numerics.get(field.number);
return getMissingBits(field.number, ne.missingOffset, ne.missingBytes);
NumericEntry ne = numerics.get(field.name);
return getMissingBits(field, ne.missingOffset, ne.missingBytes);
default:
throw new AssertionError();
}
@ -696,9 +728,28 @@ class MemoryDocValuesProducer extends DocValuesProducer {
long valueCount;
}
static class BytesAndAddresses {
static class BytesAndAddresses implements Accountable {
PagedBytes.Reader reader;
MonotonicBlockPackedReader addresses;
@Override
public long ramBytesUsed() {
long bytesUsed = reader.ramBytesUsed();
if (addresses != null) {
bytesUsed += addresses.ramBytesUsed();
}
return bytesUsed;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
List<Accountable> resources = new ArrayList<>();
if (addresses != null) {
resources.add(Accountables.namedAccountable("addresses", addresses));
}
resources.add(Accountables.namedAccountable("term bytes", reader));
return Collections.unmodifiableList(resources);
}
}
// exposes FSTEnum directly as a TermsEnum: avoids binary-search next()

View File

@ -47,6 +47,7 @@ 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.Accountables;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
@ -953,6 +954,20 @@ public final class MemoryPostingsFormat extends PostingsFormat {
public long ramBytesUsed() {
return ((fst!=null) ? fst.ramBytesUsed() : 0);
}
@Override
public Iterable<? extends Accountable> getChildResources() {
if (fst == null) {
return Collections.emptyList();
} else {
return Collections.singletonList(Accountables.namedAccountable("terms", fst));
}
}
@Override
public String toString() {
return "MemoryTerms(terms=" + termCount + ",postings=" + sumDocFreq + ",positions=" + sumTotalTermFreq + ",docs=" + docCount + ")";
}
}
@Override
@ -1012,6 +1027,16 @@ public final class MemoryPostingsFormat extends PostingsFormat {
return sizeInBytes;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Accountables.namedAccountables("field", fields);
}
@Override
public String toString() {
return "MemoryPostings(fields=" + fields.size() + ")";
}
@Override
public void checkIntegrity() throws IOException {}
};

View File

@ -34,6 +34,7 @@ import java.nio.charset.StandardCharsets;
import java.text.DecimalFormat;
import java.text.DecimalFormatSymbols;
import java.text.ParseException;
import java.util.Collections;
import java.util.HashMap;
import java.util.Locale;
import java.util.Map;
@ -53,6 +54,7 @@ import org.apache.lucene.index.SortedSetDocValues;
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.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
@ -508,6 +510,16 @@ class SimpleTextDocValuesReader extends DocValuesProducer {
+ fields.size() * (RamUsageEstimator.NUM_BYTES_OBJECT_REF * 2L + OneField.BASE_RAM_BYTES_USED);
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
@Override
public String toString() {
return getClass().getSimpleName() + "(fields=" + fields.size() + ")";
}
@Override
public void checkIntegrity() throws IOException {
BytesRefBuilder scratch = new BytesRefBuilder();

View File

@ -48,6 +48,7 @@ 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.Accountables;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
@ -595,6 +596,20 @@ class SimpleTextFieldsReader extends FieldsProducer {
+ RamUsageEstimator.sizeOf(scratch.bytes()) + RamUsageEstimator.sizeOf(scratchUTF16.chars());
}
@Override
public Iterable<? extends Accountable> getChildResources() {
if (fst == null) {
return Collections.emptyList();
} else {
return Collections.singletonList(Accountables.namedAccountable("term cache", fst));
}
}
@Override
public String toString() {
return getClass().getSimpleName() + "(terms=" + termCount + ",postings=" + sumDocFreq + ",positions=" + sumTotalTermFreq + ",docs=" + docCount + ")";
}
@Override
public TermsEnum iterator(TermsEnum reuse) throws IOException {
if (fst != null) {
@ -686,6 +701,16 @@ class SimpleTextFieldsReader extends FieldsProducer {
return sizeInBytes;
}
@Override
public synchronized Iterable<? extends Accountable> getChildResources() {
return Accountables.namedAccountables("field", termsCache);
}
@Override
public String toString() {
return getClass().getSimpleName() + "(fields=" + fields.size() + ")";
}
@Override
public void checkIntegrity() throws IOException {}
}

View File

@ -28,6 +28,7 @@ import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.util.Accountable;
/**
* plain-text norms format.
@ -80,10 +81,20 @@ public class SimpleTextNormsFormat extends NormsFormat {
return impl.ramBytesUsed();
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return impl.getChildResources();
}
@Override
public void checkIntegrity() throws IOException {
impl.checkIntegrity();
}
@Override
public String toString() {
return getClass().getSimpleName() + "(" + impl + ")";
}
}
/**

View File

@ -19,6 +19,7 @@ package org.apache.lucene.codecs.simpletext;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.Collections;
import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.index.FieldInfo;
@ -32,6 +33,7 @@ import org.apache.lucene.store.ChecksumIndexInput;
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.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
@ -212,6 +214,16 @@ public class SimpleTextStoredFieldsReader extends StoredFieldsReader {
+ RamUsageEstimator.sizeOf(scratch.bytes()) + RamUsageEstimator.sizeOf(scratchUTF16.chars());
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
@Override
public String toString() {
return getClass().getSimpleName();
}
@Override
public void checkIntegrity() throws IOException {}
}

View File

@ -38,6 +38,7 @@ import org.apache.lucene.store.ChecksumIndexInput;
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.ArrayUtil;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
@ -551,6 +552,16 @@ public class SimpleTextTermVectorsReader extends TermVectorsReader {
return BASE_RAM_BYTES_USED + RamUsageEstimator.sizeOf(offsets);
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
@Override
public String toString() {
return getClass().getSimpleName();
}
@Override
public void checkIntegrity() throws IOException {}
}

View File

@ -18,8 +18,10 @@ package org.apache.lucene.codecs.blocktree;
*/
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.TreeMap;
import org.apache.lucene.codecs.CodecUtil;
@ -35,6 +37,8 @@ import org.apache.lucene.index.Terms;
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.Accountables;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
@ -293,11 +297,19 @@ public final class BlockTreeTermsReader extends FieldsProducer {
@Override
public long ramBytesUsed() {
long sizeInByes = ((postingsReader!=null) ? postingsReader.ramBytesUsed() : 0);
long sizeInBytes = postingsReader.ramBytesUsed();
for(FieldReader reader : fields.values()) {
sizeInByes += reader.ramBytesUsed();
sizeInBytes += reader.ramBytesUsed();
}
return sizeInByes;
return sizeInBytes;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
List<Accountable> resources = new ArrayList<>();
resources.addAll(Accountables.namedAccountables("field", fields));
resources.add(Accountables.namedAccountable("delegate", postingsReader));
return Collections.unmodifiableList(resources);
}
@Override
@ -310,4 +322,9 @@ public final class BlockTreeTermsReader extends FieldsProducer {
postingsReader.checkIntegrity();
}
}
@Override
public String toString() {
return getClass().getSimpleName() + "(fields=" + fields.size() + ",delegate=" + postingsReader + ")";
}
}

View File

@ -18,6 +18,7 @@ package org.apache.lucene.codecs.blocktree;
*/
import java.io.IOException;
import java.util.Collections;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfo.IndexOptions;
@ -26,6 +27,7 @@ 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.Accountables;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.automaton.CompiledAutomaton;
@ -180,4 +182,18 @@ public final class FieldReader extends Terms implements Accountable {
public long ramBytesUsed() {
return BASE_RAM_BYTES_USED + ((index!=null)? index.ramBytesUsed() : 0);
}
@Override
public Iterable<? extends Accountable> getChildResources() {
if (index == null) {
return Collections.emptyList();
} else {
return Collections.singleton(Accountables.namedAccountable("term index", index));
}
}
@Override
public String toString() {
return "BlockTreeTerms(terms=" + numTerms + ",postings=" + sumDocFreq + ",positions=" + sumTotalTermFreq + ",docs=" + docCount + ")";
}
}

View File

@ -20,12 +20,16 @@ package org.apache.lucene.codecs.compressing;
import static org.apache.lucene.util.BitUtil.zigZagDecode;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
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.Accountables;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.packed.PackedInts;
@ -184,4 +188,27 @@ public final class CompressingStoredFieldsIndexReader implements Cloneable, Acco
return res;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
List<Accountable> resources = new ArrayList<>();
long docBaseDeltaBytes = RamUsageEstimator.shallowSizeOf(docBasesDeltas);
for (PackedInts.Reader r : docBasesDeltas) {
docBaseDeltaBytes += r.ramBytesUsed();
}
resources.add(Accountables.namedAccountable("doc base deltas", docBaseDeltaBytes));
long startPointerDeltaBytes = RamUsageEstimator.shallowSizeOf(startPointersDeltas);
for (PackedInts.Reader r : startPointersDeltas) {
startPointerDeltaBytes += r.ramBytesUsed();
}
resources.add(Accountables.namedAccountable("start pointer deltas", startPointerDeltaBytes));
return resources;
}
@Override
public String toString() {
return getClass().getSimpleName() + "(blocks=" + docBases.length + ")";
}
}

View File

@ -38,6 +38,7 @@ import java.io.EOFException;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.Arrays;
import java.util.Collections;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.StoredFieldsReader;
@ -55,6 +56,8 @@ import org.apache.lucene.store.DataInput;
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.Accountables;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
@ -528,6 +531,11 @@ public final class CompressingStoredFieldsReader extends StoredFieldsReader {
public long ramBytesUsed() {
return indexReader.ramBytesUsed();
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.singleton(Accountables.namedAccountable("stored field index", indexReader));
}
@Override
public void checkIntegrity() throws IOException {
@ -536,4 +544,8 @@ public final class CompressingStoredFieldsReader extends StoredFieldsReader {
}
}
@Override
public String toString() {
return getClass().getSimpleName() + "(mode=" + compressionMode + ",chunksize=" + chunkSize + ")";
}
}

View File

@ -32,6 +32,7 @@ import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.
import java.io.Closeable;
import java.io.IOException;
import java.util.Collections;
import java.util.Iterator;
import java.util.NoSuchElementException;
@ -53,6 +54,8 @@ import org.apache.lucene.store.ChecksumIndexInput;
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.Accountables;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
@ -1068,6 +1071,11 @@ public final class CompressingTermVectorsReader extends TermVectorsReader implem
return indexReader.ramBytesUsed();
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.singleton(Accountables.namedAccountable("term vector index", indexReader));
}
@Override
public void checkIntegrity() throws IOException {
if (version >= VERSION_CHECKSUM) {
@ -1075,4 +1083,8 @@ public final class CompressingTermVectorsReader extends TermVectorsReader implem
}
}
@Override
public String toString() {
return getClass().getSimpleName() + "(mode=" + compressionMode + ",chunksize=" + chunkSize + ")";
}
}

View File

@ -24,6 +24,7 @@ import static org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter.IntBlockT
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.codecs.CodecUtil;
@ -39,6 +40,7 @@ import org.apache.lucene.store.DataInput;
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.ArrayUtil;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
@ -1580,6 +1582,11 @@ public final class Lucene41PostingsReader extends PostingsReaderBase {
public long ramBytesUsed() {
return BASE_RAM_BYTES_USED;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
@Override
public void checkIntegrity() throws IOException {
@ -1595,4 +1602,9 @@ public final class Lucene41PostingsReader extends PostingsReaderBase {
}
}
}
@Override
public String toString() {
return getClass().getSimpleName() + "(positions=" + (posIn != null) + ",payloads=" + (payIn != null) +")";
}
}

View File

@ -36,7 +36,10 @@ import static org.apache.lucene.codecs.lucene410.Lucene410DocValuesConsumer.BLOC
import java.io.Closeable; // javadocs
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicLong;
@ -60,6 +63,8 @@ import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.RandomAccessInput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
@ -71,21 +76,22 @@ import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
/** reader for {@link Lucene410DocValuesFormat} */
class Lucene410DocValuesProducer extends DocValuesProducer implements Closeable {
private final Map<Integer,NumericEntry> numerics;
private final Map<Integer,BinaryEntry> binaries;
private final Map<Integer,SortedSetEntry> sortedSets;
private final Map<Integer,SortedSetEntry> sortedNumerics;
private final Map<Integer,NumericEntry> ords;
private final Map<Integer,NumericEntry> ordIndexes;
private final Map<String,NumericEntry> numerics;
private final Map<String,BinaryEntry> binaries;
private final Map<String,SortedSetEntry> sortedSets;
private final Map<String,SortedSetEntry> sortedNumerics;
private final Map<String,NumericEntry> ords;
private final Map<String,NumericEntry> ordIndexes;
private final int numFields;
private final AtomicLong ramBytesUsed;
private final IndexInput data;
private final int maxDoc;
private final int version;
// memory-resident structures
private final Map<Integer,MonotonicBlockPackedReader> addressInstances = new HashMap<>();
private final Map<Integer,MonotonicBlockPackedReader> ordIndexInstances = new HashMap<>();
private final Map<Integer,ReverseTermsIndex> reverseIndexInstances = new HashMap<>();
private final Map<String,MonotonicBlockPackedReader> addressInstances = new HashMap<>();
private final Map<String,MonotonicBlockPackedReader> ordIndexInstances = new HashMap<>();
private final Map<String,ReverseTermsIndex> reverseIndexInstances = new HashMap<>();
/** expert: instantiates a new reader */
Lucene410DocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
@ -104,7 +110,7 @@ class Lucene410DocValuesProducer extends DocValuesProducer implements Closeable
binaries = new HashMap<>();
sortedSets = new HashMap<>();
sortedNumerics = new HashMap<>();
readFields(in, state.fieldInfos);
numFields = readFields(in, state.fieldInfos);
CodecUtil.checkFooter(in);
success = true;
@ -143,108 +149,110 @@ class Lucene410DocValuesProducer extends DocValuesProducer implements Closeable
ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOfInstance(getClass()));
}
private void readSortedField(int fieldNumber, IndexInput meta, FieldInfos infos) throws IOException {
private void readSortedField(FieldInfo info, IndexInput meta) throws IOException {
// sorted = binary + numeric
if (meta.readVInt() != fieldNumber) {
throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
if (meta.readVInt() != info.number) {
throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
}
if (meta.readByte() != Lucene410DocValuesFormat.BINARY) {
throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
}
BinaryEntry b = readBinaryEntry(meta);
binaries.put(fieldNumber, b);
binaries.put(info.name, b);
if (meta.readVInt() != fieldNumber) {
throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
if (meta.readVInt() != info.number) {
throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
}
if (meta.readByte() != Lucene410DocValuesFormat.NUMERIC) {
throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
}
NumericEntry n = readNumericEntry(meta);
ords.put(fieldNumber, n);
ords.put(info.name, n);
}
private void readSortedSetFieldWithAddresses(int fieldNumber, IndexInput meta, FieldInfos infos) throws IOException {
private void readSortedSetFieldWithAddresses(FieldInfo info, IndexInput meta) throws IOException {
// sortedset = binary + numeric (addresses) + ordIndex
if (meta.readVInt() != fieldNumber) {
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
if (meta.readVInt() != info.number) {
throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
}
if (meta.readByte() != Lucene410DocValuesFormat.BINARY) {
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
}
BinaryEntry b = readBinaryEntry(meta);
binaries.put(fieldNumber, b);
binaries.put(info.name, b);
if (meta.readVInt() != fieldNumber) {
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
if (meta.readVInt() != info.number) {
throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
}
if (meta.readByte() != Lucene410DocValuesFormat.NUMERIC) {
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
}
NumericEntry n1 = readNumericEntry(meta);
ords.put(fieldNumber, n1);
ords.put(info.name, n1);
if (meta.readVInt() != fieldNumber) {
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
if (meta.readVInt() != info.number) {
throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
}
if (meta.readByte() != Lucene410DocValuesFormat.NUMERIC) {
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
}
NumericEntry n2 = readNumericEntry(meta);
ordIndexes.put(fieldNumber, n2);
ordIndexes.put(info.name, n2);
}
private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
private int readFields(IndexInput meta, FieldInfos infos) throws IOException {
int numFields = 0;
int fieldNumber = meta.readVInt();
while (fieldNumber != -1) {
if (infos.fieldInfo(fieldNumber) == null) {
// trickier to validate more: because we re-use for norms, because we use multiple entries
// for "composite" types like sortedset, etc.
numFields++;
FieldInfo info = infos.fieldInfo(fieldNumber);
if (info == null) {
// trickier to validate more: because we use multiple entries for "composite" types like sortedset, etc.
throw new CorruptIndexException("Invalid field number: " + fieldNumber + " (resource=" + meta + ")");
}
byte type = meta.readByte();
if (type == Lucene410DocValuesFormat.NUMERIC) {
numerics.put(fieldNumber, readNumericEntry(meta));
numerics.put(info.name, readNumericEntry(meta));
} else if (type == Lucene410DocValuesFormat.BINARY) {
BinaryEntry b = readBinaryEntry(meta);
binaries.put(fieldNumber, b);
binaries.put(info.name, b);
} else if (type == Lucene410DocValuesFormat.SORTED) {
readSortedField(fieldNumber, meta, infos);
readSortedField(info, meta);
} else if (type == Lucene410DocValuesFormat.SORTED_SET) {
SortedSetEntry ss = readSortedSetEntry(meta);
sortedSets.put(fieldNumber, ss);
sortedSets.put(info.name, ss);
if (ss.format == SORTED_WITH_ADDRESSES) {
readSortedSetFieldWithAddresses(fieldNumber, meta, infos);
readSortedSetFieldWithAddresses(info, meta);
} else if (ss.format == SORTED_SINGLE_VALUED) {
if (meta.readVInt() != fieldNumber) {
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
}
if (meta.readByte() != Lucene410DocValuesFormat.SORTED) {
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
}
readSortedField(fieldNumber, meta, infos);
readSortedField(info, meta);
} else {
throw new AssertionError();
}
} else if (type == Lucene410DocValuesFormat.SORTED_NUMERIC) {
SortedSetEntry ss = readSortedSetEntry(meta);
sortedNumerics.put(fieldNumber, ss);
sortedNumerics.put(info.name, ss);
if (meta.readVInt() != fieldNumber) {
throw new CorruptIndexException("sortednumeric entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
}
if (meta.readByte() != Lucene410DocValuesFormat.NUMERIC) {
throw new CorruptIndexException("sortednumeric entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
}
numerics.put(fieldNumber, readNumericEntry(meta));
numerics.put(info.name, readNumericEntry(meta));
if (ss.format == SORTED_WITH_ADDRESSES) {
if (meta.readVInt() != fieldNumber) {
throw new CorruptIndexException("sortednumeric entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
}
if (meta.readByte() != Lucene410DocValuesFormat.NUMERIC) {
throw new CorruptIndexException("sortednumeric entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt (resource=" + meta + ")");
}
NumericEntry ordIndex = readNumericEntry(meta);
ordIndexes.put(fieldNumber, ordIndex);
ordIndexes.put(info.name, ordIndex);
} else if (ss.format != SORTED_SINGLE_VALUED) {
throw new AssertionError();
}
@ -253,6 +261,7 @@ class Lucene410DocValuesProducer extends DocValuesProducer implements Closeable
}
fieldNumber = meta.readVInt();
}
return numFields;
}
static NumericEntry readNumericEntry(IndexInput meta) throws IOException {
@ -332,7 +341,7 @@ class Lucene410DocValuesProducer extends DocValuesProducer implements Closeable
@Override
public NumericDocValues getNumeric(FieldInfo field) throws IOException {
NumericEntry entry = numerics.get(field.number);
NumericEntry entry = numerics.get(field.name);
return getNumeric(entry);
}
@ -341,11 +350,25 @@ class Lucene410DocValuesProducer extends DocValuesProducer implements Closeable
return ramBytesUsed.get();
}
@Override
public synchronized Iterable<? extends Accountable> getChildResources() {
List<Accountable> resources = new ArrayList<>();
resources.addAll(Accountables.namedAccountables("addresses field", addressInstances));
resources.addAll(Accountables.namedAccountables("ord index field", ordIndexInstances));
resources.addAll(Accountables.namedAccountables("reverse index field", reverseIndexInstances));
return Collections.unmodifiableList(resources);
}
@Override
public void checkIntegrity() throws IOException {
CodecUtil.checksumEntireFile(data);
}
@Override
public String toString() {
return getClass().getSimpleName() + "(fields=" + numFields + ")";
}
LongValues getNumeric(NumericEntry entry) throws IOException {
RandomAccessInput slice = this.data.randomAccessSlice(entry.offset, entry.endOffset - entry.offset);
switch (entry.format) {
@ -384,7 +407,7 @@ class Lucene410DocValuesProducer extends DocValuesProducer implements Closeable
@Override
public BinaryDocValues getBinary(FieldInfo field) throws IOException {
BinaryEntry bytes = binaries.get(field.number);
BinaryEntry bytes = binaries.get(field.name);
switch(bytes.format) {
case BINARY_FIXED_UNCOMPRESSED:
return getFixedBinary(field, bytes);
@ -420,11 +443,11 @@ class Lucene410DocValuesProducer extends DocValuesProducer implements Closeable
/** returns an address instance for variable-length binary values. */
private synchronized MonotonicBlockPackedReader getAddressInstance(FieldInfo field, BinaryEntry bytes) throws IOException {
MonotonicBlockPackedReader addresses = addressInstances.get(field.number);
MonotonicBlockPackedReader addresses = addressInstances.get(field.name);
if (addresses == null) {
data.seek(bytes.addressesOffset);
addresses = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, bytes.count+1, false);
addressInstances.put(field.number, addresses);
addressInstances.put(field.name, addresses);
ramBytesUsed.addAndGet(addresses.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
}
return addresses;
@ -457,12 +480,12 @@ class Lucene410DocValuesProducer extends DocValuesProducer implements Closeable
/** returns an address instance for prefix-compressed binary values. */
private synchronized MonotonicBlockPackedReader getIntervalInstance(FieldInfo field, BinaryEntry bytes) throws IOException {
MonotonicBlockPackedReader addresses = addressInstances.get(field.number);
MonotonicBlockPackedReader addresses = addressInstances.get(field.name);
if (addresses == null) {
data.seek(bytes.addressesOffset);
final long size = (bytes.count + INTERVAL_MASK) >>> INTERVAL_SHIFT;
addresses = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
addressInstances.put(field.number, addresses);
addressInstances.put(field.name, addresses);
ramBytesUsed.addAndGet(addresses.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
}
return addresses;
@ -470,7 +493,7 @@ class Lucene410DocValuesProducer extends DocValuesProducer implements Closeable
/** returns a reverse lookup instance for prefix-compressed binary values. */
private synchronized ReverseTermsIndex getReverseIndexInstance(FieldInfo field, BinaryEntry bytes) throws IOException {
ReverseTermsIndex index = reverseIndexInstances.get(field.number);
ReverseTermsIndex index = reverseIndexInstances.get(field.name);
if (index == null) {
index = new ReverseTermsIndex();
data.seek(bytes.reverseIndexOffset);
@ -480,8 +503,8 @@ class Lucene410DocValuesProducer extends DocValuesProducer implements Closeable
PagedBytes pagedBytes = new PagedBytes(15);
pagedBytes.copy(data, dataSize);
index.terms = pagedBytes.freeze(true);
reverseIndexInstances.put(field.number, index);
ramBytesUsed.addAndGet(index.termAddresses.ramBytesUsed() + index.terms.ramBytesUsed());
reverseIndexInstances.put(field.name, index);
ramBytesUsed.addAndGet(index.ramBytesUsed());
}
return index;
}
@ -496,9 +519,9 @@ class Lucene410DocValuesProducer extends DocValuesProducer implements Closeable
@Override
public SortedDocValues getSorted(FieldInfo field) throws IOException {
final int valueCount = (int) binaries.get(field.number).count;
final int valueCount = (int) binaries.get(field.name).count;
final BinaryDocValues binary = getBinary(field);
NumericEntry entry = ords.get(field.number);
NumericEntry entry = ords.get(field.name);
final LongValues ordinals = getNumeric(entry);
return new SortedDocValues() {
@ -539,11 +562,11 @@ class Lucene410DocValuesProducer extends DocValuesProducer implements Closeable
/** returns an address instance for sortedset ordinal lists */
private synchronized MonotonicBlockPackedReader getOrdIndexInstance(FieldInfo field, NumericEntry entry) throws IOException {
MonotonicBlockPackedReader instance = ordIndexInstances.get(field.number);
MonotonicBlockPackedReader instance = ordIndexInstances.get(field.name);
if (instance == null) {
data.seek(entry.offset);
instance = MonotonicBlockPackedReader.of(data, entry.packedIntsVersion, entry.blockSize, entry.count+1, false);
ordIndexInstances.put(field.number, instance);
ordIndexInstances.put(field.name, instance);
ramBytesUsed.addAndGet(instance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
}
return instance;
@ -551,14 +574,14 @@ class Lucene410DocValuesProducer extends DocValuesProducer implements Closeable
@Override
public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException {
SortedSetEntry ss = sortedNumerics.get(field.number);
NumericEntry numericEntry = numerics.get(field.number);
SortedSetEntry ss = sortedNumerics.get(field.name);
NumericEntry numericEntry = numerics.get(field.name);
final LongValues values = getNumeric(numericEntry);
if (ss.format == SORTED_SINGLE_VALUED) {
final Bits docsWithField = getMissingBits(numericEntry.missingOffset);
return DocValues.singleton(values, docsWithField);
} else if (ss.format == SORTED_WITH_ADDRESSES) {
final MonotonicBlockPackedReader ordIndex = getOrdIndexInstance(field, ordIndexes.get(field.number));
final MonotonicBlockPackedReader ordIndex = getOrdIndexInstance(field, ordIndexes.get(field.name));
return new SortedNumericDocValues() {
long startOffset;
@ -587,7 +610,7 @@ class Lucene410DocValuesProducer extends DocValuesProducer implements Closeable
@Override
public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
SortedSetEntry ss = sortedSets.get(field.number);
SortedSetEntry ss = sortedSets.get(field.name);
if (ss.format == SORTED_SINGLE_VALUED) {
final SortedDocValues values = getSorted(field);
return DocValues.singleton(values);
@ -595,12 +618,12 @@ class Lucene410DocValuesProducer extends DocValuesProducer implements Closeable
throw new AssertionError();
}
final long valueCount = binaries.get(field.number).count;
final long valueCount = binaries.get(field.name).count;
// we keep the byte[]s and list of ords on disk, these could be large
final LongBinaryDocValues binary = (LongBinaryDocValues) getBinary(field);
final LongValues ordinals = getNumeric(ords.get(field.number));
final LongValues ordinals = getNumeric(ords.get(field.name));
// but the addresses to the ord stream are in RAM
final MonotonicBlockPackedReader ordIndex = getOrdIndexInstance(field, ordIndexes.get(field.number));
final MonotonicBlockPackedReader ordIndex = getOrdIndexInstance(field, ordIndexes.get(field.name));
return new RandomAccessOrds() {
long startOffset;
@ -698,10 +721,10 @@ class Lucene410DocValuesProducer extends DocValuesProducer implements Closeable
case SORTED:
return DocValues.docsWithValue(getSorted(field), maxDoc);
case BINARY:
BinaryEntry be = binaries.get(field.number);
BinaryEntry be = binaries.get(field.name);
return getMissingBits(be.missingOffset);
case NUMERIC:
NumericEntry ne = numerics.get(field.number);
NumericEntry ne = numerics.get(field.name);
return getMissingBits(ne.missingOffset);
default:
throw new AssertionError();
@ -778,9 +801,27 @@ class Lucene410DocValuesProducer extends DocValuesProducer implements Closeable
}
// used for reverse lookup to a small range of blocks
static class ReverseTermsIndex {
static class ReverseTermsIndex implements Accountable {
public MonotonicBlockPackedReader termAddresses;
public PagedBytes.Reader terms;
@Override
public long ramBytesUsed() {
return termAddresses.ramBytesUsed() + terms.ramBytesUsed();
}
@Override
public Iterable<? extends Accountable> getChildResources() {
List<Accountable> resources = new ArrayList<>();
resources.add(Accountables.namedAccountable("term bytes", terms));
resources.add(Accountables.namedAccountable("term addresses", termAddresses));
return Collections.unmodifiableList(resources);
}
@Override
public String toString() {
return getClass().getSimpleName() + "(size=" + termAddresses.size() + ")";
}
}
//in the compressed case, we add a few additional operations for

View File

@ -20,6 +20,7 @@ package org.apache.lucene.codecs.lucene49;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.codecs.CodecUtil;
@ -32,6 +33,8 @@ import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.packed.BlockPackedReader;
@ -49,15 +52,17 @@ import static org.apache.lucene.codecs.lucene49.Lucene49NormsConsumer.UNCOMPRESS
*/
class Lucene49NormsProducer extends NormsProducer {
// metadata maps (just file pointers and minimal stuff)
private final Map<Integer,NormsEntry> norms = new HashMap<>();
private final Map<String,NormsEntry> norms = new HashMap<>();
private final IndexInput data;
private final int version;
// ram instances we have already loaded
final Map<Integer,NumericDocValues> instances = new HashMap<>();
final Map<String,NumericDocValues> instances = new HashMap<>();
final Map<String,Accountable> instancesInfo = new HashMap<>();
private final int maxDoc;
private final AtomicLong ramBytesUsed;
private final AtomicInteger activeCount = new AtomicInteger();
Lucene49NormsProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
maxDoc = state.segmentInfo.getDocCount();
@ -123,17 +128,18 @@ class Lucene49NormsProducer extends NormsProducer {
default:
throw new CorruptIndexException("Unknown format: " + entry.format + ", input=" + meta);
}
norms.put(fieldNumber, entry);
norms.put(info.name, entry);
fieldNumber = meta.readVInt();
}
}
@Override
public synchronized NumericDocValues getNorms(FieldInfo field) throws IOException {
NumericDocValues instance = instances.get(field.number);
NumericDocValues instance = instances.get(field.name);
if (instance == null) {
instance = loadNorms(field);
instances.put(field.number, instance);
instances.put(field.name, instance);
activeCount.incrementAndGet();
}
return instance;
}
@ -143,15 +149,22 @@ class Lucene49NormsProducer extends NormsProducer {
return ramBytesUsed.get();
}
@Override
public synchronized Iterable<? extends Accountable> getChildResources() {
return Accountables.namedAccountables("field", instancesInfo);
}
@Override
public void checkIntegrity() throws IOException {
CodecUtil.checksumEntireFile(data);
}
private NumericDocValues loadNorms(FieldInfo field) throws IOException {
NormsEntry entry = norms.get(field.number);
NormsEntry entry = norms.get(field.name);
switch(entry.format) {
case CONST_COMPRESSED:
instancesInfo.put(field.name, Accountables.namedAccountable("constant", 8));
ramBytesUsed.addAndGet(8);
final long v = entry.offset;
return new NumericDocValues() {
@Override
@ -164,6 +177,7 @@ class Lucene49NormsProducer extends NormsProducer {
final byte bytes[] = new byte[maxDoc];
data.readBytes(bytes, 0, bytes.length);
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(bytes));
instancesInfo.put(field.name, Accountables.namedAccountable("byte array", maxDoc));
return new NumericDocValues() {
@Override
public long get(int docID) {
@ -176,6 +190,7 @@ class Lucene49NormsProducer extends NormsProducer {
int blockSize = data.readVInt();
final BlockPackedReader reader = new BlockPackedReader(data, packedIntsVersion, blockSize, maxDoc, false);
ramBytesUsed.addAndGet(reader.ramBytesUsed());
instancesInfo.put(field.name, Accountables.namedAccountable("delta compressed", reader));
return reader;
case TABLE_COMPRESSED:
data.seek(entry.offset);
@ -192,6 +207,7 @@ class Lucene49NormsProducer extends NormsProducer {
final int bitsPerValue = data.readVInt();
final PackedInts.Reader ordsReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatID), packedVersion, maxDoc, bitsPerValue);
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(decode) + ordsReader.ramBytesUsed());
instancesInfo.put(field.name, Accountables.namedAccountable("table compressed", ordsReader));
return new NumericDocValues() {
@Override
public long get(int docID) {
@ -212,4 +228,9 @@ class Lucene49NormsProducer extends NormsProducer {
byte format;
long offset;
}
@Override
public String toString() {
return getClass().getSimpleName() + "(fields=" + norms.size() + ",active=" + activeCount.get() + ")";
}
}

View File

@ -19,6 +19,7 @@ package org.apache.lucene.codecs.perfield;
import java.io.Closeable;
import java.io.IOException;
import java.util.Collections;
import java.util.HashMap;
import java.util.IdentityHashMap;
import java.util.Map;
@ -37,6 +38,8 @@ import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.SortedNumericDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
@ -322,6 +325,11 @@ public abstract class PerFieldDocValuesFormat extends DocValuesFormat {
}
return size;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Accountables.namedAccountables("format", formats);
}
@Override
public void checkIntegrity() throws IOException {
@ -329,6 +337,11 @@ public abstract class PerFieldDocValuesFormat extends DocValuesFormat {
format.checkIntegrity();
}
}
@Override
public String toString() {
return "PerFieldDocValues(formats=" + formats.size() + ")";
}
}
@Override

View File

@ -38,6 +38,8 @@ import org.apache.lucene.index.Fields;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.Terms;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.RamUsageEstimator;
@ -268,6 +270,11 @@ public abstract class PerFieldPostingsFormat extends PostingsFormat {
}
return ramBytesUsed;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Accountables.namedAccountables("format", formats);
}
@Override
public void checkIntegrity() throws IOException {
@ -275,6 +282,11 @@ public abstract class PerFieldPostingsFormat extends PostingsFormat {
producer.checkIntegrity();
}
}
@Override
public String toString() {
return "PerFieldPostings(formats=" + formats.size() + ")";
}
}
@Override

View File

@ -116,6 +116,11 @@ class BufferedUpdatesStream implements Accountable {
public long ramBytesUsed() {
return bytesUsed.get();
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
public static class ApplyDeletesResult {

View File

@ -39,6 +39,7 @@ import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FSDirectory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
@ -532,7 +533,7 @@ public class CheckIndex {
int toLoseDocCount = info.info.getDocCount();
AtomicReader reader = null;
SegmentReader reader = null;
try {
msg(infoStream, " version=" + (version == null ? "3.0" : version));
@ -660,6 +661,11 @@ public class CheckIndex {
}
msg(infoStream, "");
if (verbose) {
msg(infoStream, "detailed segment RAM usage: ");
msg(infoStream, Accountables.toString(reader));
}
} catch (Throwable t) {
if (failFast) {

View File

@ -20,6 +20,7 @@ package org.apache.lucene.index;
import java.io.Closeable;
import java.io.IOException;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.Queue;
import java.util.Set;
@ -669,6 +670,11 @@ final class DocumentsWriter implements Closeable, Accountable {
public long ramBytesUsed() {
return flushControl.ramBytesUsed();
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
static final class ApplyDeletesEvent implements Event {
static final Event INSTANCE = new ApplyDeletesEvent();

View File

@ -18,6 +18,7 @@ package org.apache.lucene.index;
*/
import java.util.Arrays;
import java.util.Collections;
import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
import java.util.concurrent.locks.ReentrantLock;
@ -455,6 +456,11 @@ final class DocumentsWriterDeleteQueue implements Accountable {
return globalBufferedUpdates.bytesUsed.get();
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
@Override
public String toString() {
return "DWDQ: [ generation: " + generation + " ]";

View File

@ -18,6 +18,7 @@ package org.apache.lucene.index;
*/
import java.util.ArrayList;
import java.util.Collections;
import java.util.IdentityHashMap;
import java.util.Iterator;
import java.util.LinkedList;
@ -437,6 +438,12 @@ final class DocumentsWriterFlushControl implements Accountable {
return getDeleteBytesUsed() + netBytes();
}
@Override
public Iterable<? extends Accountable> getChildResources() {
// TODO: improve this?
return Collections.emptyList();
}
synchronized int numFlushingDWPT() {
return flushingWriters.size();
}

View File

@ -463,6 +463,11 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
ensureOpen();
return docWriter.ramBytesUsed();
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
/** Holds shared SegmentReader instances. IndexWriter uses
* SegmentReaders for 1) applying deletes, 2) doing

View File

@ -18,12 +18,15 @@ package org.apache.lucene.index;
*/
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
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.Accountables;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.InPlaceMergeSorter;
@ -429,6 +432,10 @@ public class MultiDocValues {
return BASE_RAM_BYTES_USED + RamUsageEstimator.sizeOf(newToOld) + RamUsageEstimator.sizeOf(oldToNew);
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
}
/**
@ -634,6 +641,16 @@ public class MultiDocValues {
public long ramBytesUsed() {
return ramBytesUsed;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
List<Accountable> resources = new ArrayList<>();
resources.add(Accountables.namedAccountable("global ord deltas", globalOrdDeltas));
resources.add(Accountables.namedAccountable("first segments", firstSegments));
resources.add(Accountables.namedAccountable("segment map", segmentMap));
// TODO: would be nice to return actual child segment deltas too, but the optimizations are confusing
return resources;
}
}
/**

View File

@ -18,6 +18,7 @@ package org.apache.lucene.index;
*/
import java.io.IOException;
import java.util.Collections;
import java.util.Iterator;
import org.apache.lucene.store.IndexInput;
@ -44,6 +45,11 @@ class PrefixCodedTerms implements Iterable<Term>, Accountable {
return buffer.ramBytesUsed();
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
/** @return iterator over the bytes */
@Override
public Iterator<Term> iterator() {

View File

@ -215,6 +215,7 @@ final class SegmentCoreReaders implements Accountable {
coreClosedListeners.remove(listener);
}
// TODO: remove this, it can just be on SR
@Override
public long ramBytesUsed() {
return BASE_RAM_BYTES_USED +
@ -223,4 +224,9 @@ final class SegmentCoreReaders implements Accountable {
((fieldsReaderOrig!=null)? fieldsReaderOrig.ramBytesUsed() : 0) +
((termVectorsReaderOrig!=null) ? termVectorsReaderOrig.ramBytesUsed() : 0);
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
}

View File

@ -30,6 +30,8 @@ import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.Version;
@ -185,5 +187,19 @@ class SegmentDocValuesProducer extends DocValuesProducer {
ramBytesUsed += producer.ramBytesUsed();
}
return ramBytesUsed;
}
}
@Override
public Iterable<? extends Accountable> getChildResources() {
List<Accountable> resources = new ArrayList<>();
for (Accountable producer : dvProducers) {
resources.add(Accountables.namedAccountable("delegate", producer));
}
return Collections.unmodifiableList(resources);
}
@Override
public String toString() {
return getClass().getSimpleName() + "(producers=" + dvProducers.size() + ")";
}
}

View File

@ -18,14 +18,17 @@ package org.apache.lucene.index;
*/
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.TermVectorsReader;
@ -34,6 +37,7 @@ 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.Accountables;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.CloseableThreadLocal;
import org.apache.lucene.util.IOUtils;
@ -258,7 +262,7 @@ public final class SegmentReader extends AtomicReader implements Accountable {
}
@Override
public Fields fields() {
public FieldsProducer fields() {
ensureOpen();
return core.fields;
}
@ -537,6 +541,28 @@ public final class SegmentReader extends AtomicReader implements Accountable {
return ramBytesUsed;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
ensureOpen();
List<Accountable> resources = new ArrayList<>();
if (core.fields != null) {
resources.add(Accountables.namedAccountable("postings", core.fields));
}
if (core.normsProducer != null) {
resources.add(Accountables.namedAccountable("norms", core.normsProducer));
}
if (docValuesProducer != null) {
resources.add(Accountables.namedAccountable("docvalues", docValuesProducer));
}
if (getFieldsReader() != null) {
resources.add(Accountables.namedAccountable("stored fields", getFieldsReader()));
}
if (getTermVectorsReader() != null) {
resources.add(Accountables.namedAccountable("term vectors", getTermVectorsReader()));
}
return resources;
}
@Override
public void checkIntegrity() throws IOException {
ensureOpen();

View File

@ -29,6 +29,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.Accountables;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.WAH8DocIdSet;
@ -147,4 +148,10 @@ public class CachingWrapperFilter extends Filter implements Accountable {
return total;
}
@Override
public synchronized Iterable<? extends Accountable> getChildResources() {
// Sync only to pull the current set of values:
return Accountables.namedAccountables("segment", cache);
}
}

View File

@ -18,6 +18,7 @@ package org.apache.lucene.search;
*/
import java.io.IOException;
import java.util.Collections;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Bits;
@ -93,4 +94,8 @@ public abstract class DocIdSet implements Accountable {
return false;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
}

View File

@ -19,6 +19,7 @@ package org.apache.lucene.search;
import java.io.IOException;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.RamUsageEstimator;
@ -62,6 +63,11 @@ public abstract class FilteredDocIdSet extends DocIdSet {
public long ramBytesUsed() {
return RamUsageEstimator.NUM_BYTES_OBJECT_REF + _innerSet.ramBytesUsed();
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return _innerSet.getChildResources();
}
@Override
public Bits bits() throws IOException {

View File

@ -20,11 +20,13 @@ package org.apache.lucene.store;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.Set;
import org.apache.lucene.store.RAMDirectory; // javadocs
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.IOUtils;
// TODO
@ -256,4 +258,9 @@ public class NRTCachingDirectory extends FilterDirectory implements Accountable
public long ramBytesUsed() {
return cache.ramBytesUsed();
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.singleton(Accountables.namedAccountable("cache", cache));
}
}

View File

@ -28,6 +28,7 @@ import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
/**
@ -150,6 +151,16 @@ public class RAMDirectory extends BaseDirectory implements Accountable {
return sizeInBytes.get();
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Accountables.namedAccountables("file", fileMap);
}
@Override
public String toString() {
return getClass().getSimpleName() + "(id=" + getLockID() + ")";
}
/** Removes an existing file in the directory.
* @throws IOException if the file does not exist
*/

View File

@ -18,6 +18,7 @@ package org.apache.lucene.store;
*/
import java.util.ArrayList;
import java.util.Collections;
import org.apache.lucene.util.Accountable;
@ -82,4 +83,13 @@ public class RAMFile implements Accountable {
return sizeInBytes;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
@Override
public String toString() {
return getClass().getSimpleName() + "(length=" + length + ")";
}
}

View File

@ -18,10 +18,12 @@ package org.apache.lucene.store;
*/
import java.io.IOException;
import java.util.Collections;
import java.util.zip.CRC32;
import java.util.zip.Checksum;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
/**
* A memory-resident {@link IndexOutput} implementation.
@ -182,6 +184,11 @@ public class RAMOutputStream extends IndexOutput implements Accountable {
public long ramBytesUsed() {
return (long) file.numBuffers() * (long) BUFFER_SIZE;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.singleton(Accountables.namedAccountable("file", file));
}
@Override
public long getChecksum() throws IOException {

View File

@ -29,4 +29,12 @@ public interface Accountable {
*/
long ramBytesUsed();
/**
* Returns nested resources of this class.
* The result should be a point-in-time snapshot (to avoid race conditions).
* @see Accountables
*/
// TODO: on java8 make this a default method returning emptyList
Iterable<? extends Accountable> getChildResources();
}

View File

@ -0,0 +1,141 @@
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.
*/
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
/**
* Helper methods for constructing nested resource descriptions
* and debugging RAM usage.
* <p>
* {@code toString(Accountable}} can be used to quickly debug the nested
* structure of any Accountable.
* <p>
* The {@code namedAccountable} and {@code namedAccountables} methods return
* type-safe, point-in-time snapshots of the provided resources.
*/
public class Accountables {
private Accountables() {}
/**
* Returns a String description of an Accountable and any nested resources.
* This is intended for development and debugging.
*/
public static String toString(Accountable a) {
StringBuilder sb = new StringBuilder();
toString(sb, a, 0);
return sb.toString();
}
private static StringBuilder toString(StringBuilder dest, Accountable a, int depth) {
for (int i = 1; i < depth; i++) {
dest.append(" ");
}
if (depth > 0) {
dest.append("|-- ");
}
dest.append(a.toString());
dest.append(": ");
dest.append(RamUsageEstimator.humanReadableUnits(a.ramBytesUsed()));
dest.append(System.lineSeparator());
for (Accountable child : a.getChildResources()) {
toString(dest, child, depth + 1);
}
return dest;
}
/**
* Augments an existing accountable with the provided description.
* <p>
* The resource description is constructed in this format:
* {@code description [toString()]}
* <p>
* This is a point-in-time type safe view: consumers
* will not be able to cast or manipulate the resource in any way.
*/
public static Accountable namedAccountable(String description, Accountable in) {
return namedAccountable(description + " [" + in + "]", in.getChildResources(), in.ramBytesUsed());
}
/**
* Returns an accountable with the provided description and bytes.
*/
public static Accountable namedAccountable(String description, long bytes) {
return namedAccountable(description, Collections.<Accountable>emptyList(), bytes);
}
/**
* Converts a map of resources to a collection.
* <p>
* The resource descriptions are constructed in this format:
* {@code prefix 'key' [toString()]}
* <p>
* This is a point-in-time type safe view: consumers
* will not be able to cast or manipulate the resources in any way.
*/
public static Collection<Accountable> namedAccountables(String prefix, Map<?,? extends Accountable> in) {
List<Accountable> resources = new ArrayList<>();
for (Map.Entry<?,? extends Accountable> kv : in.entrySet()) {
resources.add(namedAccountable(prefix + " '" + kv.getKey() + "'", kv.getValue()));
}
Collections.sort(resources, new Comparator<Accountable>() {
@Override
public int compare(Accountable o1, Accountable o2) {
return o1.toString().compareTo(o2.toString());
}
});
return Collections.unmodifiableList(resources);
}
/**
* Returns an accountable with the provided description, bytes, and children.
* <p>
* The resource descriptions are constructed in this format:
* {@code description [toString()]}
* <p>
* This is a point-in-time type safe view: consumers
* will not be able to cast or manipulate the resources in any way..
*/
private static Accountable namedAccountable(final String description, final Iterable<? extends Accountable> children, final long bytes) {
return new Accountable() {
@Override
public long ramBytesUsed() {
return bytes;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return children;
}
@Override
public String toString() {
return description;
}
};
}
}

View File

@ -19,6 +19,7 @@ package org.apache.lucene.util;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
@ -134,6 +135,16 @@ public final class PagedBytes implements Accountable {
}
return size;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
@Override
public String toString() {
return "PagedBytes(blocksize=" + blockSize + ")";
}
}
/** 1&lt;&lt;blockBits must be bigger than biggest single
@ -246,6 +257,11 @@ public final class PagedBytes implements Accountable {
}
return size;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
/** Copy bytes in, writing the length as a 1 or 2 byte
* vInt prefix. */

View File

@ -157,4 +157,9 @@ public final class ByteSequenceOutputs extends Outputs<BytesRef> {
public long ramBytesUsed(BytesRef output) {
return BASE_NUM_BYTES + RamUsageEstimator.sizeOf(output.bytes);
}
@Override
public String toString() {
return "ByteSequenceOutputs";
}
}

View File

@ -19,6 +19,7 @@ package org.apache.lucene.util.fst;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import org.apache.lucene.store.DataInput;
@ -480,5 +481,14 @@ class BytesStore extends DataOutput implements Accountable {
}
return size;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
@Override
public String toString() {
return getClass().getSimpleName() + "(numBlocks=" + blocks.size() + ")";
}
}

View File

@ -22,9 +22,11 @@ import java.io.BufferedOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.util.ArrayList;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.lucene.codecs.CodecUtil;
@ -35,6 +37,7 @@ 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.Accountables;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Constants;
import org.apache.lucene.util.IOUtils;
@ -444,6 +447,23 @@ public final class FST<T> implements Accountable {
return size;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
List<Accountable> resources = new ArrayList<>();
if (packed) {
resources.add(Accountables.namedAccountable("node ref to address", nodeRefToAddress));
} else if (nodeAddress != null) {
resources.add(Accountables.namedAccountable("node addresses", nodeAddress));
resources.add(Accountables.namedAccountable("in counts", inCounts));
}
return resources;
}
@Override
public String toString() {
return getClass().getSimpleName() + "(input=" + inputType + ",output=" + outputs + ",packed=" + packed + ",nodes=" + nodeCount + ",arcs=" + arcCount + ")";
}
void finish(long newStartNode) throws IOException {
if (startNode != -1) {
throw new IllegalStateException("already finished");

View File

@ -160,4 +160,9 @@ public final class IntSequenceOutputs extends Outputs<IntsRef> {
public long ramBytesUsed(IntsRef output) {
return BASE_NUM_BYTES + RamUsageEstimator.sizeOf(output.ints);
}
@Override
public String toString() {
return "IntSequenceOutputs";
}
}

View File

@ -106,4 +106,9 @@ public final class NoOutputs extends Outputs<Object> {
public long ramBytesUsed(Object output) {
return 0;
}
@Override
public String toString() {
return "NoOutputs";
}
}

View File

@ -27,6 +27,7 @@ import static org.apache.lucene.util.packed.PackedInts.checkBlockSize;
import static org.apache.lucene.util.packed.PackedInts.numBlocks;
import java.io.IOException;
import java.util.Collections;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Accountable;
@ -42,6 +43,7 @@ public final class BlockPackedReader extends LongValues implements Accountable {
private final long valueCount;
private final long[] minValues;
private final PackedInts.Reader[] subReaders;
private final long sumBPV;
/** Sole constructor. */
public BlockPackedReader(IndexInput in, int packedIntsVersion, int blockSize, long valueCount, boolean direct) throws IOException {
@ -51,9 +53,11 @@ public final class BlockPackedReader extends LongValues implements Accountable {
final int numBlocks = numBlocks(valueCount, blockSize);
long[] minValues = null;
subReaders = new PackedInts.Reader[numBlocks];
long sumBPV = 0;
for (int i = 0; i < numBlocks; ++i) {
final int token = in.readByte() & 0xFF;
final int bitsPerValue = token >>> BPV_SHIFT;
sumBPV += bitsPerValue;
if (bitsPerValue > 64) {
throw new IOException("Corrupted");
}
@ -77,6 +81,7 @@ public final class BlockPackedReader extends LongValues implements Accountable {
}
}
this.minValues = minValues;
this.sumBPV = sumBPV;
}
@Override
@ -95,4 +100,15 @@ public final class BlockPackedReader extends LongValues implements Accountable {
}
return size;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
@Override
public String toString() {
long avgBPV = subReaders.length == 0 ? 0 : sumBPV / subReaders.length;
return getClass().getSimpleName() + "(blocksize=" + (1<<blockShift) + ",size=" + valueCount + ",avgBPV=" + avgBPV + ")";
}
}

View File

@ -18,6 +18,7 @@
package org.apache.lucene.util.packed;
import java.util.Arrays;
import java.util.Collections;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.FixedBitSet; // for javadocs
@ -362,5 +363,10 @@ public class EliasFanoEncoder implements Accountable {
+ RamUsageEstimator.sizeOf(upperLongs)
+ RamUsageEstimator.sizeOf(upperZeroBitPositionIndex);
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
}

View File

@ -24,6 +24,7 @@ import static org.apache.lucene.util.packed.PackedInts.checkBlockSize;
import static org.apache.lucene.util.packed.PackedInts.numBlocks;
import java.io.IOException;
import java.util.Collections;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Accountable;
@ -46,6 +47,7 @@ public class MonotonicBlockPackedReader extends LongValues implements Accountabl
final long[] minValues;
final float[] averages;
final PackedInts.Reader[] subReaders;
final long sumBPV;
/** Sole constructor. */
public static MonotonicBlockPackedReader of(IndexInput in, int packedIntsVersion, int blockSize, long valueCount, boolean direct) throws IOException {
@ -68,6 +70,7 @@ public class MonotonicBlockPackedReader extends LongValues implements Accountabl
minValues = new long[numBlocks];
averages = new float[numBlocks];
subReaders = new PackedInts.Reader[numBlocks];
long sumBPV = 0;
for (int i = 0; i < numBlocks; ++i) {
if (packedIntsVersion < PackedInts.VERSION_MONOTONIC_WITHOUT_ZIGZAG) {
minValues[i] = in.readVLong();
@ -76,6 +79,7 @@ public class MonotonicBlockPackedReader extends LongValues implements Accountabl
}
averages[i] = Float.intBitsToFloat(in.readInt());
final int bitsPerValue = in.readVInt();
sumBPV += bitsPerValue;
if (bitsPerValue > 64) {
throw new IOException("Corrupted");
}
@ -92,6 +96,7 @@ public class MonotonicBlockPackedReader extends LongValues implements Accountabl
}
}
}
this.sumBPV = sumBPV;
}
@Override
@ -121,5 +126,15 @@ public class MonotonicBlockPackedReader extends LongValues implements Accountabl
}
return sizeInBytes;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
@Override
public String toString() {
long avgBPV = subReaders.length == 0 ? 0 : sumBPV / subReaders.length;
return getClass().getSimpleName() + "(blocksize=" + (1<<blockShift) + ",size=" + valueCount + ",avgBPV=" + avgBPV + ")";
}
}

View File

@ -126,6 +126,6 @@ final class Packed16ThreeBlocks extends PackedInts.MutableImpl {
@Override
public String toString() {
return getClass().getSimpleName() + "(bitsPerValue=" + bitsPerValue
+ ", size=" + size() + ", elements.length=" + blocks.length + ")";
+ ",size=" + size() + ",blocks=" + blocks.length + ")";
}
}

View File

@ -238,8 +238,8 @@ class Packed64 extends PackedInts.MutableImpl {
@Override
public String toString() {
return "Packed64(bitsPerValue=" + bitsPerValue + ", size="
+ size() + ", elements.length=" + blocks.length + ")";
return "Packed64(bitsPerValue=" + bitsPerValue + ",size="
+ size() + ",blocks=" + blocks.length + ")";
}
@Override

View File

@ -203,7 +203,7 @@ abstract class Packed64SingleBlock extends PackedInts.MutableImpl {
@Override
public String toString() {
return getClass().getSimpleName() + "(bitsPerValue=" + bitsPerValue
+ ", size=" + size() + ", elements.length=" + blocks.length + ")";
+ ",size=" + size() + ",blocks=" + blocks.length + ")";
}
public static Packed64SingleBlock create(DataInput in,

View File

@ -124,6 +124,6 @@ final class Packed8ThreeBlocks extends PackedInts.MutableImpl {
@Override
public String toString() {
return getClass().getSimpleName() + "(bitsPerValue=" + bitsPerValue
+ ", size=" + size() + ", elements.length=" + blocks.length + ")";
+ ",size=" + size() + ",blocks=" + blocks.length + ")";
}
}

View File

@ -19,6 +19,7 @@ package org.apache.lucene.util.packed;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.index.NumericDocValues;
@ -478,6 +479,10 @@ public class PackedInts {
*/
public abstract int size();
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
}
/**
@ -627,7 +632,6 @@ public class PackedInts {
public final int size() {
return valueCount;
}
}
static abstract class MutableImpl extends Mutable {
@ -651,6 +655,10 @@ public class PackedInts {
return valueCount;
}
@Override
public String toString() {
return getClass().getSimpleName() + "(valueCount=" + valueCount + ",bitsPerValue=" + bitsPerValue + ")";
}
}
/** A {@link Reader} which has all its values equal to 0 (bitsPerValue = 0). */
@ -686,7 +694,6 @@ public class PackedInts {
public long ramBytesUsed() {
return RamUsageEstimator.alignObjectSize(RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + RamUsageEstimator.NUM_BYTES_INT);
}
}
/** A write-once Writer.

View File

@ -20,6 +20,7 @@ package org.apache.lucene.util.packed;
import static org.apache.lucene.util.packed.PackedInts.checkBlockSize;
import java.util.Arrays;
import java.util.Collections;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.ArrayUtil;
@ -114,6 +115,11 @@ public class PackedLongValues extends LongValues implements Accountable {
public long ramBytesUsed() {
return ramBytesUsed;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
/** Return an iterator over the values of this array. */
public Iterator iterator() {
@ -208,6 +214,11 @@ public class PackedLongValues extends LongValues implements Accountable {
return ramBytesUsed;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
/** Return the number of elements that have been added to this builder. */
public final long size() {
return size;

View File

@ -222,7 +222,7 @@ abstract class Packed64SingleBlock extends PackedInts.MutableImpl {
@Override
public String toString() {
return getClass().getSimpleName() + "(bitsPerValue=" + bitsPerValue
+ ", size=" + size() + ", elements.length=" + blocks.length + ")";
+ ",size=" + size() + ",blocks=" + blocks.length + ")";
}
public static Packed64SingleBlock create(DataInput in,

View File

@ -158,7 +158,7 @@ if __name__ == '__main__':
@Override
public String toString() {
return getClass().getSimpleName() + "(bitsPerValue=" + bitsPerValue
+ ", size=" + size() + ", elements.length=" + blocks.length + ")";
+ ",size=" + size() + ",blocks=" + blocks.length + ")";
}
}
""" %(MASKS[bpv], 2*bpv, MASKS[bpv], bpv, MASKS[bpv], MASKS[bpv], 2*bpv, MASKS[bpv], bpv, MASKS[bpv], CASTS[bpv], 2*bpv, CASTS[bpv], bpv, CASTS[bpv], CASTS[bpv],

View File

@ -18,6 +18,7 @@ package org.apache.lucene.facet.taxonomy;
*/
import java.io.IOException;
import java.util.Collections;
import java.util.Map;
import java.util.WeakHashMap;
@ -25,6 +26,7 @@ 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.Accountables;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.RamUsageEstimator;
@ -146,6 +148,11 @@ public class CachedOrdinalsReader extends OrdinalsReader implements Accountable
}
return mem;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
}
@Override
@ -157,4 +164,9 @@ public class CachedOrdinalsReader extends OrdinalsReader implements Accountable
return bytes;
}
@Override
public synchronized Iterable<? extends Accountable> getChildResources() {
return Accountables.namedAccountables("segment", ordsCache);
}
}

View File

@ -20,6 +20,7 @@ package org.apache.lucene.uninverting;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import org.apache.lucene.codecs.PostingsFormat; // javadocs
@ -181,6 +182,11 @@ public class DocTermOrds implements Accountable {
return sz;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
/** Inverts all terms */
public DocTermOrds(AtomicReader reader, Bits liveDocs, String field) throws IOException {
this(reader, liveDocs, field, null, Integer.MAX_VALUE);

View File

@ -19,6 +19,7 @@ package org.apache.lucene.uninverting;
import java.io.IOException;
import java.io.PrintStream;
import java.util.Collections;
import org.apache.lucene.analysis.NumericTokenStream;
import org.apache.lucene.document.DoubleField;
@ -63,6 +64,11 @@ interface FieldCache {
// don't call on the in-progress value, might make things angry.
return RamUsageEstimator.NUM_BYTES_OBJECT_REF;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
}
/**

View File

@ -20,6 +20,7 @@ package org.apache.lucene.uninverting;
import java.io.IOException;
import java.io.PrintStream;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -38,6 +39,7 @@ import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.FixedBitSet;
@ -379,6 +381,11 @@ class FieldCacheImpl implements FieldCache {
return base + (bits.length() >>> 3);
}
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
}
static final class DocsWithFieldCache extends Cache {
@ -479,6 +486,11 @@ class FieldCacheImpl implements FieldCache {
public long ramBytesUsed() {
return values.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_OBJECT_REF + RamUsageEstimator.NUM_BYTES_LONG;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
}
static final class LongCache extends Cache {
@ -597,6 +609,15 @@ class FieldCacheImpl implements FieldCache {
3*RamUsageEstimator.NUM_BYTES_OBJECT_REF +
RamUsageEstimator.NUM_BYTES_INT;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
List<Accountable> resources = new ArrayList<>();
resources.add(Accountables.namedAccountable("term bytes", bytes));
resources.add(Accountables.namedAccountable("ord -> term", termOrdToBytesOffset));
resources.add(Accountables.namedAccountable("doc -> ord", docToTermOrd));
return resources;
}
}
public SortedDocValues getTermsIndex(AtomicReader reader, String field) throws IOException {
@ -731,6 +752,13 @@ class FieldCacheImpl implements FieldCache {
public long ramBytesUsed() {
return bytes.ramBytesUsed() + docToOffset.ramBytesUsed() + 2*RamUsageEstimator.NUM_BYTES_OBJECT_REF;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
List<Accountable> resources = new ArrayList<>();
resources.add(Accountables.namedAccountable("term bytes", bytes));
return resources;
}
}
// TODO: this if DocTermsIndex was already created, we

View File

@ -18,6 +18,7 @@ package org.apache.lucene.codecs.idversion;
*/
import java.io.IOException;
import java.util.Collections;
import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.codecs.CodecUtil;
@ -27,6 +28,7 @@ import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.BitUtil;
import org.apache.lucene.util.Bits;
@ -95,8 +97,18 @@ final class IDVersionPostingsReader extends PostingsReaderBase {
public long ramBytesUsed() {
return 0;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
@Override
public void checkIntegrity() throws IOException {
}
@Override
public String toString() {
return getClass().getSimpleName();
}
}

View File

@ -18,8 +18,10 @@ package org.apache.lucene.codecs.idversion;
*/
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.TreeMap;
import org.apache.lucene.codecs.CodecUtil;
@ -34,6 +36,8 @@ import org.apache.lucene.index.Terms;
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.Accountables;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.fst.PairOutputs.Pair;
@ -242,11 +246,19 @@ public final class VersionBlockTreeTermsReader extends FieldsProducer {
@Override
public long ramBytesUsed() {
long sizeInByes = ((postingsReader!=null) ? postingsReader.ramBytesUsed() : 0);
long sizeInBytes = postingsReader.ramBytesUsed();
for(VersionFieldReader reader : fields.values()) {
sizeInByes += reader.ramBytesUsed();
sizeInBytes += reader.ramBytesUsed();
}
return sizeInByes;
return sizeInBytes;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
List<Accountable> resources = new ArrayList<>();
resources.addAll(Accountables.namedAccountables("field", fields));
resources.add(Accountables.namedAccountable("delegate", postingsReader));
return Collections.unmodifiableList(resources);
}
@Override
@ -257,4 +269,9 @@ public final class VersionBlockTreeTermsReader extends FieldsProducer {
// postings
postingsReader.checkIntegrity();
}
@Override
public String toString() {
return getClass().getSimpleName() + "(fields=" + fields.size() + ",delegate=" + postingsReader.toString() + ")";
}
}

View File

@ -18,6 +18,7 @@ package org.apache.lucene.codecs.idversion;
*/
import java.io.IOException;
import java.util.Collections;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfo.IndexOptions;
@ -26,6 +27,7 @@ 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.Accountables;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.fst.FST;
import org.apache.lucene.util.fst.PairOutputs.Pair;
@ -159,4 +161,18 @@ final class VersionFieldReader extends Terms implements Accountable {
public long ramBytesUsed() {
return ((index!=null)? index.ramBytesUsed() : 0);
}
@Override
public Iterable<? extends Accountable> getChildResources() {
if (index == null) {
return Collections.emptyList();
} else {
return Collections.singletonList(Accountables.namedAccountable("term index", index));
}
}
@Override
public String toString() {
return "IDVersionTerms(terms=" + numTerms + ",postings=" + sumDocFreq + ",positions=" + sumTotalTermFreq + ",docs=" + docCount + ")";
}
}

View File

@ -30,6 +30,7 @@ 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.Accountables;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.PriorityQueue;
@ -268,4 +269,13 @@ public abstract class Lookup implements Accountable {
*/
public abstract boolean load(DataInput input) throws IOException;
/**
* Returns nested resources of this class.
* The result should be a point-in-time snapshot (to avoid race conditions).
* @see Accountables
*/
@Override
public Iterable<? extends Accountable> getChildResources() {
return null;
}
}

View File

@ -78,6 +78,8 @@ import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FSDirectory;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.Version;
@ -724,6 +726,29 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
}
}
@Override
public Iterable<? extends Accountable> getChildResources() {
List<Accountable> resources = new ArrayList<>();
try {
if (searcherMgr != null) {
IndexSearcher searcher = searcherMgr.acquire();
try {
for (AtomicReaderContext context : searcher.getIndexReader().leaves()) {
AtomicReader reader = FilterAtomicReader.unwrap(context.reader());
if (reader instanceof SegmentReader) {
resources.add(Accountables.namedAccountable("segment", (SegmentReader)reader));
}
}
} finally {
searcherMgr.release(searcher);
}
}
return resources;
} catch (IOException ioe) {
throw new RuntimeException(ioe);
}
}
@Override
public long getCount() throws IOException {
if (searcherMgr == null) {

View File

@ -37,6 +37,8 @@ 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.Accountables;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
@ -258,6 +260,15 @@ public class AnalyzingSuggester extends Lookup {
return fst == null ? 0 : fst.ramBytesUsed();
}
@Override
public Iterable<? extends Accountable> getChildResources() {
if (fst == null) {
return Collections.emptyList();
} else {
return Collections.singletonList(Accountables.namedAccountable("fst", fst));
}
}
private int[] topoSortStates(Automaton a) {
int[] states = new int[a.getNumStates()];
final Set<Integer> visited = new HashSet<>();

View File

@ -49,6 +49,8 @@ import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FSDirectory;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.CharsRefBuilder;
@ -210,6 +212,15 @@ public class FreeTextSuggester extends Lookup {
return fst.ramBytesUsed();
}
@Override
public Iterable<? extends Accountable> getChildResources() {
if (fst == null) {
return Collections.emptyList();
} else {
return Collections.singletonList(Accountables.namedAccountable("fst", fst));
}
}
private static class AnalyzingComparator implements Comparator<BytesRef> {
private final ByteArrayDataInput readerA = new ByteArrayDataInput();

View File

@ -33,6 +33,7 @@ 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.Accountables;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
@ -313,6 +314,18 @@ public class FSTCompletionLookup extends Lookup implements Accountable {
return mem;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
List<Accountable> resources = new ArrayList<>();
if (normalCompletion != null) {
resources.add(Accountables.namedAccountable("fst", normalCompletion.getFST()));
}
if (higherWeightsCompletion != null && (normalCompletion == null || normalCompletion.getFST() != higherWeightsCompletion.getFST())) {
resources.add(Accountables.namedAccountable("higher weights fst", higherWeightsCompletion.getFST()));
}
return resources;
}
@Override
public long getCount() {
return count;

View File

@ -31,6 +31,8 @@ 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.Accountables;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
@ -298,6 +300,15 @@ public class WFSTCompletionLookup extends Lookup {
return (fst == null) ? 0 : fst.ramBytesUsed();
}
@Override
public Iterable<? extends Accountable> getChildResources() {
if (fst == null) {
return Collections.emptyList();
} else {
return Collections.singleton(Accountables.namedAccountable("fst", fst));
}
}
@Override
public long getCount() {
return count;

View File

@ -32,6 +32,7 @@ package org.apache.lucene.search.suggest.jaspell;
import java.io.BufferedReader;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.Collections;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.List;
@ -110,7 +111,11 @@ public class JaspellTernarySearchTrie implements Accountable {
}
return mem;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
}
/**
@ -898,5 +903,9 @@ public class JaspellTernarySearchTrie implements Accountable {
}
return mem;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
}

View File

@ -19,7 +19,6 @@ package org.apache.lucene.codecs.asserting;
import java.io.IOException;
import java.util.Iterator;
import java.util.NoSuchElementException;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesFormat;
@ -34,10 +33,12 @@ import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.SortedNumericDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.LongBitSet;
import org.apache.lucene.util.TestUtil;
/**
* Just like {@link Lucene410DocValuesFormat} but with additional asserts.
@ -80,7 +81,7 @@ public class AssertingDocValuesFormat extends DocValuesFormat {
count++;
}
assert count == maxDoc;
checkIterator(values.iterator(), maxDoc, true);
TestUtil.checkIterator(values.iterator(), maxDoc, true);
in.addNumericField(field, values);
}
@ -92,7 +93,7 @@ public class AssertingDocValuesFormat extends DocValuesFormat {
count++;
}
assert count == maxDoc;
checkIterator(values.iterator(), maxDoc, true);
TestUtil.checkIterator(values.iterator(), maxDoc, true);
in.addBinaryField(field, values);
}
@ -126,8 +127,8 @@ public class AssertingDocValuesFormat extends DocValuesFormat {
assert count == maxDoc;
assert seenOrds.cardinality() == valueCount;
checkIterator(values.iterator(), valueCount, false);
checkIterator(docToOrd.iterator(), maxDoc, false);
TestUtil.checkIterator(values.iterator(), valueCount, false);
TestUtil.checkIterator(docToOrd.iterator(), maxDoc, false);
in.addSortedField(field, values, docToOrd);
}
@ -150,8 +151,8 @@ public class AssertingDocValuesFormat extends DocValuesFormat {
}
}
assert valueIterator.hasNext() == false;
checkIterator(docToValueCount.iterator(), maxDoc, false);
checkIterator(values.iterator(), valueCount, false);
TestUtil.checkIterator(docToValueCount.iterator(), maxDoc, false);
TestUtil.checkIterator(values.iterator(), valueCount, false);
in.addSortedNumericField(field, docToValueCount, values);
}
@ -195,9 +196,9 @@ public class AssertingDocValuesFormat extends DocValuesFormat {
assert docCount == maxDoc;
assert seenOrds.cardinality() == valueCount;
checkIterator(values.iterator(), valueCount, false);
checkIterator(docToOrdCount.iterator(), maxDoc, false);
checkIterator(ords.iterator(), ordCount, false);
TestUtil.checkIterator(values.iterator(), valueCount, false);
TestUtil.checkIterator(docToOrdCount.iterator(), maxDoc, false);
TestUtil.checkIterator(ords.iterator(), ordCount, false);
in.addSortedSetField(field, values, docToOrdCount, ords);
}
@ -207,28 +208,6 @@ public class AssertingDocValuesFormat extends DocValuesFormat {
}
}
static <T> void checkIterator(Iterator<T> iterator, long expectedSize, boolean allowNull) {
for (long i = 0; i < expectedSize; i++) {
boolean hasNext = iterator.hasNext();
assert hasNext;
T v = iterator.next();
assert allowNull || v != null;
try {
iterator.remove();
throw new AssertionError("broken iterator (supports remove): " + iterator);
} catch (UnsupportedOperationException expected) {
// ok
}
}
assert !iterator.hasNext();
try {
iterator.next();
throw new AssertionError("broken iterator (allows next() when hasNext==false) " + iterator);
} catch (NoSuchElementException expected) {
// ok
}
}
static class AssertingDocValuesProducer extends DocValuesProducer {
private final DocValuesProducer in;
private final int maxDoc;
@ -236,6 +215,10 @@ public class AssertingDocValuesFormat extends DocValuesFormat {
AssertingDocValuesProducer(DocValuesProducer in, int maxDoc) {
this.in = in;
this.maxDoc = maxDoc;
// do a few simple checks on init
assert toString() != null;
assert ramBytesUsed() >= 0;
assert getChildResources() != null;
}
@Override
@ -294,12 +277,26 @@ public class AssertingDocValuesFormat extends DocValuesFormat {
@Override
public long ramBytesUsed() {
return in.ramBytesUsed();
long v = in.ramBytesUsed();
assert v >= 0;
return v;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
Iterable<? extends Accountable> res = in.getChildResources();
TestUtil.checkIterator(res.iterator());
return res;
}
@Override
public void checkIntegrity() throws IOException {
in.checkIntegrity();
}
@Override
public String toString() {
return getClass().getSimpleName() + "(" + in.toString() + ")";
}
}
}

View File

@ -28,6 +28,8 @@ import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.TestUtil;
/**
* Just like {@link Lucene49NormsFormat} but with additional asserts.
@ -67,7 +69,7 @@ public class AssertingNormsFormat extends NormsFormat {
count++;
}
assert count == maxDoc;
AssertingDocValuesFormat.checkIterator(values.iterator(), maxDoc, false);
TestUtil.checkIterator(values.iterator(), maxDoc, false);
in.addNormsField(field, values);
}
@ -84,6 +86,10 @@ public class AssertingNormsFormat extends NormsFormat {
AssertingNormsProducer(NormsProducer in, int maxDoc) {
this.in = in;
this.maxDoc = maxDoc;
// do a few simple checks on init
assert toString() != null;
assert ramBytesUsed() >= 0;
assert getChildResources() != null;
}
@Override
@ -101,12 +107,26 @@ public class AssertingNormsFormat extends NormsFormat {
@Override
public long ramBytesUsed() {
return in.ramBytesUsed();
long v = in.ramBytesUsed();
assert v >= 0;
return v;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
Iterable<? extends Accountable> res = in.getChildResources();
TestUtil.checkIterator(res.iterator());
return res;
}
@Override
public void checkIntegrity() throws IOException {
in.checkIntegrity();
}
@Override
public String toString() {
return getClass().getSimpleName() + "(" + in.toString() + ")";
}
}
}

View File

@ -33,8 +33,10 @@ import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.TestUtil;
/**
* Just like {@link Lucene41PostingsFormat} but with additional asserts.
@ -61,6 +63,10 @@ public final class AssertingPostingsFormat extends PostingsFormat {
AssertingFieldsProducer(FieldsProducer in) {
this.in = in;
// do a few simple checks on init
assert toString() != null;
assert ramBytesUsed() >= 0;
assert getChildResources() != null;
}
@Override
@ -88,13 +94,27 @@ public final class AssertingPostingsFormat extends PostingsFormat {
@Override
public long ramBytesUsed() {
return in.ramBytesUsed();
long v = in.ramBytesUsed();
assert v >= 0;
return v;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
Iterable<? extends Accountable> res = in.getChildResources();
TestUtil.checkIterator(res.iterator());
return res;
}
@Override
public void checkIntegrity() throws IOException {
in.checkIntegrity();
}
@Override
public String toString() {
return getClass().getSimpleName() + "(" + in.toString() + ")";
}
}
static class AssertingFieldsConsumer extends FieldsConsumer {

View File

@ -30,6 +30,8 @@ import org.apache.lucene.index.StorableField;
import org.apache.lucene.index.StoredFieldVisitor;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.TestUtil;
/**
* Just like {@link Lucene41StoredFieldsFormat} but with additional asserts.
@ -54,6 +56,10 @@ public class AssertingStoredFieldsFormat extends StoredFieldsFormat {
AssertingStoredFieldsReader(StoredFieldsReader in, int maxDoc) {
this.in = in;
this.maxDoc = maxDoc;
// do a few simple checks on init
assert toString() != null;
assert ramBytesUsed() >= 0;
assert getChildResources() != null;
}
@Override
@ -74,13 +80,27 @@ public class AssertingStoredFieldsFormat extends StoredFieldsFormat {
@Override
public long ramBytesUsed() {
return in.ramBytesUsed();
long v = in.ramBytesUsed();
assert v >= 0;
return v;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
Iterable<? extends Accountable> res = in.getChildResources();
TestUtil.checkIterator(res.iterator());
return res;
}
@Override
public void checkIntegrity() throws IOException {
in.checkIntegrity();
}
@Override
public String toString() {
return getClass().getSimpleName() + "(" + in.toString() + ")";
}
}
enum Status {

View File

@ -30,7 +30,9 @@ import org.apache.lucene.index.Fields;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.TestUtil;
/**
* Just like {@link Lucene42TermVectorsFormat} but with additional asserts.
@ -53,6 +55,10 @@ public class AssertingTermVectorsFormat extends TermVectorsFormat {
AssertingTermVectorsReader(TermVectorsReader in) {
this.in = in;
// do a few simple checks on init
assert toString() != null;
assert ramBytesUsed() >= 0;
assert getChildResources() != null;
}
@Override
@ -73,13 +79,27 @@ public class AssertingTermVectorsFormat extends TermVectorsFormat {
@Override
public long ramBytesUsed() {
return in.ramBytesUsed();
long v = in.ramBytesUsed();
assert v >= 0;
return v;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
Iterable<? extends Accountable> res = in.getChildResources();
TestUtil.checkIterator(res.iterator());
return res;
}
@Override
public void checkIntegrity() throws IOException {
in.checkIntegrity();
}
@Override
public String toString() {
return getClass().getSimpleName() + "(" + in.toString() + ")";
}
}
enum Status {

View File

@ -46,6 +46,7 @@ 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.Accountables;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.FixedBitSet;
@ -95,6 +96,11 @@ public final class RAMOnlyPostingsFormat extends PostingsFormat {
}
return sizeInBytes;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Accountables.namedAccountables("field", fieldToTerms);
}
@Override
public void checkIntegrity() throws IOException {}
@ -122,6 +128,11 @@ public final class RAMOnlyPostingsFormat extends PostingsFormat {
return sizeInBytes;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
@Override
public long size() {
return termToDocs.size();
@ -184,6 +195,11 @@ public final class RAMOnlyPostingsFormat extends PostingsFormat {
}
return sizeInBytes;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
}
static class RAMDoc implements Accountable {
@ -208,6 +224,11 @@ public final class RAMOnlyPostingsFormat extends PostingsFormat {
}
return sizeInBytes;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
}
// Classes for writing to the postings state

View File

@ -29,8 +29,10 @@ import java.nio.file.Files;
import java.nio.file.Path;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.NoSuchElementException;
import java.util.Random;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
@ -67,6 +69,7 @@ import org.apache.lucene.index.ConcurrentMergeScheduler;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.index.FilterAtomicReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexableField;
@ -74,6 +77,7 @@ import org.apache.lucene.index.LogMergePolicy;
import org.apache.lucene.index.MergePolicy;
import org.apache.lucene.index.MergeScheduler;
import org.apache.lucene.index.MultiFields;
import org.apache.lucene.index.SegmentReader;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.index.TieredMergePolicy;
@ -127,6 +131,66 @@ public final class TestUtil {
}
}
/**
* Checks that the provided iterator is well-formed.
* <ul>
* <li>is read-only: does not allow {@code remove}
* <li>returns {@code expectedSize} number of elements
* <li>does not return null elements, unless {@code allowNull} is true.
* <li>throws NoSuchElementException if {@code next} is called
* after {@code hasNext} returns false.
* </ul>
*/
public static <T> void checkIterator(Iterator<T> iterator, long expectedSize, boolean allowNull) {
for (long i = 0; i < expectedSize; i++) {
boolean hasNext = iterator.hasNext();
assert hasNext;
T v = iterator.next();
assert allowNull || v != null;
try {
iterator.remove();
throw new AssertionError("broken iterator (supports remove): " + iterator);
} catch (UnsupportedOperationException expected) {
// ok
}
}
assert !iterator.hasNext();
try {
iterator.next();
throw new AssertionError("broken iterator (allows next() when hasNext==false) " + iterator);
} catch (NoSuchElementException expected) {
// ok
}
}
/**
* Checks that the provided iterator is well-formed.
* <ul>
* <li>is read-only: does not allow {@code remove}
* <li>does not return null elements.
* <li>throws NoSuchElementException if {@code next} is called
* after {@code hasNext} returns false.
* </ul>
*/
public static <T> void checkIterator(Iterator<T> iterator) {
while (iterator.hasNext()) {
T v = iterator.next();
assert v != null;
try {
iterator.remove();
throw new AssertionError("broken iterator (supports remove): " + iterator);
} catch (UnsupportedOperationException expected) {
// ok
}
}
try {
iterator.next();
throw new AssertionError("broken iterator (allows next() when hasNext==false) " + iterator);
} catch (NoSuchElementException expected) {
// ok
}
}
public static void syncConcurrentMerges(IndexWriter writer) {
syncConcurrentMerges(writer.getConfig().getMergeScheduler());
}
@ -190,6 +254,16 @@ public final class TestUtil {
if (LuceneTestCase.INFOSTREAM) {
System.out.println(bos.toString(IOUtils.UTF_8));
}
AtomicReader unwrapped = FilterAtomicReader.unwrap(reader);
if (unwrapped instanceof SegmentReader) {
SegmentReader sr = (SegmentReader) unwrapped;
long bytesUsed = sr.ramBytesUsed();
if (sr.ramBytesUsed() < 0) {
throw new IllegalStateException("invalid ramBytesUsed for reader: " + bytesUsed);
}
assert Accountables.toString(sr) != null;
}
}
/** start and end are BOTH inclusive */

View File

@ -33,6 +33,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.Accountables;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CharsRef;
import org.apache.solr.common.SolrException;
@ -340,6 +341,11 @@ public class SuggestComponent extends SearchComponent implements SolrCoreAware,
return sizeInBytes;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Accountables.namedAccountables("field", suggesters);
}
private Set<SolrSuggester> getSuggesters(SolrParams params) {
Set<SolrSuggester> solrSuggesters = new HashSet<>();
for(String suggesterName : getSuggesterNames(params)) {

View File

@ -22,6 +22,7 @@ import java.io.File;
import java.io.FileInputStream;
import java.io.FileOutputStream;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
import org.apache.lucene.search.spell.Dictionary;
@ -211,6 +212,11 @@ public class SolrSuggester implements Accountable {
return lookup.ramBytesUsed();
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return lookup.getChildResources();
}
@Override
public String toString() {
return "SolrSuggester [ name=" + name + ", "