mirror of https://github.com/apache/lucene.git
LUCENE-6104: simplify internals of Lucene50NormsProducer
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1645711 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
199610cd78
commit
765403b2af
|
@ -854,7 +854,7 @@ public class BlockTermsReader extends FieldsProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
if (indexReader != null) {
|
||||
resources.add(Accountables.namedAccountable("term index", indexReader));
|
||||
|
|
|
@ -262,7 +262,7 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
if (termOffsets != null) {
|
||||
resources.add(Accountables.namedAccountable("term lengths", termOffsets));
|
||||
|
@ -303,7 +303,7 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
return Accountables.namedAccountables("field", fields);
|
||||
}
|
||||
|
||||
|
|
|
@ -165,7 +165,7 @@ public class VariableGapTermsIndexReader extends TermsIndexReaderBase {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
if (fst == null) {
|
||||
return Collections.emptyList();
|
||||
} else {
|
||||
|
@ -208,7 +208,7 @@ public class VariableGapTermsIndexReader extends TermsIndexReaderBase {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
return Accountables.namedAccountables("field", fields);
|
||||
}
|
||||
|
||||
|
|
|
@ -234,7 +234,7 @@ public final class OrdsBlockTreeTermsReader extends FieldsProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
resources.addAll(Accountables.namedAccountables("field", fields));
|
||||
resources.add(Accountables.namedAccountable("delegate", postingsReader));
|
||||
|
|
|
@ -174,7 +174,7 @@ final class OrdsFieldReader extends Terms implements Accountable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
if (index == null) {
|
||||
return Collections.emptyList();
|
||||
} else {
|
||||
|
|
|
@ -407,7 +407,7 @@ public final class BloomFilteringPostingsFormat extends PostingsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
resources.addAll(Accountables.namedAccountables("field", bloomsByFieldName));
|
||||
if (delegateFieldsProducer != null) {
|
||||
|
|
|
@ -259,7 +259,7 @@ class DirectDocValuesProducer extends DocValuesProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public synchronized Iterable<? extends Accountable> getChildResources() {
|
||||
public synchronized Iterable<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
resources.addAll(Accountables.namedAccountables("numeric field", numericInstances));
|
||||
resources.addAll(Accountables.namedAccountables("binary field", binaryInstances));
|
||||
|
@ -661,7 +661,7 @@ class DirectDocValuesProducer extends DocValuesProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
if (address != null) {
|
||||
resources.add(Accountables.namedAccountable("addresses", RamUsageEstimator.sizeOf(address)));
|
||||
|
@ -700,7 +700,7 @@ class DirectDocValuesProducer extends DocValuesProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
return docToOrd.getChildResources();
|
||||
}
|
||||
|
||||
|
@ -724,7 +724,7 @@ class DirectDocValuesProducer extends DocValuesProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
if (docToAddress != null) {
|
||||
resources.add(Accountables.namedAccountable("addresses", docToAddress));
|
||||
|
@ -753,7 +753,7 @@ class DirectDocValuesProducer extends DocValuesProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
if (docToOrdAddress != null) {
|
||||
resources.add(Accountables.namedAccountable("addresses", docToOrdAddress));
|
||||
|
|
|
@ -162,7 +162,7 @@ public final class DirectPostingsFormat extends PostingsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
return Accountables.namedAccountables("field", fields);
|
||||
}
|
||||
|
||||
|
|
|
@ -288,7 +288,7 @@ public class FSTOrdTermsReader extends FieldsProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
if (index == null) {
|
||||
return Collections.emptyList();
|
||||
} else {
|
||||
|
@ -865,7 +865,7 @@ public class FSTOrdTermsReader extends FieldsProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
resources.addAll(Accountables.namedAccountables("field", fields));
|
||||
resources.add(Accountables.namedAccountable("delegate", postingsReader));
|
||||
|
|
|
@ -191,7 +191,7 @@ public class FSTTermsReader extends FieldsProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
if (dict == null) {
|
||||
return Collections.emptyList();
|
||||
} else {
|
||||
|
@ -763,7 +763,7 @@ public class FSTTermsReader extends FieldsProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
resources.addAll(Accountables.namedAccountables("field", fields));
|
||||
resources.add(Accountables.namedAccountable("delegate", postingsReader));
|
||||
|
|
|
@ -296,7 +296,7 @@ class MemoryDocValuesProducer extends DocValuesProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public synchronized Iterable<? extends Accountable> getChildResources() {
|
||||
public synchronized Iterable<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
resources.addAll(Accountables.namedAccountables("numeric field", numericInfo));
|
||||
resources.addAll(Accountables.namedAccountables("pagedbytes field", pagedBytesInstances));
|
||||
|
@ -799,7 +799,7 @@ class MemoryDocValuesProducer extends DocValuesProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
if (addresses != null) {
|
||||
resources.add(Accountables.namedAccountable("addresses", addresses));
|
||||
|
|
|
@ -958,7 +958,7 @@ public final class MemoryPostingsFormat extends PostingsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
if (fst == null) {
|
||||
return Collections.emptyList();
|
||||
} else {
|
||||
|
@ -1033,7 +1033,7 @@ public final class MemoryPostingsFormat extends PostingsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
return Accountables.namedAccountables("field", fields);
|
||||
}
|
||||
|
||||
|
|
|
@ -595,7 +595,7 @@ class SimpleTextFieldsReader extends FieldsProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
if (fst == null) {
|
||||
return Collections.emptyList();
|
||||
} else {
|
||||
|
@ -700,7 +700,7 @@ class SimpleTextFieldsReader extends FieldsProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public synchronized Iterable<? extends Accountable> getChildResources() {
|
||||
public synchronized Iterable<Accountable> getChildResources() {
|
||||
return Accountables.namedAccountables("field", termsCache);
|
||||
}
|
||||
|
||||
|
|
|
@ -82,7 +82,7 @@ public class SimpleTextNormsFormat extends NormsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
return impl.getChildResources();
|
||||
}
|
||||
|
||||
|
|
|
@ -281,7 +281,7 @@ public final class BlockTreeTermsReader extends FieldsProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
resources.addAll(Accountables.namedAccountables("field", fields));
|
||||
resources.add(Accountables.namedAccountable("delegate", postingsReader));
|
||||
|
|
|
@ -186,7 +186,7 @@ public final class FieldReader extends Terms implements Accountable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
if (index == null) {
|
||||
return Collections.emptyList();
|
||||
} else {
|
||||
|
|
|
@ -22,6 +22,7 @@ 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;
|
||||
|
@ -188,7 +189,7 @@ public final class CompressingStoredFieldsIndexReader implements Cloneable, Acco
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
|
||||
long docBaseDeltaBytes = RamUsageEstimator.shallowSizeOf(docBasesDeltas);
|
||||
|
@ -203,7 +204,7 @@ public final class CompressingStoredFieldsIndexReader implements Cloneable, Acco
|
|||
}
|
||||
resources.add(Accountables.namedAccountable("start pointer deltas", startPointerDeltaBytes));
|
||||
|
||||
return resources;
|
||||
return Collections.unmodifiableList(resources);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -611,7 +611,7 @@ public final class CompressingStoredFieldsReader extends StoredFieldsReader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
return Collections.singleton(Accountables.namedAccountable("stored field index", indexReader));
|
||||
}
|
||||
|
||||
|
|
|
@ -1074,7 +1074,7 @@ public final class CompressingTermVectorsReader extends TermVectorsReader implem
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
return Collections.singleton(Accountables.namedAccountable("term vector index", indexReader));
|
||||
}
|
||||
|
||||
|
|
|
@ -382,7 +382,7 @@ class Lucene50DocValuesProducer extends DocValuesProducer implements Closeable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public synchronized Iterable<? extends Accountable> getChildResources() {
|
||||
public synchronized Iterable<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
resources.addAll(Accountables.namedAccountables("addresses field", addressInstances));
|
||||
resources.addAll(Accountables.namedAccountables("ord index field", ordIndexInstances));
|
||||
|
@ -872,7 +872,7 @@ class Lucene50DocValuesProducer extends DocValuesProducer implements Closeable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
resources.add(Accountables.namedAccountable("term bytes", terms));
|
||||
resources.add(Accountables.namedAccountable("term addresses", termAddresses));
|
||||
|
|
|
@ -18,7 +18,10 @@ package org.apache.lucene.codecs.lucene50;
|
|||
*/
|
||||
|
||||
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.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
@ -61,8 +64,7 @@ class Lucene50NormsProducer extends NormsProducer {
|
|||
private final IndexInput data;
|
||||
|
||||
// ram instances we have already loaded
|
||||
final Map<String,NumericDocValues> instances = new HashMap<>();
|
||||
final Map<String,Accountable> instancesInfo = new HashMap<>();
|
||||
final Map<String,Norms> instances = new HashMap<>();
|
||||
|
||||
private final AtomicLong ramBytesUsed;
|
||||
private final AtomicInteger activeCount = new AtomicInteger();
|
||||
|
@ -76,7 +78,6 @@ class Lucene50NormsProducer extends NormsProducer {
|
|||
norms.putAll(original.norms);
|
||||
data = original.data.clone();
|
||||
instances.putAll(original.instances);
|
||||
instancesInfo.putAll(original.instancesInfo);
|
||||
ramBytesUsed = new AtomicLong(original.ramBytesUsed.get());
|
||||
activeCount.set(original.activeCount.get());
|
||||
maxDoc = original.maxDoc;
|
||||
|
@ -168,15 +169,13 @@ class Lucene50NormsProducer extends NormsProducer {
|
|||
|
||||
@Override
|
||||
public synchronized NumericDocValues getNorms(FieldInfo field) throws IOException {
|
||||
NumericDocValues instance = instances.get(field.name);
|
||||
Norms instance = instances.get(field.name);
|
||||
if (instance == null) {
|
||||
LoadedNorms loaded = loadNorms(norms.get(field.name));
|
||||
instance = loaded.norms;
|
||||
instance = loadNorms(norms.get(field.name));
|
||||
if (!merging) {
|
||||
instances.put(field.name, instance);
|
||||
activeCount.incrementAndGet();
|
||||
ramBytesUsed.addAndGet(loaded.ramBytesUsed);
|
||||
instancesInfo.put(field.name, loaded.info);
|
||||
ramBytesUsed.addAndGet(instance.ramBytesUsed());
|
||||
}
|
||||
}
|
||||
return instance;
|
||||
|
@ -188,8 +187,8 @@ class Lucene50NormsProducer extends NormsProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public synchronized Iterable<? extends Accountable> getChildResources() {
|
||||
return Accountables.namedAccountables("field", instancesInfo);
|
||||
public synchronized Iterable<Accountable> getChildResources() {
|
||||
return Accountables.namedAccountables("field", instances);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -197,44 +196,74 @@ class Lucene50NormsProducer extends NormsProducer {
|
|||
CodecUtil.checksumEntireFile(data);
|
||||
}
|
||||
|
||||
private LoadedNorms loadNorms(NormsEntry entry) throws IOException {
|
||||
LoadedNorms instance = new LoadedNorms();
|
||||
private Norms loadNorms(NormsEntry entry) throws IOException {
|
||||
switch(entry.format) {
|
||||
case CONST_COMPRESSED: {
|
||||
final long v = entry.offset;
|
||||
instance.info = Accountables.namedAccountable("constant", 8);
|
||||
instance.ramBytesUsed = 8;
|
||||
instance.norms = new NumericDocValues() {
|
||||
return new Norms() {
|
||||
@Override
|
||||
public long get(int docID) {
|
||||
return v;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return 8;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "constant";
|
||||
}
|
||||
};
|
||||
break;
|
||||
}
|
||||
case UNCOMPRESSED: {
|
||||
data.seek(entry.offset);
|
||||
final byte bytes[] = new byte[entry.count];
|
||||
data.readBytes(bytes, 0, bytes.length);
|
||||
instance.info = Accountables.namedAccountable("byte array", bytes.length);
|
||||
instance.ramBytesUsed = RamUsageEstimator.sizeOf(bytes);
|
||||
instance.norms = new NumericDocValues() {
|
||||
return new Norms() {
|
||||
@Override
|
||||
public long get(int docID) {
|
||||
return bytes[docID];
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return RamUsageEstimator.sizeOf(bytes);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "byte array";
|
||||
}
|
||||
};
|
||||
break;
|
||||
}
|
||||
case DELTA_COMPRESSED: {
|
||||
data.seek(entry.offset);
|
||||
int packedIntsVersion = data.readVInt();
|
||||
int blockSize = data.readVInt();
|
||||
final BlockPackedReader reader = new BlockPackedReader(data, packedIntsVersion, blockSize, entry.count, false);
|
||||
instance.info = Accountables.namedAccountable("delta compressed", reader);
|
||||
instance.ramBytesUsed = reader.ramBytesUsed();
|
||||
instance.norms = reader;
|
||||
break;
|
||||
return new Norms() {
|
||||
@Override
|
||||
public long get(int docID) {
|
||||
return reader.get(docID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return reader.ramBytesUsed();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
return Collections.singleton(reader);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "delta compressed";
|
||||
}
|
||||
};
|
||||
}
|
||||
case TABLE_COMPRESSED: {
|
||||
data.seek(entry.offset);
|
||||
|
@ -256,15 +285,27 @@ class Lucene50NormsProducer extends NormsProducer {
|
|||
}
|
||||
|
||||
final PackedInts.Reader ordsReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatID), packedIntsVersion, entry.count, bitsPerValue);
|
||||
instance.info = Accountables.namedAccountable("table compressed", ordsReader);
|
||||
instance.ramBytesUsed = RamUsageEstimator.sizeOf(decode) + ordsReader.ramBytesUsed();
|
||||
instance.norms = new NumericDocValues() {
|
||||
return new Norms() {
|
||||
@Override
|
||||
public long get(int docID) {
|
||||
return decode[(int)ordsReader.get(docID)];
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return RamUsageEstimator.sizeOf(decode) + ordsReader.ramBytesUsed();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
return Collections.singleton(ordsReader);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "table compressed";
|
||||
}
|
||||
};
|
||||
break;
|
||||
}
|
||||
case INDIRECT: {
|
||||
data.seek(entry.offset);
|
||||
|
@ -272,12 +313,9 @@ class Lucene50NormsProducer extends NormsProducer {
|
|||
int packedIntsVersion = data.readVInt();
|
||||
int blockSize = data.readVInt();
|
||||
final MonotonicBlockPackedReader live = MonotonicBlockPackedReader.of(data, packedIntsVersion, blockSize, entry.count, false);
|
||||
LoadedNorms nestedInstance = loadNorms(entry.nested);
|
||||
instance.ramBytesUsed = live.ramBytesUsed() + nestedInstance.ramBytesUsed;
|
||||
instance.info = Accountables.namedAccountable("indirect -> " + nestedInstance.info, instance.ramBytesUsed);
|
||||
final NumericDocValues values = nestedInstance.norms;
|
||||
final Norms nestedInstance = loadNorms(entry.nested);
|
||||
final int upperBound = entry.count-1;
|
||||
instance.norms = new NumericDocValues() {
|
||||
return new Norms() {
|
||||
@Override
|
||||
public long get(int docID) {
|
||||
int low = 0;
|
||||
|
@ -291,13 +329,30 @@ class Lucene50NormsProducer extends NormsProducer {
|
|||
} else if (doc > docID) {
|
||||
high = mid - 1;
|
||||
} else {
|
||||
return values.get(mid);
|
||||
return nestedInstance.get(mid);
|
||||
}
|
||||
}
|
||||
return common;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return live.ramBytesUsed() + nestedInstance.ramBytesUsed();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
List<Accountable> children = new ArrayList<>();
|
||||
children.add(Accountables.namedAccountable("keys", live));
|
||||
children.add(Accountables.namedAccountable("values", nestedInstance));
|
||||
return Collections.unmodifiableList(children);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "indirect";
|
||||
}
|
||||
};
|
||||
break;
|
||||
}
|
||||
case PATCHED_BITSET: {
|
||||
data.seek(entry.offset);
|
||||
|
@ -310,21 +365,35 @@ class Lucene50NormsProducer extends NormsProducer {
|
|||
int doc = (int) live.get(i);
|
||||
set.set(doc);
|
||||
}
|
||||
LoadedNorms nestedInstance = loadNorms(entry.nested);
|
||||
instance.ramBytesUsed = set.ramBytesUsed() + nestedInstance.ramBytesUsed;
|
||||
instance.info = Accountables.namedAccountable("patched bitset -> " + nestedInstance.info, instance.ramBytesUsed);
|
||||
final NumericDocValues values = nestedInstance.norms;
|
||||
instance.norms = new NumericDocValues() {
|
||||
Norms nestedInstance = loadNorms(entry.nested);
|
||||
return new Norms() {
|
||||
@Override
|
||||
public long get(int docID) {
|
||||
if (set.get(docID)) {
|
||||
return values.get(docID);
|
||||
return nestedInstance.get(docID);
|
||||
} else {
|
||||
return common;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return set.ramBytesUsed() + nestedInstance.ramBytesUsed();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
List<Accountable> children = new ArrayList<>();
|
||||
children.add(Accountables.namedAccountable("keys", set));
|
||||
children.add(Accountables.namedAccountable("values", nestedInstance));
|
||||
return Collections.unmodifiableList(children);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "patched bitset";
|
||||
}
|
||||
};
|
||||
break;
|
||||
}
|
||||
case PATCHED_TABLE: {
|
||||
data.seek(entry.offset);
|
||||
|
@ -344,11 +413,9 @@ class Lucene50NormsProducer extends NormsProducer {
|
|||
}
|
||||
|
||||
final PackedInts.Reader ordsReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatID), packedIntsVersion, entry.count, bitsPerValue);
|
||||
final LoadedNorms nestedInstance = loadNorms(entry.nested);
|
||||
instance.ramBytesUsed = RamUsageEstimator.sizeOf(decode) + ordsReader.ramBytesUsed() + nestedInstance.ramBytesUsed;
|
||||
instance.info = Accountables.namedAccountable("patched table -> " + nestedInstance.info, instance.ramBytesUsed);
|
||||
final NumericDocValues values = nestedInstance.norms;
|
||||
instance.norms = new NumericDocValues() {
|
||||
final Norms nestedInstance = loadNorms(entry.nested);
|
||||
|
||||
return new Norms() {
|
||||
@Override
|
||||
public long get(int docID) {
|
||||
int ord = (int)ordsReader.get(docID);
|
||||
|
@ -356,16 +423,32 @@ class Lucene50NormsProducer extends NormsProducer {
|
|||
// doing a try/catch here eliminates a seemingly unavoidable branch in hotspot...
|
||||
return decode[ord];
|
||||
} catch (IndexOutOfBoundsException e) {
|
||||
return values.get(docID);
|
||||
return nestedInstance.get(docID);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return RamUsageEstimator.sizeOf(decode) + ordsReader.ramBytesUsed() + nestedInstance.ramBytesUsed();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
List<Accountable> children = new ArrayList<>();
|
||||
children.add(Accountables.namedAccountable("common", ordsReader));
|
||||
children.add(Accountables.namedAccountable("uncommon", nestedInstance));
|
||||
return Collections.unmodifiableList(children);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "patched table";
|
||||
}
|
||||
};
|
||||
break;
|
||||
}
|
||||
default:
|
||||
throw new AssertionError();
|
||||
}
|
||||
return instance;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -380,10 +463,7 @@ class Lucene50NormsProducer extends NormsProducer {
|
|||
NormsEntry nested;
|
||||
}
|
||||
|
||||
static class LoadedNorms {
|
||||
NumericDocValues norms;
|
||||
long ramBytesUsed;
|
||||
Accountable info;
|
||||
static abstract class Norms extends NumericDocValues implements Accountable {
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -330,7 +330,7 @@ public abstract class PerFieldDocValuesFormat extends DocValuesFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
return Accountables.namedAccountables("format", formats);
|
||||
}
|
||||
|
||||
|
|
|
@ -302,7 +302,7 @@ public abstract class PerFieldPostingsFormat extends PostingsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
return Accountables.namedAccountables("format", formats);
|
||||
}
|
||||
|
||||
|
|
|
@ -638,7 +638,7 @@ public class MultiDocValues {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
resources.add(Accountables.namedAccountable("global ord deltas", globalOrdDeltas));
|
||||
resources.add(Accountables.namedAccountable("first segments", firstSegments));
|
||||
|
|
|
@ -159,7 +159,7 @@ class SegmentDocValuesProducer extends DocValuesProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
for (Accountable producer : dvProducers) {
|
||||
resources.add(Accountables.namedAccountable("delegate", producer));
|
||||
|
|
|
@ -500,7 +500,7 @@ public final class SegmentReader extends LeafReader implements Accountable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
ensureOpen();
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
resources.add(Accountables.namedAccountable("postings", core.fields));
|
||||
|
|
|
@ -159,7 +159,7 @@ public class CachingWrapperFilter extends Filter implements Accountable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
// Sync to pull the current set of values:
|
||||
synchronized (cache) {
|
||||
// no need to clone, Accountable#namedAccountables already copies the data
|
||||
|
|
|
@ -70,7 +70,7 @@ public abstract class FilteredDocIdSet extends DocIdSet {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
return _innerSet.getChildResources();
|
||||
}
|
||||
|
||||
|
|
|
@ -261,7 +261,7 @@ public class LRUFilterCache implements FilterCache, Accountable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
synchronized (this) {
|
||||
return Accountables.namedAccountables("segment", cache);
|
||||
}
|
||||
|
|
|
@ -260,7 +260,7 @@ public class NRTCachingDirectory extends FilterDirectory implements Accountable
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
return Collections.singleton(Accountables.namedAccountable("cache", cache));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -143,7 +143,7 @@ public class RAMDirectory extends BaseDirectory implements Accountable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
return Accountables.namedAccountables("file", fileMap);
|
||||
}
|
||||
|
||||
|
|
|
@ -193,7 +193,7 @@ public class RAMOutputStream extends IndexOutput implements Accountable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
return Collections.singleton(Accountables.namedAccountable("file", file));
|
||||
}
|
||||
|
||||
|
|
|
@ -36,7 +36,7 @@ public interface Accountable {
|
|||
* The result should be a point-in-time snapshot (to avoid race conditions).
|
||||
* @see Accountables
|
||||
*/
|
||||
default Iterable<? extends Accountable> getChildResources() {
|
||||
default Iterable<Accountable> getChildResources() {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
|
|
|
@ -128,8 +128,8 @@ public class Accountables {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
return children;
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
return (Iterable<Accountable>) children;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -445,7 +445,7 @@ public final class FST<T> implements Accountable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
if (packed) {
|
||||
resources.add(Accountables.namedAccountable("node ref to address", nodeRefToAddress));
|
||||
|
|
|
@ -161,7 +161,7 @@ public class CachedOrdinalsReader extends OrdinalsReader implements Accountable
|
|||
}
|
||||
|
||||
@Override
|
||||
public synchronized Iterable<? extends Accountable> getChildResources() {
|
||||
public synchronized Iterable<Accountable> getChildResources() {
|
||||
return Accountables.namedAccountables("segment", ordsCache);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -90,7 +90,7 @@ public class BitDocIdSetCachingWrapperFilter extends BitDocIdSetFilter implement
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
return filter.getChildResources();
|
||||
}
|
||||
|
||||
|
|
|
@ -603,7 +603,7 @@ class FieldCacheImpl implements FieldCache {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
resources.add(Accountables.namedAccountable("term bytes", bytes));
|
||||
resources.add(Accountables.namedAccountable("ord -> term", termOrdToBytesOffset));
|
||||
|
@ -746,7 +746,7 @@ class FieldCacheImpl implements FieldCache {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
resources.add(Accountables.namedAccountable("term bytes", bytes));
|
||||
resources.add(Accountables.namedAccountable("addresses", docToOffset));
|
||||
|
|
|
@ -232,7 +232,7 @@ public final class VersionBlockTreeTermsReader extends FieldsProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
resources.addAll(Accountables.namedAccountables("field", fields));
|
||||
resources.add(Accountables.namedAccountable("delegate", postingsReader));
|
||||
|
|
|
@ -163,7 +163,7 @@ final class VersionFieldReader extends Terms implements Accountable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
if (index == null) {
|
||||
return Collections.emptyList();
|
||||
} else {
|
||||
|
|
|
@ -730,7 +730,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
try {
|
||||
if (searcherMgr != null) {
|
||||
|
|
|
@ -262,7 +262,7 @@ public class AnalyzingSuggester extends Lookup {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
if (fst == null) {
|
||||
return Collections.emptyList();
|
||||
} else {
|
||||
|
|
|
@ -213,7 +213,7 @@ public class FreeTextSuggester extends Lookup {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
if (fst == null) {
|
||||
return Collections.emptyList();
|
||||
} else {
|
||||
|
|
|
@ -315,7 +315,7 @@ public class FSTCompletionLookup extends Lookup implements Accountable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
if (normalCompletion != null) {
|
||||
resources.add(Accountables.namedAccountable("fst", normalCompletion.getFST()));
|
||||
|
|
|
@ -301,7 +301,7 @@ public class WFSTCompletionLookup extends Lookup {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
if (fst == null) {
|
||||
return Collections.emptyList();
|
||||
} else {
|
||||
|
|
|
@ -290,8 +290,8 @@ public class AssertingDocValuesFormat extends DocValuesFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
Iterable<? extends Accountable> res = in.getChildResources();
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
Iterable<Accountable> res = in.getChildResources();
|
||||
TestUtil.checkIterator(res.iterator());
|
||||
return res;
|
||||
}
|
||||
|
|
|
@ -112,8 +112,8 @@ public class AssertingNormsFormat extends NormsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
Iterable<? extends Accountable> res = in.getChildResources();
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
Iterable<Accountable> res = in.getChildResources();
|
||||
TestUtil.checkIterator(res.iterator());
|
||||
return res;
|
||||
}
|
||||
|
|
|
@ -100,8 +100,8 @@ public final class AssertingPostingsFormat extends PostingsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
Iterable<? extends Accountable> res = in.getChildResources();
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
Iterable<Accountable> res = in.getChildResources();
|
||||
TestUtil.checkIterator(res.iterator());
|
||||
return res;
|
||||
}
|
||||
|
|
|
@ -85,8 +85,8 @@ public class AssertingStoredFieldsFormat extends StoredFieldsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
Iterable<? extends Accountable> res = in.getChildResources();
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
Iterable<Accountable> res = in.getChildResources();
|
||||
TestUtil.checkIterator(res.iterator());
|
||||
return res;
|
||||
}
|
||||
|
|
|
@ -84,8 +84,8 @@ public class AssertingTermVectorsFormat extends TermVectorsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
Iterable<? extends Accountable> res = in.getChildResources();
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
Iterable<Accountable> res = in.getChildResources();
|
||||
TestUtil.checkIterator(res.iterator());
|
||||
return res;
|
||||
}
|
||||
|
|
|
@ -98,7 +98,7 @@ public final class RAMOnlyPostingsFormat extends PostingsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
return Accountables.namedAccountables("field", fieldToTerms);
|
||||
}
|
||||
|
||||
|
|
|
@ -241,7 +241,7 @@ public abstract class BaseBitSetTestCase<T extends BitSet> extends LuceneTestCas
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
|
|
|
@ -342,7 +342,7 @@ public class SuggestComponent extends SearchComponent implements SolrCoreAware,
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
return Accountables.namedAccountables("field", suggesters);
|
||||
}
|
||||
|
||||
|
|
|
@ -213,7 +213,7 @@ public class SolrSuggester implements Accountable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Accountable> getChildResources() {
|
||||
public Iterable<Accountable> getChildResources() {
|
||||
return lookup.getChildResources();
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue