LUCENE-5373: Fix memory usage estimation for [Lucene40/Lucene42/Memory/Direct]DocValuesProducer.

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1552751 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Adrien Grand 2013-12-20 17:26:51 +00:00
parent ad6c85ef96
commit 533777f620
5 changed files with 125 additions and 84 deletions

View File

@ -104,6 +104,10 @@ Bug fixes
* LUCENE-5285: Improved highlighting of multi-valued fields with
FastVectorHighlighter. (Nik Everett via Adrien Grand)
* LUCENE-5373: Memory usage of
[Lucene40/Lucene42/Memory/Direct]DocValuesFormat was over-estimated.
(Shay Banon, Adrien Grand, Robert Muir)
* LUCENE-5374: IndexWriter processes internal events after the it
closed itself internally. This rare condition can happen if an
IndexWriter has internal changes that were not fully applied yet

View File

@ -20,6 +20,7 @@ package org.apache.lucene.codecs.memory;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.DocValuesProducer;
@ -62,6 +63,7 @@ class DirectDocValuesProducer extends DocValuesProducer {
private final Map<Integer,Bits> docsWithFieldInstances = new HashMap<Integer,Bits>();
private final int maxDoc;
private final AtomicLong ramBytesUsed;
static final byte NUMBER = 0;
static final byte BYTES = 1;
@ -76,6 +78,7 @@ class DirectDocValuesProducer extends DocValuesProducer {
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
// read in the entries from the metadata file.
IndexInput in = state.directory.openInput(metaName, state.context);
ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOfInstance(getClass()));
boolean success = false;
final int version;
try {
@ -178,8 +181,7 @@ class DirectDocValuesProducer extends DocValuesProducer {
@Override
public long ramBytesUsed() {
// TODO: optimize me
return RamUsageEstimator.sizeOf(this);
return ramBytesUsed.get();
}
@Override
@ -199,9 +201,8 @@ class DirectDocValuesProducer extends DocValuesProducer {
case 1:
{
final byte[] values = new byte[entry.count];
for(int i=0;i<entry.count;i++) {
values[i] = data.readByte();
}
data.readBytes(values, 0, entry.count);
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(values));
return new NumericDocValues() {
@Override
public long get(int idx) {
@ -216,6 +217,7 @@ class DirectDocValuesProducer extends DocValuesProducer {
for(int i=0;i<entry.count;i++) {
values[i] = data.readShort();
}
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(values));
return new NumericDocValues() {
@Override
public long get(int idx) {
@ -230,6 +232,7 @@ class DirectDocValuesProducer extends DocValuesProducer {
for(int i=0;i<entry.count;i++) {
values[i] = data.readInt();
}
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(values));
return new NumericDocValues() {
@Override
public long get(int idx) {
@ -244,6 +247,7 @@ class DirectDocValuesProducer extends DocValuesProducer {
for(int i=0;i<entry.count;i++) {
values[i] = data.readLong();
}
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(values));
return new NumericDocValues() {
@Override
public long get(int idx) {
@ -280,6 +284,8 @@ class DirectDocValuesProducer extends DocValuesProducer {
}
address[entry.count] = data.readInt();
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(bytes) + RamUsageEstimator.sizeOf(address));
return new BinaryDocValues() {
@Override
public void get(int docID, BytesRef result) {

View File

@ -20,6 +20,7 @@ package org.apache.lucene.codecs.memory;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.DocValuesProducer;
@ -75,7 +76,7 @@ class MemoryDocValuesProducer extends DocValuesProducer {
private final Map<Integer,Bits> docsWithFieldInstances = new HashMap<Integer,Bits>();
private final int maxDoc;
private final AtomicLong ramBytesUsed;
static final byte NUMBER = 0;
static final byte BYTES = 1;
@ -107,7 +108,7 @@ class MemoryDocValuesProducer extends DocValuesProducer {
binaries = new HashMap<Integer,BinaryEntry>();
fsts = new HashMap<Integer,FSTEntry>();
readFields(in, state.fieldInfos);
ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOfInstance(getClass()));
success = true;
} finally {
if (success) {
@ -204,8 +205,7 @@ class MemoryDocValuesProducer extends DocValuesProducer {
@Override
public long ramBytesUsed() {
// TODO: optimize me
return RamUsageEstimator.sizeOf(this);
return ramBytesUsed.get();
}
private NumericDocValues loadNumeric(FieldInfo field) throws IOException {
@ -224,6 +224,7 @@ class MemoryDocValuesProducer extends DocValuesProducer {
final int formatID = data.readVInt();
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());
return new NumericDocValues() {
@Override
public long get(int docID) {
@ -233,10 +234,12 @@ class MemoryDocValuesProducer extends DocValuesProducer {
case DELTA_COMPRESSED:
final int blockSize = data.readVInt();
final BlockPackedReader reader = new BlockPackedReader(data, entry.packedIntsVersion, blockSize, maxDoc, false);
ramBytesUsed.addAndGet(reader.ramBytesUsed());
return reader;
case UNCOMPRESSED:
final byte bytes[] = new byte[maxDoc];
data.readBytes(bytes, 0, bytes.length);
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(bytes));
return new NumericDocValues() {
@Override
public long get(int docID) {
@ -248,6 +251,7 @@ class MemoryDocValuesProducer extends DocValuesProducer {
final long mult = data.readLong();
final int quotientBlockSize = data.readVInt();
final BlockPackedReader quotientReader = new BlockPackedReader(data, entry.packedIntsVersion, quotientBlockSize, maxDoc, false);
ramBytesUsed.addAndGet(quotientReader.ramBytesUsed());
return new NumericDocValues() {
@Override
public long get(int docID) {
@ -277,6 +281,7 @@ class MemoryDocValuesProducer extends DocValuesProducer {
final PagedBytes.Reader bytesReader = bytes.freeze(true);
if (entry.minLength == entry.maxLength) {
final int fixedLength = entry.minLength;
ramBytesUsed.addAndGet(bytes.ramBytesUsed());
return new BinaryDocValues() {
@Override
public void get(int docID, BytesRef result) {
@ -286,6 +291,7 @@ class MemoryDocValuesProducer extends DocValuesProducer {
} else {
data.seek(data.getFilePointer() + entry.missingBytes);
final MonotonicBlockPackedReader addresses = new MonotonicBlockPackedReader(data, entry.packedIntsVersion, entry.blockSize, maxDoc, false);
ramBytesUsed.addAndGet(bytes.ramBytesUsed() + addresses.ramBytesUsed());
return new BinaryDocValues() {
@Override
public void get(int docID, BytesRef result) {
@ -309,6 +315,7 @@ class MemoryDocValuesProducer extends DocValuesProducer {
if (instance == null) {
data.seek(entry.offset);
instance = new FST<Long>(data, PositiveIntOutputs.getSingleton());
ramBytesUsed.addAndGet(instance.sizeInBytes());
fstInstances.put(field.number, instance);
}
}
@ -383,6 +390,7 @@ class MemoryDocValuesProducer extends DocValuesProducer {
if (instance == null) {
data.seek(entry.offset);
instance = new FST<Long>(data, PositiveIntOutputs.getSingleton());
ramBytesUsed.addAndGet(instance.sizeInBytes());
fstInstances.put(field.number, instance);
}
}

View File

@ -20,6 +20,7 @@ package org.apache.lucene.codecs.lucene40;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.DocValuesProducer;
@ -55,19 +56,22 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
private static final String segmentSuffix = "dv";
// ram instances we have already loaded
private final Map<Integer,NumericDocValues> numericInstances =
private final Map<Integer,NumericDocValues> numericInstances =
new HashMap<Integer,NumericDocValues>();
private final Map<Integer,BinaryDocValues> binaryInstances =
private final Map<Integer,BinaryDocValues> binaryInstances =
new HashMap<Integer,BinaryDocValues>();
private final Map<Integer,SortedDocValues> sortedInstances =
private final Map<Integer,SortedDocValues> sortedInstances =
new HashMap<Integer,SortedDocValues>();
private final AtomicLong ramBytesUsed;
Lucene40DocValuesReader(SegmentReadState state, String filename, String legacyKey) throws IOException {
this.state = state;
this.legacyKey = legacyKey;
this.dir = new CompoundFileDirectory(state.directory, filename, state.context, false);
ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOf(getClass()));
}
@Override
public synchronized NumericDocValues getNumeric(FieldInfo field) throws IOException {
NumericDocValues instance = numericInstances.get(field.number);
@ -98,7 +102,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
case FLOAT_64:
instance = loadDoubleField(field, input);
break;
default:
default:
throw new AssertionError();
}
if (input.getFilePointer() != input.length()) {
@ -116,10 +120,10 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
}
return instance;
}
private NumericDocValues loadVarIntsField(FieldInfo field, IndexInput input) throws IOException {
CodecUtil.checkHeader(input, Lucene40DocValuesFormat.VAR_INTS_CODEC_NAME,
Lucene40DocValuesFormat.VAR_INTS_VERSION_START,
CodecUtil.checkHeader(input, Lucene40DocValuesFormat.VAR_INTS_CODEC_NAME,
Lucene40DocValuesFormat.VAR_INTS_VERSION_START,
Lucene40DocValuesFormat.VAR_INTS_VERSION_CURRENT);
byte header = input.readByte();
if (header == Lucene40DocValuesFormat.VAR_INTS_FIXED_64) {
@ -128,6 +132,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
for (int i = 0; i < values.length; i++) {
values[i] = input.readLong();
}
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(values));
return new NumericDocValues() {
@Override
public long get(int docID) {
@ -138,6 +143,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
final long minValue = input.readLong();
final long defaultValue = input.readLong();
final PackedInts.Reader reader = PackedInts.getReader(input);
ramBytesUsed.addAndGet(reader.ramBytesUsed());
return new NumericDocValues() {
@Override
public long get(int docID) {
@ -153,10 +159,10 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
throw new CorruptIndexException("invalid VAR_INTS header byte: " + header + " (resource=" + input + ")");
}
}
private NumericDocValues loadByteField(FieldInfo field, IndexInput input) throws IOException {
CodecUtil.checkHeader(input, Lucene40DocValuesFormat.INTS_CODEC_NAME,
Lucene40DocValuesFormat.INTS_VERSION_START,
CodecUtil.checkHeader(input, Lucene40DocValuesFormat.INTS_CODEC_NAME,
Lucene40DocValuesFormat.INTS_VERSION_START,
Lucene40DocValuesFormat.INTS_VERSION_CURRENT);
int valueSize = input.readInt();
if (valueSize != 1) {
@ -165,6 +171,7 @@ 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));
return new NumericDocValues() {
@Override
public long get(int docID) {
@ -172,10 +179,10 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
}
};
}
private NumericDocValues loadShortField(FieldInfo field, IndexInput input) throws IOException {
CodecUtil.checkHeader(input, Lucene40DocValuesFormat.INTS_CODEC_NAME,
Lucene40DocValuesFormat.INTS_VERSION_START,
CodecUtil.checkHeader(input, Lucene40DocValuesFormat.INTS_CODEC_NAME,
Lucene40DocValuesFormat.INTS_VERSION_START,
Lucene40DocValuesFormat.INTS_VERSION_CURRENT);
int valueSize = input.readInt();
if (valueSize != 2) {
@ -186,6 +193,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
for (int i = 0; i < values.length; i++) {
values[i] = input.readShort();
}
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(values));
return new NumericDocValues() {
@Override
public long get(int docID) {
@ -193,10 +201,10 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
}
};
}
private NumericDocValues loadIntField(FieldInfo field, IndexInput input) throws IOException {
CodecUtil.checkHeader(input, Lucene40DocValuesFormat.INTS_CODEC_NAME,
Lucene40DocValuesFormat.INTS_VERSION_START,
CodecUtil.checkHeader(input, Lucene40DocValuesFormat.INTS_CODEC_NAME,
Lucene40DocValuesFormat.INTS_VERSION_START,
Lucene40DocValuesFormat.INTS_VERSION_CURRENT);
int valueSize = input.readInt();
if (valueSize != 4) {
@ -207,6 +215,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
for (int i = 0; i < values.length; i++) {
values[i] = input.readInt();
}
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(values));
return new NumericDocValues() {
@Override
public long get(int docID) {
@ -214,10 +223,10 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
}
};
}
private NumericDocValues loadLongField(FieldInfo field, IndexInput input) throws IOException {
CodecUtil.checkHeader(input, Lucene40DocValuesFormat.INTS_CODEC_NAME,
Lucene40DocValuesFormat.INTS_VERSION_START,
CodecUtil.checkHeader(input, Lucene40DocValuesFormat.INTS_CODEC_NAME,
Lucene40DocValuesFormat.INTS_VERSION_START,
Lucene40DocValuesFormat.INTS_VERSION_CURRENT);
int valueSize = input.readInt();
if (valueSize != 8) {
@ -228,6 +237,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
for (int i = 0; i < values.length; i++) {
values[i] = input.readLong();
}
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(values));
return new NumericDocValues() {
@Override
public long get(int docID) {
@ -235,10 +245,10 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
}
};
}
private NumericDocValues loadFloatField(FieldInfo field, IndexInput input) throws IOException {
CodecUtil.checkHeader(input, Lucene40DocValuesFormat.FLOATS_CODEC_NAME,
Lucene40DocValuesFormat.FLOATS_VERSION_START,
CodecUtil.checkHeader(input, Lucene40DocValuesFormat.FLOATS_CODEC_NAME,
Lucene40DocValuesFormat.FLOATS_VERSION_START,
Lucene40DocValuesFormat.FLOATS_VERSION_CURRENT);
int valueSize = input.readInt();
if (valueSize != 4) {
@ -249,6 +259,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
for (int i = 0; i < values.length; i++) {
values[i] = input.readInt();
}
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(values));
return new NumericDocValues() {
@Override
public long get(int docID) {
@ -256,10 +267,10 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
}
};
}
private NumericDocValues loadDoubleField(FieldInfo field, IndexInput input) throws IOException {
CodecUtil.checkHeader(input, Lucene40DocValuesFormat.FLOATS_CODEC_NAME,
Lucene40DocValuesFormat.FLOATS_VERSION_START,
CodecUtil.checkHeader(input, Lucene40DocValuesFormat.FLOATS_CODEC_NAME,
Lucene40DocValuesFormat.FLOATS_VERSION_START,
Lucene40DocValuesFormat.FLOATS_VERSION_CURRENT);
int valueSize = input.readInt();
if (valueSize != 8) {
@ -270,6 +281,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
for (int i = 0; i < values.length; i++) {
values[i] = input.readLong();
}
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(values));
return new NumericDocValues() {
@Override
public long get(int docID) {
@ -302,14 +314,14 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
}
return instance;
}
private BinaryDocValues loadBytesFixedStraight(FieldInfo field) throws IOException {
String fileName = IndexFileNames.segmentFileName(state.segmentInfo.name + "_" + Integer.toString(field.number), segmentSuffix, "dat");
IndexInput input = dir.openInput(fileName, state.context);
boolean success = false;
try {
CodecUtil.checkHeader(input, Lucene40DocValuesFormat.BYTES_FIXED_STRAIGHT_CODEC_NAME,
Lucene40DocValuesFormat.BYTES_FIXED_STRAIGHT_VERSION_START,
CodecUtil.checkHeader(input, Lucene40DocValuesFormat.BYTES_FIXED_STRAIGHT_CODEC_NAME,
Lucene40DocValuesFormat.BYTES_FIXED_STRAIGHT_VERSION_START,
Lucene40DocValuesFormat.BYTES_FIXED_STRAIGHT_VERSION_CURRENT);
final int fixedLength = input.readInt();
PagedBytes bytes = new PagedBytes(16);
@ -319,6 +331,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
throw new CorruptIndexException("did not read all bytes from file \"" + fileName + "\": read " + input.getFilePointer() + " vs size " + input.length() + " (resource: " + input + ")");
}
success = true;
ramBytesUsed.addAndGet(bytes.ramBytesUsed());
return new BinaryDocValues() {
@Override
public void get(int docID, BytesRef result) {
@ -333,7 +346,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
}
}
}
private BinaryDocValues loadBytesVarStraight(FieldInfo field) throws IOException {
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");
@ -342,12 +355,12 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
boolean success = false;
try {
data = dir.openInput(dataName, state.context);
CodecUtil.checkHeader(data, Lucene40DocValuesFormat.BYTES_VAR_STRAIGHT_CODEC_NAME_DAT,
Lucene40DocValuesFormat.BYTES_VAR_STRAIGHT_VERSION_START,
CodecUtil.checkHeader(data, Lucene40DocValuesFormat.BYTES_VAR_STRAIGHT_CODEC_NAME_DAT,
Lucene40DocValuesFormat.BYTES_VAR_STRAIGHT_VERSION_START,
Lucene40DocValuesFormat.BYTES_VAR_STRAIGHT_VERSION_CURRENT);
index = dir.openInput(indexName, state.context);
CodecUtil.checkHeader(index, Lucene40DocValuesFormat.BYTES_VAR_STRAIGHT_CODEC_NAME_IDX,
Lucene40DocValuesFormat.BYTES_VAR_STRAIGHT_VERSION_START,
CodecUtil.checkHeader(index, Lucene40DocValuesFormat.BYTES_VAR_STRAIGHT_CODEC_NAME_IDX,
Lucene40DocValuesFormat.BYTES_VAR_STRAIGHT_VERSION_START,
Lucene40DocValuesFormat.BYTES_VAR_STRAIGHT_VERSION_CURRENT);
long totalBytes = index.readVLong();
PagedBytes bytes = new PagedBytes(16);
@ -361,6 +374,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
throw new CorruptIndexException("did not read all bytes from file \"" + indexName + "\": read " + index.getFilePointer() + " vs size " + index.length() + " (resource: " + index + ")");
}
success = true;
ramBytesUsed.addAndGet(bytes.ramBytesUsed() + reader.ramBytesUsed());
return new BinaryDocValues() {
@Override
public void get(int docID, BytesRef result) {
@ -377,7 +391,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
}
}
}
private BinaryDocValues loadBytesFixedDeref(FieldInfo field) throws IOException {
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");
@ -386,14 +400,14 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
boolean success = false;
try {
data = dir.openInput(dataName, state.context);
CodecUtil.checkHeader(data, Lucene40DocValuesFormat.BYTES_FIXED_DEREF_CODEC_NAME_DAT,
Lucene40DocValuesFormat.BYTES_FIXED_DEREF_VERSION_START,
CodecUtil.checkHeader(data, Lucene40DocValuesFormat.BYTES_FIXED_DEREF_CODEC_NAME_DAT,
Lucene40DocValuesFormat.BYTES_FIXED_DEREF_VERSION_START,
Lucene40DocValuesFormat.BYTES_FIXED_DEREF_VERSION_CURRENT);
index = dir.openInput(indexName, state.context);
CodecUtil.checkHeader(index, Lucene40DocValuesFormat.BYTES_FIXED_DEREF_CODEC_NAME_IDX,
Lucene40DocValuesFormat.BYTES_FIXED_DEREF_VERSION_START,
CodecUtil.checkHeader(index, Lucene40DocValuesFormat.BYTES_FIXED_DEREF_CODEC_NAME_IDX,
Lucene40DocValuesFormat.BYTES_FIXED_DEREF_VERSION_START,
Lucene40DocValuesFormat.BYTES_FIXED_DEREF_VERSION_CURRENT);
final int fixedLength = data.readInt();
final int valueCount = index.readInt();
PagedBytes bytes = new PagedBytes(16);
@ -406,6 +420,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
if (index.getFilePointer() != index.length()) {
throw new CorruptIndexException("did not read all bytes from file \"" + indexName + "\": read " + index.getFilePointer() + " vs size " + index.length() + " (resource: " + index + ")");
}
ramBytesUsed.addAndGet(bytes.ramBytesUsed() + reader.ramBytesUsed());
success = true;
return new BinaryDocValues() {
@Override
@ -422,7 +437,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
}
}
}
private BinaryDocValues loadBytesVarDeref(FieldInfo field) throws IOException {
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");
@ -431,14 +446,14 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
boolean success = false;
try {
data = dir.openInput(dataName, state.context);
CodecUtil.checkHeader(data, Lucene40DocValuesFormat.BYTES_VAR_DEREF_CODEC_NAME_DAT,
Lucene40DocValuesFormat.BYTES_VAR_DEREF_VERSION_START,
CodecUtil.checkHeader(data, Lucene40DocValuesFormat.BYTES_VAR_DEREF_CODEC_NAME_DAT,
Lucene40DocValuesFormat.BYTES_VAR_DEREF_VERSION_START,
Lucene40DocValuesFormat.BYTES_VAR_DEREF_VERSION_CURRENT);
index = dir.openInput(indexName, state.context);
CodecUtil.checkHeader(index, Lucene40DocValuesFormat.BYTES_VAR_DEREF_CODEC_NAME_IDX,
Lucene40DocValuesFormat.BYTES_VAR_DEREF_VERSION_START,
CodecUtil.checkHeader(index, Lucene40DocValuesFormat.BYTES_VAR_DEREF_CODEC_NAME_IDX,
Lucene40DocValuesFormat.BYTES_VAR_DEREF_VERSION_START,
Lucene40DocValuesFormat.BYTES_VAR_DEREF_VERSION_CURRENT);
final long totalBytes = index.readLong();
final PagedBytes bytes = new PagedBytes(16);
bytes.copy(data, totalBytes);
@ -450,6 +465,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
if (index.getFilePointer() != index.length()) {
throw new CorruptIndexException("did not read all bytes from file \"" + indexName + "\": read " + index.getFilePointer() + " vs size " + index.length() + " (resource: " + index + ")");
}
ramBytesUsed.addAndGet(bytes.ramBytesUsed() + reader.ramBytesUsed());
success = true;
return new BinaryDocValues() {
@Override
@ -517,23 +533,24 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
}
return instance;
}
private SortedDocValues loadBytesFixedSorted(FieldInfo field, IndexInput data, IndexInput index) throws IOException {
CodecUtil.checkHeader(data, Lucene40DocValuesFormat.BYTES_FIXED_SORTED_CODEC_NAME_DAT,
Lucene40DocValuesFormat.BYTES_FIXED_SORTED_VERSION_START,
CodecUtil.checkHeader(data, Lucene40DocValuesFormat.BYTES_FIXED_SORTED_CODEC_NAME_DAT,
Lucene40DocValuesFormat.BYTES_FIXED_SORTED_VERSION_START,
Lucene40DocValuesFormat.BYTES_FIXED_SORTED_VERSION_CURRENT);
CodecUtil.checkHeader(index, Lucene40DocValuesFormat.BYTES_FIXED_SORTED_CODEC_NAME_IDX,
Lucene40DocValuesFormat.BYTES_FIXED_SORTED_VERSION_START,
CodecUtil.checkHeader(index, Lucene40DocValuesFormat.BYTES_FIXED_SORTED_CODEC_NAME_IDX,
Lucene40DocValuesFormat.BYTES_FIXED_SORTED_VERSION_START,
Lucene40DocValuesFormat.BYTES_FIXED_SORTED_VERSION_CURRENT);
final int fixedLength = data.readInt();
final int valueCount = index.readInt();
PagedBytes bytes = new PagedBytes(16);
bytes.copy(data, fixedLength * (long) valueCount);
final PagedBytes.Reader bytesReader = bytes.freeze(true);
final PackedInts.Reader reader = PackedInts.getReader(index);
ramBytesUsed.addAndGet(bytes.ramBytesUsed() + reader.ramBytesUsed());
return correctBuggyOrds(new SortedDocValues() {
@Override
public int getOrd(int docID) {
@ -551,24 +568,25 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
}
});
}
private SortedDocValues loadBytesVarSorted(FieldInfo field, IndexInput data, IndexInput index) throws IOException {
CodecUtil.checkHeader(data, Lucene40DocValuesFormat.BYTES_VAR_SORTED_CODEC_NAME_DAT,
Lucene40DocValuesFormat.BYTES_VAR_SORTED_VERSION_START,
CodecUtil.checkHeader(data, Lucene40DocValuesFormat.BYTES_VAR_SORTED_CODEC_NAME_DAT,
Lucene40DocValuesFormat.BYTES_VAR_SORTED_VERSION_START,
Lucene40DocValuesFormat.BYTES_VAR_SORTED_VERSION_CURRENT);
CodecUtil.checkHeader(index, Lucene40DocValuesFormat.BYTES_VAR_SORTED_CODEC_NAME_IDX,
Lucene40DocValuesFormat.BYTES_VAR_SORTED_VERSION_START,
CodecUtil.checkHeader(index, Lucene40DocValuesFormat.BYTES_VAR_SORTED_CODEC_NAME_IDX,
Lucene40DocValuesFormat.BYTES_VAR_SORTED_VERSION_START,
Lucene40DocValuesFormat.BYTES_VAR_SORTED_VERSION_CURRENT);
long maxAddress = index.readLong();
PagedBytes bytes = new PagedBytes(16);
bytes.copy(data, maxAddress);
final PagedBytes.Reader bytesReader = bytes.freeze(true);
final PackedInts.Reader addressReader = PackedInts.getReader(index);
final PackedInts.Reader ordsReader = PackedInts.getReader(index);
final int valueCount = addressReader.size() - 1;
ramBytesUsed.addAndGet(bytes.ramBytesUsed() + addressReader.ramBytesUsed() + ordsReader.ramBytesUsed());
return correctBuggyOrds(new SortedDocValues() {
@Override
public int getOrd(int docID) {
@ -588,7 +606,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
}
});
}
// detects and corrects LUCENE-4717 in old indexes
private SortedDocValues correctBuggyOrds(final SortedDocValues in) {
final int maxDoc = state.segmentInfo.getDocCount();
@ -597,7 +615,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
return in; // ok
}
}
// we had ord holes, return an ord-shifting-impl that corrects the bug
return new SortedDocValues() {
@Override
@ -616,7 +634,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
}
};
}
@Override
public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
throw new IllegalStateException("Lucene 4.0 does not support SortedSet: how did you pull this off?");
@ -634,6 +652,6 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
@Override
public long ramBytesUsed() {
return RamUsageEstimator.sizeOf(this);
return ramBytesUsed.get();
}
}

View File

@ -20,6 +20,7 @@ package org.apache.lucene.codecs.lucene42;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.DocValuesProducer;
@ -73,7 +74,7 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
new HashMap<Integer,FST<Long>>();
private final int maxDoc;
private final AtomicLong ramBytesUsed;
static final byte NUMBER = 0;
static final byte BYTES = 1;
@ -96,6 +97,7 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
// read in the entries from the metadata file.
IndexInput in = state.directory.openInput(metaName, state.context);
boolean success = false;
ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOfInstance(getClass()));
final int version;
try {
version = CodecUtil.checkHeader(in, metaCodec,
@ -190,7 +192,7 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
@Override
public long ramBytesUsed() {
return RamUsageEstimator.sizeOf(this);
return ramBytesUsed.get();
}
private NumericDocValues loadNumeric(FieldInfo field) throws IOException {
@ -209,6 +211,7 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
final int formatID = data.readVInt();
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());
return new NumericDocValues() {
@Override
public long get(int docID) {
@ -218,15 +221,12 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
case DELTA_COMPRESSED:
final int blockSize = data.readVInt();
final BlockPackedReader reader = new BlockPackedReader(data, entry.packedIntsVersion, blockSize, maxDoc, false);
return new NumericDocValues() {
@Override
public long get(int docID) {
return reader.get(docID);
}
};
ramBytesUsed.addAndGet(reader.ramBytesUsed());
return reader;
case UNCOMPRESSED:
final byte bytes[] = new byte[maxDoc];
data.readBytes(bytes, 0, bytes.length);
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(bytes));
return new NumericDocValues() {
@Override
public long get(int docID) {
@ -238,6 +238,7 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
final long mult = data.readLong();
final int quotientBlockSize = data.readVInt();
final BlockPackedReader quotientReader = new BlockPackedReader(data, entry.packedIntsVersion, quotientBlockSize, maxDoc, false);
ramBytesUsed.addAndGet(quotientReader.ramBytesUsed());
return new NumericDocValues() {
@Override
public long get(int docID) {
@ -267,6 +268,7 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
final PagedBytes.Reader bytesReader = bytes.freeze(true);
if (entry.minLength == entry.maxLength) {
final int fixedLength = entry.minLength;
ramBytesUsed.addAndGet(bytes.ramBytesUsed());
return new BinaryDocValues() {
@Override
public void get(int docID, BytesRef result) {
@ -275,6 +277,7 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
};
} else {
final MonotonicBlockPackedReader addresses = new MonotonicBlockPackedReader(data, entry.packedIntsVersion, entry.blockSize, maxDoc, false);
ramBytesUsed.addAndGet(bytes.ramBytesUsed() + addresses.ramBytesUsed());
return new BinaryDocValues() {
@Override
public void get(int docID, BytesRef result) {
@ -295,6 +298,7 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
if (instance == null) {
data.seek(entry.offset);
instance = new FST<Long>(data, PositiveIntOutputs.getSingleton());
ramBytesUsed.addAndGet(instance.sizeInBytes());
fstInstances.put(field.number, instance);
}
}
@ -369,6 +373,7 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
if (instance == null) {
data.seek(entry.offset);
instance = new FST<Long>(data, PositiveIntOutputs.getSingleton());
ramBytesUsed.addAndGet(instance.sizeInBytes());
fstInstances.put(field.number, instance);
}
}