Remove unused "implements Accountable". (#13330)

A number of file formats no longer implement `Accountable`. This allows
removing this interface on a number of internal classes as well.
This commit is contained in:
Adrien Grand 2024-05-09 23:27:18 +02:00 committed by GitHub
parent b60e86c4b9
commit f3a52113a4
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
10 changed files with 9 additions and 156 deletions

View File

@ -44,11 +44,9 @@ import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.RandomAccessInput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.hnsw.HnswGraph;
import org.apache.lucene.util.hnsw.HnswGraphSearcher;
import org.apache.lucene.util.hnsw.OrdinalTranslatedKnnCollector;
@ -392,10 +390,7 @@ public final class Lucene95HnswVectorsReader extends KnnVectorsReader implements
// Contains the configuration for reading sparse vectors and translating vector ordinals to
// docId
OrdToDocDISIReaderConfiguration ordToDocVectorValues)
implements Accountable {
private static final long SHALLOW_SIZE =
RamUsageEstimator.shallowSizeOfInstance(FieldEntry.class);
OrdToDocDISIReaderConfiguration ordToDocVectorValues) {
static FieldEntry create(
IndexInput input,
@ -462,14 +457,6 @@ public final class Lucene95HnswVectorsReader extends KnnVectorsReader implements
offsetsLength,
ordToDocVectorValues);
}
@Override
public long ramBytesUsed() {
return SHALLOW_SIZE
+ Arrays.stream(nodesByLevel).mapToLong(nodes -> RamUsageEstimator.sizeOf(nodes)).sum()
+ RamUsageEstimator.sizeOf(ordToDocVectorValues)
+ RamUsageEstimator.sizeOf(offsetsMeta);
}
}
/** Read the nearest-neighbors graph from the index input */

View File

@ -324,20 +324,6 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
input.seek(dirOffset);
}
@Override
public long ramBytesUsed() {
long sizeInBytes = ((termBytesReader != null) ? termBytesReader.ramBytesUsed() : 0);
for (FieldIndexData entry : fields.values()) {
sizeInBytes += entry.ramBytesUsed();
}
return sizeInBytes;
}
@Override
public Collection<Accountable> getChildResources() {
return Accountables.namedAccountables("field", fields);
}
@Override
public String toString() {
return getClass().getSimpleName()

View File

@ -19,7 +19,6 @@ package org.apache.lucene.codecs.blockterms;
import java.io.Closeable;
import java.io.IOException;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.BytesRef;
// TODO
@ -34,7 +33,7 @@ import org.apache.lucene.util.BytesRef;
*
* @lucene.experimental
*/
public abstract class TermsIndexReaderBase implements Closeable, Accountable {
public abstract class TermsIndexReaderBase implements Closeable {
public abstract FieldIndexEnum getFieldEnum(FieldInfo fieldInfo);

View File

@ -19,7 +19,6 @@ package org.apache.lucene.codecs.blockterms;
import static org.apache.lucene.util.fst.FST.readMetadata;
import java.io.IOException;
import java.util.Collection;
import java.util.HashMap;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.index.CorruptIndexException;
@ -27,8 +26,6 @@ import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.fst.BytesRefFSTEnum;
import org.apache.lucene.util.fst.FST;
@ -168,22 +165,6 @@ public class VariableGapTermsIndexReader extends TermsIndexReaderBase {
@Override
public void close() throws IOException {}
@Override
public long ramBytesUsed() {
long sizeInBytes = 0;
for (FST<Long> entry : fields.values()) {
if (entry != null) {
sizeInBytes += entry.ramBytesUsed();
}
}
return sizeInBytes;
}
@Override
public Collection<Accountable> getChildResources() {
return Accountables.namedAccountables("field", fields);
}
@Override
public String toString() {
return getClass().getSimpleName() + "(fields=" + fields.size() + ")";

View File

@ -29,7 +29,6 @@ import static org.apache.lucene.codecs.simpletext.SimpleTextSkipWriter.SKIP_LIST
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
@ -51,17 +50,13 @@ import org.apache.lucene.search.DocIdSetIterator;
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.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.CharsRef;
import org.apache.lucene.util.CharsRefBuilder;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.IntsRefBuilder;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.StringHelper;
import org.apache.lucene.util.fst.BytesRefFSTEnum;
import org.apache.lucene.util.fst.FST;
@ -650,12 +645,7 @@ class SimpleTextFieldsReader extends FieldsProducer {
}
}
private static final long TERMS_BASE_RAM_BYTES_USED =
RamUsageEstimator.shallowSizeOfInstance(SimpleTextTerms.class)
+ RamUsageEstimator.shallowSizeOfInstance(BytesRef.class)
+ RamUsageEstimator.shallowSizeOfInstance(CharsRef.class);
private class SimpleTextTerms extends Terms implements Accountable {
private class SimpleTextTerms extends Terms {
private final long termsStart;
private final FieldInfo fieldInfo;
private final int maxDoc;
@ -748,23 +738,6 @@ class SimpleTextFieldsReader extends FieldsProducer {
// System.out.println("FST " + fst.sizeInBytes());
}
@Override
public long ramBytesUsed() {
return TERMS_BASE_RAM_BYTES_USED
+ (fst != null ? fst.ramBytesUsed() : 0)
+ RamUsageEstimator.sizeOf(scratch.bytes())
+ RamUsageEstimator.sizeOf(scratchUTF16.chars());
}
@Override
public Collection<Accountable> getChildResources() {
if (fst == null) {
return Collections.emptyList();
} else {
return Collections.singletonList(Accountables.namedAccountable("term cache", fst));
}
}
@Override
public String toString() {
return getClass().getSimpleName()

View File

@ -24,8 +24,6 @@ import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.RandomAccessInput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.packed.DirectMonotonicReader;
import org.apache.lucene.util.packed.DirectMonotonicWriter;
@ -33,10 +31,7 @@ import org.apache.lucene.util.packed.DirectMonotonicWriter;
* Configuration for {@link DirectMonotonicReader} and {@link IndexedDISI} for reading sparse
* vectors. The format in the static writing methods adheres to the Lucene95HnswVectorsFormat
*/
public class OrdToDocDISIReaderConfiguration implements Accountable {
private static final long SHALLOW_SIZE =
RamUsageEstimator.shallowSizeOfInstance(OrdToDocDISIReaderConfiguration.class);
public class OrdToDocDISIReaderConfiguration {
/**
* Writes out the docsWithField and ordToDoc mapping to the outputMeta and vectorData
@ -188,11 +183,6 @@ public class OrdToDocDISIReaderConfiguration implements Accountable {
this.meta = meta;
}
@Override
public long ramBytesUsed() {
return SHALLOW_SIZE + RamUsageEstimator.sizeOf(meta);
}
/**
* @param dataIn the dataIn
* @return the IndexedDISI for sparse values

View File

@ -42,7 +42,6 @@ import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.ReadAdvice;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.hnsw.RandomVectorScorer;
@ -271,9 +270,7 @@ public final class Lucene99FlatVectorsReader extends FlatVectorsReader {
int dimension,
int size,
OrdToDocDISIReaderConfiguration ordToDoc,
FieldInfo info)
implements Accountable {
static final long SHALLOW_SIZE = RamUsageEstimator.shallowSizeOfInstance(FieldEntry.class);
FieldInfo info) {
FieldEntry {
if (similarityFunction != info.getVectorSimilarityFunction()) {
@ -336,10 +333,5 @@ public final class Lucene99FlatVectorsReader extends FlatVectorsReader {
ordToDoc,
info);
}
@Override
public long ramBytesUsed() {
return SHALLOW_SIZE + RamUsageEstimator.sizeOf(ordToDoc);
}
}
}

View File

@ -44,7 +44,6 @@ import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.RandomAccessInput;
import org.apache.lucene.store.ReadAdvice;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.RamUsageEstimator;
@ -361,10 +360,7 @@ public final class Lucene99HnswVectorsReader extends KnnVectorsReader
DirectMonotonicReader.Meta offsetsMeta,
long offsetsOffset,
int offsetsBlockShift,
long offsetsLength)
implements Accountable {
private static final long SHALLOW_SIZE =
RamUsageEstimator.shallowSizeOfInstance(FieldEntry.class);
long offsetsLength) {
static FieldEntry create(
IndexInput input,
@ -423,13 +419,6 @@ public final class Lucene99HnswVectorsReader extends KnnVectorsReader
offsetsBlockShift,
offsetsLength);
}
@Override
public long ramBytesUsed() {
return SHALLOW_SIZE
+ Arrays.stream(nodesByLevel).mapToLong(nodes -> RamUsageEstimator.sizeOf(nodes)).sum()
+ RamUsageEstimator.sizeOf(offsetsMeta);
}
}
/** Read the nearest-neighbors graph from the index input */

View File

@ -41,7 +41,6 @@ import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.ReadAdvice;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.hnsw.RandomVectorScorer;
@ -329,10 +328,7 @@ public final class Lucene99ScalarQuantizedVectorsReader extends FlatVectorsReade
int size,
byte bits,
boolean compress,
OrdToDocDISIReaderConfiguration ordToDoc)
implements Accountable {
private static final long SHALLOW_SIZE =
RamUsageEstimator.shallowSizeOfInstance(FieldEntry.class);
OrdToDocDISIReaderConfiguration ordToDoc) {
static FieldEntry create(
IndexInput input,
@ -385,11 +381,6 @@ public final class Lucene99ScalarQuantizedVectorsReader extends FlatVectorsReade
compress,
ordToDoc);
}
@Override
public long ramBytesUsed() {
return SHALLOW_SIZE + RamUsageEstimator.sizeOf(ordToDoc);
}
}
private static final class QuantizedVectorValues extends FloatVectorValues {

View File

@ -20,32 +20,24 @@ import java.io.IOException;
import java.util.Arrays;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.RandomAccessInput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.LongValues;
import org.apache.lucene.util.RamUsageEstimator;
/**
* Retrieves an instance previously written by {@link DirectMonotonicWriter}.
*
* @see DirectMonotonicWriter
*/
public final class DirectMonotonicReader extends LongValues implements Accountable {
private static final long BASE_RAM_BYTES_USED =
RamUsageEstimator.shallowSizeOfInstance(DirectMonotonicReader.class);
public final class DirectMonotonicReader extends LongValues {
/**
* In-memory metadata that needs to be kept around for {@link DirectMonotonicReader} to read data
* from disk.
*/
public static class Meta implements Accountable {
public static class Meta {
// Use a shift of 63 so that there would be a single block regardless of the number of values.
private static final Meta SINGLE_ZERO_BLOCK = new Meta(1L, 63);
private static final long BASE_RAM_BYTES_USED =
RamUsageEstimator.shallowSizeOfInstance(Meta.class);
private final int blockShift;
private final int numBlocks;
private final long[] mins;
@ -65,15 +57,6 @@ public final class DirectMonotonicReader extends LongValues implements Accountab
this.bpvs = new byte[this.numBlocks];
this.offsets = new long[this.numBlocks];
}
@Override
public long ramBytesUsed() {
return BASE_RAM_BYTES_USED
+ RamUsageEstimator.sizeOf(mins)
+ RamUsageEstimator.sizeOf(avgs)
+ RamUsageEstimator.sizeOf(bpvs)
+ RamUsageEstimator.sizeOf(offsets);
}
}
/**
@ -132,7 +115,6 @@ public final class DirectMonotonicReader extends LongValues implements Accountab
private final long[] mins;
private final float[] avgs;
private final byte[] bpvs;
private final int nonZeroBpvs;
private DirectMonotonicReader(
int blockShift, LongValues[] readers, long[] mins, float[] avgs, byte[] bpvs) {
@ -147,13 +129,6 @@ public final class DirectMonotonicReader extends LongValues implements Accountab
|| readers.length != bpvs.length) {
throw new IllegalArgumentException();
}
int nonZeroBpvs = 0;
for (byte b : bpvs) {
if (b != 0) {
nonZeroBpvs++;
}
}
this.nonZeroBpvs = nonZeroBpvs;
}
@Override
@ -213,14 +188,4 @@ public final class DirectMonotonicReader extends LongValues implements Accountab
return -1 - lo;
}
@Override
public long ramBytesUsed() {
// Don't include meta, which should be accounted separately
return BASE_RAM_BYTES_USED
+ RamUsageEstimator.shallowSizeOf(readers)
+
// Assume empty objects for the readers
nonZeroBpvs * RamUsageEstimator.alignObjectSize(RamUsageEstimator.NUM_BYTES_ARRAY_HEADER);
}
}