benwtrent commented on code in PR #12582: URL: https://github.com/apache/lucene/pull/12582#discussion_r1343092520
########## lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsWriter.java: ########## @@ -139,7 +139,7 @@ public int nextDoc() throws IOException { } /** View over multiple vector values supporting iterator-style access via DocIdMerger. */ - protected static final class MergedVectorValues { + public static final class MergedVectorValues { Review Comment: This is so I can get a merged view over vectors. I don't strictly need the doc updates. I just need to be able to iterate all the vectors in row and randomly accept some. ########## lucene/core/src/java/org/apache/lucene/codecs/HnswGraphProvider.java: ########## @@ -0,0 +1,37 @@ +/* + * 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. + */ + +package org.apache.lucene.codecs; + +import java.io.IOException; +import org.apache.lucene.util.hnsw.HnswGraph; + +/** + * Provides a hnsw graph + * + * @lucene.experimental + */ +public interface HnswGraphProvider { Review Comment: I am thinking about adding this to Lucene directly. Right now we do an `instance of Lucene95...`. For future flexibility, we should have a thing that indicates it can provide a graph for a field. ########## lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99Codec.java: ########## @@ -0,0 +1,217 @@ +/* + * 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. + */ +package org.apache.lucene.codecs.lucene99; + +import java.util.Objects; +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.codecs.CompoundFormat; +import org.apache.lucene.codecs.DocValuesFormat; +import org.apache.lucene.codecs.FieldInfosFormat; +import org.apache.lucene.codecs.FilterCodec; +import org.apache.lucene.codecs.KnnVectorsFormat; +import org.apache.lucene.codecs.LiveDocsFormat; +import org.apache.lucene.codecs.NormsFormat; +import org.apache.lucene.codecs.PointsFormat; +import org.apache.lucene.codecs.PostingsFormat; +import org.apache.lucene.codecs.SegmentInfoFormat; +import org.apache.lucene.codecs.StoredFieldsFormat; +import org.apache.lucene.codecs.TermVectorsFormat; +import org.apache.lucene.codecs.lucene90.Lucene90CompoundFormat; +import org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat; +import org.apache.lucene.codecs.lucene90.Lucene90LiveDocsFormat; +import org.apache.lucene.codecs.lucene90.Lucene90NormsFormat; +import org.apache.lucene.codecs.lucene90.Lucene90PointsFormat; +import org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat; +import org.apache.lucene.codecs.lucene90.Lucene90SegmentInfoFormat; +import org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat; +import org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat; +import org.apache.lucene.codecs.lucene94.Lucene94FieldInfosFormat; +import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat; +import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat; +import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat; + +/** + * Implements the Lucene 9.8 index format Review Comment: ```suggestion * Implements the Lucene 9.9 index format ``` ########## lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsFormat.java: ########## @@ -0,0 +1,209 @@ +/* + * 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. + */ + +package org.apache.lucene.codecs.lucene99; + +import java.io.IOException; +import org.apache.lucene.codecs.KnnVectorsFormat; +import org.apache.lucene.codecs.KnnVectorsReader; +import org.apache.lucene.codecs.KnnVectorsWriter; +import org.apache.lucene.codecs.lucene90.IndexedDISI; +import org.apache.lucene.index.SegmentReadState; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.hnsw.HnswGraph; + +/** + * Lucene 9.5 vector format, which encodes numeric vector values and an optional associated graph + * connecting the documents having values. The graph is used to power HNSW search. The format + * consists of three files: + * + * <h2>.vec (vector data) file</h2> + * + * <p>For each field: + * + * <ul> + * <li>Vector data ordered by field, document ordinal, and vector dimension. When the + * vectorEncoding is BYTE, each sample is stored as a single byte. When it is FLOAT32, each + * sample is stored as an IEEE float in little-endian byte order. + * <li>DocIds encoded by {@link IndexedDISI#writeBitSet(DocIdSetIterator, IndexOutput, byte)}, + * note that only in sparse case + * <li>OrdToDoc was encoded by {@link org.apache.lucene.util.packed.DirectMonotonicWriter}, note + * that only in sparse case + * </ul> + * + * <h2>.vex (vector index)</h2> + * + * <p>Stores graphs connecting the documents for each field organized as a list of nodes' neighbours + * as following: + * + * <ul> + * <li>For each level: + * <ul> + * <li>For each node: + * <ul> + * <li><b>[vint]</b> the number of neighbor nodes + * <li><b>array[vint]</b> the delta encoded neighbor ordinals + * </ul> + * </ul> + * <li>After all levels are encoded memory offsets for each node's neighbor nodes encoded by + * {@link org.apache.lucene.util.packed.DirectMonotonicWriter} are appened to the end of the + * file. + * </ul> + * + * <h2>.vem (vector metadata) file</h2> + * + * <p>For each field: + * + * <ul> + * <li><b>[int32]</b> field number + * <li><b>[int32]</b> vector similarity function ordinal + * <li><b>[vlong]</b> offset to this field's vectors in the .vec file + * <li><b>[vlong]</b> length of this field's vectors, in bytes + * <li><b>[vlong]</b> offset to this field's index in the .vex file + * <li><b>[vlong]</b> length of this field's index data, in bytes + * <li><b>[vint]</b> dimension of this field's vectors + * <li><b>[int]</b> the number of documents having values for this field + * <li><b>[int8]</b> if equals to -1, dense – all documents have values for a field. If equals to + * 0, sparse – some documents missing values. + * <li>DocIds were encoded by {@link IndexedDISI#writeBitSet(DocIdSetIterator, IndexOutput, byte)} + * <li>OrdToDoc was encoded by {@link org.apache.lucene.util.packed.DirectMonotonicWriter}, note + * that only in sparse case + * <li><b>[vint]</b> the maximum number of connections (neigbours) that each node can have + * <li><b>[vint]</b> number of levels in the graph + * <li>Graph nodes by level. For each level + * <ul> + * <li><b>[vint]</b> the number of nodes on this level + * <li><b>array[vint]</b> for levels greater than 0 list of nodes on this level, stored as + * the level 0th delta encoded nodes' ordinals. + * </ul> + * </ul> + * + * @lucene.experimental + */ +public final class Lucene99HnswVectorsFormat extends KnnVectorsFormat { Review Comment: Need to update the docs here to talk about quantization and how its stored. ########## lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsReader.java: ########## @@ -0,0 +1,739 @@ +/* + * 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. + */ + +package org.apache.lucene.codecs.lucene99; + +import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.HnswGraphProvider; +import org.apache.lucene.codecs.KnnVectorsReader; +import org.apache.lucene.codecs.lucene90.IndexedDISI; +import org.apache.lucene.index.ByteVectorValues; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.index.FloatVectorValues; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.SegmentReadState; +import org.apache.lucene.index.VectorEncoding; +import org.apache.lucene.index.VectorSimilarityFunction; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.search.KnnCollector; +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.Bits; +import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.RamUsageEstimator; +import org.apache.lucene.util.ScalarQuantizer; +import org.apache.lucene.util.hnsw.HnswGraph; +import org.apache.lucene.util.hnsw.HnswGraphSearcher; +import org.apache.lucene.util.hnsw.OrdinalTranslatedKnnCollector; +import org.apache.lucene.util.hnsw.RandomVectorScorer; +import org.apache.lucene.util.packed.DirectMonotonicReader; + +/** + * Reads vectors from the index segments along with index data structures supporting KNN search. + * + * @lucene.experimental + */ +public final class Lucene99HnswVectorsReader extends KnnVectorsReader + implements QuantizedVectorsReader, HnswGraphProvider { + + private static final long SHALLOW_SIZE = + RamUsageEstimator.shallowSizeOfInstance(Lucene99HnswVectorsFormat.class); + + private final FieldInfos fieldInfos; + private final Map<String, FieldEntry> fields = new HashMap<>(); + private final IndexInput vectorData; + private final IndexInput vectorIndex; + private final IndexInput quantizedVectorData; Review Comment: File life cycle is owned by the HNSW vector reader. If we separate "flat" storage from "index" storage, this would be moved out of here. ########## lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsReader.java: ########## @@ -0,0 +1,739 @@ +/* + * 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. + */ + +package org.apache.lucene.codecs.lucene99; + +import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.HnswGraphProvider; +import org.apache.lucene.codecs.KnnVectorsReader; +import org.apache.lucene.codecs.lucene90.IndexedDISI; +import org.apache.lucene.index.ByteVectorValues; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.index.FloatVectorValues; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.SegmentReadState; +import org.apache.lucene.index.VectorEncoding; +import org.apache.lucene.index.VectorSimilarityFunction; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.search.KnnCollector; +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.Bits; +import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.RamUsageEstimator; +import org.apache.lucene.util.ScalarQuantizer; +import org.apache.lucene.util.hnsw.HnswGraph; +import org.apache.lucene.util.hnsw.HnswGraphSearcher; +import org.apache.lucene.util.hnsw.OrdinalTranslatedKnnCollector; +import org.apache.lucene.util.hnsw.RandomVectorScorer; +import org.apache.lucene.util.packed.DirectMonotonicReader; + +/** + * Reads vectors from the index segments along with index data structures supporting KNN search. + * + * @lucene.experimental + */ +public final class Lucene99HnswVectorsReader extends KnnVectorsReader + implements QuantizedVectorsReader, HnswGraphProvider { + + private static final long SHALLOW_SIZE = + RamUsageEstimator.shallowSizeOfInstance(Lucene99HnswVectorsFormat.class); + + private final FieldInfos fieldInfos; + private final Map<String, FieldEntry> fields = new HashMap<>(); + private final IndexInput vectorData; + private final IndexInput vectorIndex; + private final IndexInput quantizedVectorData; + private final Lucene99ScalarQuantizedVectorsReader quantizedVectorsReader; + + Lucene99HnswVectorsReader(SegmentReadState state) throws IOException { + this.fieldInfos = state.fieldInfos; + int versionMeta = readMetadata(state); + boolean success = false; + try { + vectorData = + openDataInput( + state, + versionMeta, + Lucene99HnswVectorsFormat.VECTOR_DATA_EXTENSION, + Lucene99HnswVectorsFormat.VECTOR_DATA_CODEC_NAME); + vectorIndex = + openDataInput( + state, + versionMeta, + Lucene99HnswVectorsFormat.VECTOR_INDEX_EXTENSION, + Lucene99HnswVectorsFormat.VECTOR_INDEX_CODEC_NAME); + if (fields.values().stream().anyMatch(FieldEntry::hasQuantizedVectors)) { + quantizedVectorData = + openDataInput( + state, + versionMeta, + Lucene99ScalarQuantizedVectorsFormat.QUANTIZED_VECTOR_DATA_EXTENSION, + Lucene99ScalarQuantizedVectorsFormat.QUANTIZED_VECTOR_DATA_CODEC_NAME); + quantizedVectorsReader = new Lucene99ScalarQuantizedVectorsReader(quantizedVectorData); + } else { + quantizedVectorData = null; + quantizedVectorsReader = null; + } + success = true; + } finally { + if (success == false) { + IOUtils.closeWhileHandlingException(this); + } + } + } + + private int readMetadata(SegmentReadState state) throws IOException { + String metaFileName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, state.segmentSuffix, Lucene99HnswVectorsFormat.META_EXTENSION); + int versionMeta = -1; + try (ChecksumIndexInput meta = state.directory.openChecksumInput(metaFileName)) { + Throwable priorE = null; + try { + versionMeta = + CodecUtil.checkIndexHeader( + meta, + Lucene99HnswVectorsFormat.META_CODEC_NAME, + Lucene99HnswVectorsFormat.VERSION_START, + Lucene99HnswVectorsFormat.VERSION_CURRENT, + state.segmentInfo.getId(), + state.segmentSuffix); + readFields(meta, state.fieldInfos); + } catch (Throwable exception) { + priorE = exception; + } finally { + CodecUtil.checkFooter(meta, priorE); + } + } + return versionMeta; + } + + private static IndexInput openDataInput( + SegmentReadState state, int versionMeta, String fileExtension, String codecName) + throws IOException { + String fileName = + IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, fileExtension); + IndexInput in = state.directory.openInput(fileName, state.context); + boolean success = false; + try { + int versionVectorData = + CodecUtil.checkIndexHeader( + in, + codecName, + Lucene99HnswVectorsFormat.VERSION_START, + Lucene99HnswVectorsFormat.VERSION_CURRENT, + state.segmentInfo.getId(), + state.segmentSuffix); + if (versionMeta != versionVectorData) { + throw new CorruptIndexException( + "Format versions mismatch: meta=" + + versionMeta + + ", " + + codecName + + "=" + + versionVectorData, + in); + } + CodecUtil.retrieveChecksum(in); + success = true; + return in; + } finally { + if (success == false) { + IOUtils.closeWhileHandlingException(in); + } + } + } + + private void readFields(ChecksumIndexInput meta, FieldInfos infos) throws IOException { + for (int fieldNumber = meta.readInt(); fieldNumber != -1; fieldNumber = meta.readInt()) { + FieldInfo info = infos.fieldInfo(fieldNumber); + if (info == null) { + throw new CorruptIndexException("Invalid field number: " + fieldNumber, meta); + } + FieldEntry fieldEntry = readField(meta); + validateFieldEntry(info, fieldEntry); + fields.put(info.name, fieldEntry); + } + } + + private void validateFieldEntry(FieldInfo info, FieldEntry fieldEntry) { + int dimension = info.getVectorDimension(); + if (dimension != fieldEntry.dimension) { + throw new IllegalStateException( + "Inconsistent vector dimension for field=\"" + + info.name + + "\"; " + + dimension + + " != " + + fieldEntry.dimension); + } + + int byteSize = + switch (info.getVectorEncoding()) { + case BYTE -> Byte.BYTES; + case FLOAT32 -> Float.BYTES; + }; + long vectorBytes = Math.multiplyExact((long) dimension, byteSize); + long numBytes = Math.multiplyExact(vectorBytes, fieldEntry.size); + if (numBytes != fieldEntry.vectorDataLength) { + throw new IllegalStateException( + "Vector data length " + + fieldEntry.vectorDataLength + + " not matching size=" + + fieldEntry.size + + " * dim=" + + dimension + + " * byteSize=" + + byteSize + + " = " + + numBytes); + } + if (fieldEntry.hasQuantizedVectors()) { + Lucene99ScalarQuantizedVectorsReader.validateFieldEntry( + info, fieldEntry.dimension, fieldEntry.size, fieldEntry.quantizedVectorDataLength); + } + } + + private VectorSimilarityFunction readSimilarityFunction(DataInput input) throws IOException { + int similarityFunctionId = input.readInt(); + if (similarityFunctionId < 0 + || similarityFunctionId >= VectorSimilarityFunction.values().length) { + throw new CorruptIndexException( + "Invalid similarity function id: " + similarityFunctionId, input); + } + return VectorSimilarityFunction.values()[similarityFunctionId]; + } + + private VectorEncoding readVectorEncoding(DataInput input) throws IOException { + int encodingId = input.readInt(); + if (encodingId < 0 || encodingId >= VectorEncoding.values().length) { + throw new CorruptIndexException("Invalid vector encoding id: " + encodingId, input); + } + return VectorEncoding.values()[encodingId]; + } + + private FieldEntry readField(IndexInput input) throws IOException { + VectorEncoding vectorEncoding = readVectorEncoding(input); + VectorSimilarityFunction similarityFunction = readSimilarityFunction(input); + return new FieldEntry(input, vectorEncoding, similarityFunction); + } + + @Override + public long ramBytesUsed() { + return Lucene99HnswVectorsReader.SHALLOW_SIZE + + RamUsageEstimator.sizeOfMap( + fields, RamUsageEstimator.shallowSizeOfInstance(FieldEntry.class)); + } + + @Override + public void checkIntegrity() throws IOException { + CodecUtil.checksumEntireFile(vectorData); + CodecUtil.checksumEntireFile(vectorIndex); + if (quantizedVectorsReader != null) { + quantizedVectorsReader.checkIntegrity(); + } + } + + @Override + public FloatVectorValues getFloatVectorValues(String field) throws IOException { + FieldEntry fieldEntry = fields.get(field); + if (fieldEntry.vectorEncoding != VectorEncoding.FLOAT32) { + throw new IllegalArgumentException( + "field=\"" + + field + + "\" is encoded as: " + + fieldEntry.vectorEncoding + + " expected: " + + VectorEncoding.FLOAT32); + } + return OffHeapFloatVectorValues.load(fieldEntry, vectorData); + } + + @Override + public ByteVectorValues getByteVectorValues(String field) throws IOException { + FieldEntry fieldEntry = fields.get(field); + if (fieldEntry.vectorEncoding != VectorEncoding.BYTE) { + throw new IllegalArgumentException( + "field=\"" + + field + + "\" is encoded as: " + + fieldEntry.vectorEncoding + + " expected: " + + VectorEncoding.FLOAT32); + } + return OffHeapByteVectorValues.load(fieldEntry, vectorData); + } + + @Override + public void search(String field, float[] target, KnnCollector knnCollector, Bits acceptDocs) + throws IOException { + FieldEntry fieldEntry = fields.get(field); + + if (fieldEntry.size() == 0 + || knnCollector.k() == 0 + || fieldEntry.vectorEncoding != VectorEncoding.FLOAT32) { + return; + } + if (quantizedVectorsReader != null) { + OffHeapQuantizedByteVectorValues vectorValues = getQuantizedVectorValues(field); + if (vectorValues == null) { + return; + } + RandomVectorScorer scorer = + new ScalarQuantizedRandomVectorScorer( + fieldEntry.similarityFunction, fieldEntry.scalarQuantizer, vectorValues, target); + HnswGraphSearcher.search( + scorer, + new OrdinalTranslatedKnnCollector(knnCollector, vectorValues::ordToDoc), + getGraph(fieldEntry), + vectorValues.getAcceptOrds(acceptDocs)); + } else { + OffHeapFloatVectorValues vectorValues = OffHeapFloatVectorValues.load(fieldEntry, vectorData); + RandomVectorScorer scorer = + RandomVectorScorer.createFloats(vectorValues, fieldEntry.similarityFunction, target); + HnswGraphSearcher.search( + scorer, + new OrdinalTranslatedKnnCollector(knnCollector, vectorValues::ordToDoc), + getGraph(fieldEntry), + vectorValues.getAcceptOrds(acceptDocs)); + } + } + + @Override + public void search(String field, byte[] target, KnnCollector knnCollector, Bits acceptDocs) + throws IOException { + FieldEntry fieldEntry = fields.get(field); + + if (fieldEntry.size() == 0 + || knnCollector.k() == 0 + || fieldEntry.vectorEncoding != VectorEncoding.BYTE) { + return; + } + + OffHeapByteVectorValues vectorValues = OffHeapByteVectorValues.load(fieldEntry, vectorData); + RandomVectorScorer scorer = + RandomVectorScorer.createBytes(vectorValues, fieldEntry.similarityFunction, target); + HnswGraphSearcher.search( + scorer, + new OrdinalTranslatedKnnCollector(knnCollector, vectorValues::ordToDoc), + getGraph(fieldEntry), + vectorValues.getAcceptOrds(acceptDocs)); + } + + @Override + public HnswGraph getGraph(String field) throws IOException { + FieldInfo info = fieldInfos.fieldInfo(field); + if (info == null) { + throw new IllegalArgumentException("No such field '" + field + "'"); + } + FieldEntry entry = fields.get(field); + if (entry != null && entry.vectorIndexLength > 0) { + return getGraph(entry); + } else { + return HnswGraph.EMPTY; + } + } + + private HnswGraph getGraph(FieldEntry entry) throws IOException { + return new OffHeapHnswGraph(entry, vectorIndex); + } + + @Override + public void close() throws IOException { + IOUtils.close(vectorData, vectorIndex, quantizedVectorData); + } + + @Override + public OffHeapQuantizedByteVectorValues getQuantizedVectorValues(String field) + throws IOException { + FieldEntry fieldEntry = fields.get(field); + if (fieldEntry == null || fieldEntry.hasQuantizedVectors() == false) { + return null; + } + assert quantizedVectorsReader != null && fieldEntry.quantizedOrdToDoc != null; + return quantizedVectorsReader.getQuantizedVectorValues( + fieldEntry.quantizedOrdToDoc, + fieldEntry.dimension, + fieldEntry.size, + fieldEntry.quantizedVectorDataOffset, + fieldEntry.quantizedVectorDataLength); + } + + @Override + public ScalarQuantizer getQuantizationState(String fieldName) { + FieldEntry field = fields.get(fieldName); + if (field == null || field.hasQuantizedVectors() == false) { + return null; + } + return field.scalarQuantizer; + } + + static class FieldEntry implements Accountable { + private static final long SHALLOW_SIZE = + RamUsageEstimator.shallowSizeOfInstance(FieldEntry.class); + final VectorSimilarityFunction similarityFunction; + final VectorEncoding vectorEncoding; + final long vectorDataOffset; + final long vectorDataLength; + final long vectorIndexOffset; + final long vectorIndexLength; + final int M; + final int numLevels; + final int dimension; + final int size; + final int[][] nodesByLevel; + // for each level the start offsets in vectorIndex file from where to read neighbours + final DirectMonotonicReader.Meta offsetsMeta; + final long offsetsOffset; + final int offsetsBlockShift; + final long offsetsLength; + final OrdToDocDISReaderConfiguration ordToDoc; + + final float configuredQuantile, lowerQuantile, upperQuantile; + final long quantizedVectorDataOffset, quantizedVectorDataLength; + final ScalarQuantizer scalarQuantizer; + final boolean isQuantized; + final OrdToDocDISReaderConfiguration quantizedOrdToDoc; + + FieldEntry( + IndexInput input, + VectorEncoding vectorEncoding, + VectorSimilarityFunction similarityFunction) + throws IOException { + this.similarityFunction = similarityFunction; + this.vectorEncoding = vectorEncoding; + this.isQuantized = input.readByte() == 1; + // Has int8 quantization + if (isQuantized) { + configuredQuantile = Float.intBitsToFloat(input.readInt()); + lowerQuantile = Float.intBitsToFloat(input.readInt()); + upperQuantile = Float.intBitsToFloat(input.readInt()); + quantizedVectorDataOffset = input.readVLong(); + quantizedVectorDataLength = input.readVLong(); + scalarQuantizer = new ScalarQuantizer(lowerQuantile, upperQuantile); + } else { + configuredQuantile = -1; + lowerQuantile = -1; + upperQuantile = -1; + quantizedVectorDataOffset = -1; + quantizedVectorDataLength = -1; + scalarQuantizer = null; + } + vectorDataOffset = input.readVLong(); + vectorDataLength = input.readVLong(); + vectorIndexOffset = input.readVLong(); + vectorIndexLength = input.readVLong(); + dimension = input.readVInt(); + size = input.readInt(); + if (isQuantized) { + quantizedOrdToDoc = OrdToDocDISReaderConfiguration.fromStoredMeta(input, size); + } else { + quantizedOrdToDoc = null; + } Review Comment: This is a non-obvious refactor. Basically, since we are keeping both `float32` and quantized vectors around, we need to store the offsets for both when in a sparse index. This way when reading quantized vectors, we read the `veq` file and for full-precision vectors we are always reading the `vec` file. ########## lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsWriter.java: ########## @@ -0,0 +1,1170 @@ +/* + * 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. + */ + +package org.apache.lucene.codecs.lucene99; + +import static org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat.DIRECT_MONOTONIC_BLOCK_SHIFT; +import static org.apache.lucene.codecs.lucene99.Lucene99ScalarQuantizedVectorsFormat.calculateDefaultQuantile; +import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.HnswGraphProvider; +import org.apache.lucene.codecs.KnnFieldVectorsWriter; +import org.apache.lucene.codecs.KnnVectorsReader; +import org.apache.lucene.codecs.KnnVectorsWriter; +import org.apache.lucene.codecs.lucene90.IndexedDISI; +import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat; +import org.apache.lucene.index.ByteVectorValues; +import org.apache.lucene.index.DocsWithFieldSet; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FloatVectorValues; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.MergeState; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.index.Sorter; +import org.apache.lucene.index.VectorEncoding; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.InfoStream; +import org.apache.lucene.util.RamUsageEstimator; +import org.apache.lucene.util.hnsw.HnswGraph; +import org.apache.lucene.util.hnsw.HnswGraph.NodesIterator; +import org.apache.lucene.util.hnsw.HnswGraphBuilder; +import org.apache.lucene.util.hnsw.NeighborArray; +import org.apache.lucene.util.hnsw.OnHeapHnswGraph; +import org.apache.lucene.util.hnsw.RandomAccessVectorValues; +import org.apache.lucene.util.hnsw.RandomVectorScorerSupplier; +import org.apache.lucene.util.packed.DirectMonotonicWriter; + +/** + * Writes vector values and knn graphs to index segments. + * + * @lucene.experimental + */ +public final class Lucene99HnswVectorsWriter extends KnnVectorsWriter { + + private final SegmentWriteState segmentWriteState; + private final IndexOutput meta, vectorData, quantizedVectorData, vectorIndex; + private final int M; + private final int beamWidth; + private final Lucene99ScalarQuantizedVectorsWriter quantizedVectorsWriter; + + private final List<FieldWriter<?>> fields = new ArrayList<>(); + private boolean finished; + + Lucene99HnswVectorsWriter( + SegmentWriteState state, + int M, + int beamWidth, + Lucene99ScalarQuantizedVectorsFormat quantizedVectorsFormat) + throws IOException { + this.M = M; + this.beamWidth = beamWidth; + segmentWriteState = state; + String metaFileName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, state.segmentSuffix, Lucene99HnswVectorsFormat.META_EXTENSION); + + String vectorDataFileName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, + state.segmentSuffix, + Lucene99HnswVectorsFormat.VECTOR_DATA_EXTENSION); + + String indexDataFileName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, + state.segmentSuffix, + Lucene99HnswVectorsFormat.VECTOR_INDEX_EXTENSION); + + final String quantizedVectorDataFileName = + quantizedVectorsFormat != null + ? IndexFileNames.segmentFileName( + state.segmentInfo.name, + state.segmentSuffix, + Lucene99ScalarQuantizedVectorsFormat.QUANTIZED_VECTOR_DATA_EXTENSION) + : null; + boolean success = false; + try { + meta = state.directory.createOutput(metaFileName, state.context); + vectorData = state.directory.createOutput(vectorDataFileName, state.context); + vectorIndex = state.directory.createOutput(indexDataFileName, state.context); + + CodecUtil.writeIndexHeader( + meta, + Lucene99HnswVectorsFormat.META_CODEC_NAME, + Lucene99HnswVectorsFormat.VERSION_CURRENT, + state.segmentInfo.getId(), + state.segmentSuffix); + CodecUtil.writeIndexHeader( + vectorData, + Lucene99HnswVectorsFormat.VECTOR_DATA_CODEC_NAME, + Lucene99HnswVectorsFormat.VERSION_CURRENT, + state.segmentInfo.getId(), + state.segmentSuffix); + CodecUtil.writeIndexHeader( + vectorIndex, + Lucene99HnswVectorsFormat.VECTOR_INDEX_CODEC_NAME, + Lucene99HnswVectorsFormat.VERSION_CURRENT, + state.segmentInfo.getId(), + state.segmentSuffix); + if (quantizedVectorDataFileName != null) { + quantizedVectorData = + state.directory.createOutput(quantizedVectorDataFileName, state.context); + CodecUtil.writeIndexHeader( + quantizedVectorData, + Lucene99ScalarQuantizedVectorsFormat.QUANTIZED_VECTOR_DATA_CODEC_NAME, + Lucene99ScalarQuantizedVectorsFormat.VERSION_CURRENT, + state.segmentInfo.getId(), + state.segmentSuffix); + quantizedVectorsWriter = + new Lucene99ScalarQuantizedVectorsWriter( + quantizedVectorData, quantizedVectorsFormat.quantile); Review Comment: file life cycle is owned by the HNSW writer, if we separate out flat & index, these would be extracted out of here. ########## lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsWriter.java: ########## @@ -0,0 +1,1170 @@ +/* + * 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. + */ + +package org.apache.lucene.codecs.lucene99; + +import static org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat.DIRECT_MONOTONIC_BLOCK_SHIFT; +import static org.apache.lucene.codecs.lucene99.Lucene99ScalarQuantizedVectorsFormat.calculateDefaultQuantile; +import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.HnswGraphProvider; +import org.apache.lucene.codecs.KnnFieldVectorsWriter; +import org.apache.lucene.codecs.KnnVectorsReader; +import org.apache.lucene.codecs.KnnVectorsWriter; +import org.apache.lucene.codecs.lucene90.IndexedDISI; +import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat; +import org.apache.lucene.index.ByteVectorValues; +import org.apache.lucene.index.DocsWithFieldSet; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FloatVectorValues; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.MergeState; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.index.Sorter; +import org.apache.lucene.index.VectorEncoding; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.InfoStream; +import org.apache.lucene.util.RamUsageEstimator; +import org.apache.lucene.util.hnsw.HnswGraph; +import org.apache.lucene.util.hnsw.HnswGraph.NodesIterator; +import org.apache.lucene.util.hnsw.HnswGraphBuilder; +import org.apache.lucene.util.hnsw.NeighborArray; +import org.apache.lucene.util.hnsw.OnHeapHnswGraph; +import org.apache.lucene.util.hnsw.RandomAccessVectorValues; +import org.apache.lucene.util.hnsw.RandomVectorScorerSupplier; +import org.apache.lucene.util.packed.DirectMonotonicWriter; + +/** + * Writes vector values and knn graphs to index segments. + * + * @lucene.experimental + */ +public final class Lucene99HnswVectorsWriter extends KnnVectorsWriter { + + private final SegmentWriteState segmentWriteState; + private final IndexOutput meta, vectorData, quantizedVectorData, vectorIndex; + private final int M; + private final int beamWidth; + private final Lucene99ScalarQuantizedVectorsWriter quantizedVectorsWriter; + + private final List<FieldWriter<?>> fields = new ArrayList<>(); + private boolean finished; + + Lucene99HnswVectorsWriter( + SegmentWriteState state, + int M, + int beamWidth, + Lucene99ScalarQuantizedVectorsFormat quantizedVectorsFormat) + throws IOException { + this.M = M; + this.beamWidth = beamWidth; + segmentWriteState = state; + String metaFileName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, state.segmentSuffix, Lucene99HnswVectorsFormat.META_EXTENSION); + + String vectorDataFileName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, + state.segmentSuffix, + Lucene99HnswVectorsFormat.VECTOR_DATA_EXTENSION); + + String indexDataFileName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, + state.segmentSuffix, + Lucene99HnswVectorsFormat.VECTOR_INDEX_EXTENSION); + + final String quantizedVectorDataFileName = + quantizedVectorsFormat != null + ? IndexFileNames.segmentFileName( + state.segmentInfo.name, + state.segmentSuffix, + Lucene99ScalarQuantizedVectorsFormat.QUANTIZED_VECTOR_DATA_EXTENSION) + : null; + boolean success = false; + try { + meta = state.directory.createOutput(metaFileName, state.context); + vectorData = state.directory.createOutput(vectorDataFileName, state.context); + vectorIndex = state.directory.createOutput(indexDataFileName, state.context); + + CodecUtil.writeIndexHeader( + meta, + Lucene99HnswVectorsFormat.META_CODEC_NAME, + Lucene99HnswVectorsFormat.VERSION_CURRENT, + state.segmentInfo.getId(), + state.segmentSuffix); + CodecUtil.writeIndexHeader( + vectorData, + Lucene99HnswVectorsFormat.VECTOR_DATA_CODEC_NAME, + Lucene99HnswVectorsFormat.VERSION_CURRENT, + state.segmentInfo.getId(), + state.segmentSuffix); + CodecUtil.writeIndexHeader( + vectorIndex, + Lucene99HnswVectorsFormat.VECTOR_INDEX_CODEC_NAME, + Lucene99HnswVectorsFormat.VERSION_CURRENT, + state.segmentInfo.getId(), + state.segmentSuffix); + if (quantizedVectorDataFileName != null) { + quantizedVectorData = + state.directory.createOutput(quantizedVectorDataFileName, state.context); + CodecUtil.writeIndexHeader( + quantizedVectorData, + Lucene99ScalarQuantizedVectorsFormat.QUANTIZED_VECTOR_DATA_CODEC_NAME, + Lucene99ScalarQuantizedVectorsFormat.VERSION_CURRENT, + state.segmentInfo.getId(), + state.segmentSuffix); + quantizedVectorsWriter = + new Lucene99ScalarQuantizedVectorsWriter( + quantizedVectorData, quantizedVectorsFormat.quantile); + } else { + quantizedVectorData = null; + quantizedVectorsWriter = null; + } + success = true; + } finally { + if (success == false) { + IOUtils.closeWhileHandlingException(this); + } + } + } + + @Override + public KnnFieldVectorsWriter<?> addField(FieldInfo fieldInfo) throws IOException { + Lucene99ScalarQuantizedVectorsWriter.QuantizationVectorWriter quantizedVectorFieldWriter = null; + // Quantization only supports FLOAT32 for now + if (quantizedVectorsWriter != null + && fieldInfo.getVectorEncoding().equals(VectorEncoding.FLOAT32)) { + quantizedVectorFieldWriter = quantizedVectorsWriter.addField(fieldInfo); + } + FieldWriter<?> newField = + FieldWriter.create( + fieldInfo, M, beamWidth, segmentWriteState.infoStream, quantizedVectorFieldWriter); + fields.add(newField); + return newField; + } + + @Override + public void flush(int maxDoc, Sorter.DocMap sortMap) throws IOException { + for (FieldWriter<?> field : fields) { + long[] quantizedVectorOffsetAndLen = null; + if (field.quantizedWriter != null) { + assert quantizedVectorsWriter != null; + quantizedVectorOffsetAndLen = + quantizedVectorsWriter.flush(sortMap, field.quantizedWriter, field.docsWithField); + } Review Comment: this separates concerns, but I couldn't see a nicer API for bubbling up offsets from the `veq` file for the given field. Basically, this writes the quantized vectors and returns the offset & len within the `veq` file. ########## lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsWriter.java: ########## @@ -0,0 +1,1170 @@ +/* + * 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. + */ + +package org.apache.lucene.codecs.lucene99; + +import static org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat.DIRECT_MONOTONIC_BLOCK_SHIFT; +import static org.apache.lucene.codecs.lucene99.Lucene99ScalarQuantizedVectorsFormat.calculateDefaultQuantile; +import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.HnswGraphProvider; +import org.apache.lucene.codecs.KnnFieldVectorsWriter; +import org.apache.lucene.codecs.KnnVectorsReader; +import org.apache.lucene.codecs.KnnVectorsWriter; +import org.apache.lucene.codecs.lucene90.IndexedDISI; +import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat; +import org.apache.lucene.index.ByteVectorValues; +import org.apache.lucene.index.DocsWithFieldSet; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FloatVectorValues; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.MergeState; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.index.Sorter; +import org.apache.lucene.index.VectorEncoding; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.InfoStream; +import org.apache.lucene.util.RamUsageEstimator; +import org.apache.lucene.util.hnsw.HnswGraph; +import org.apache.lucene.util.hnsw.HnswGraph.NodesIterator; +import org.apache.lucene.util.hnsw.HnswGraphBuilder; +import org.apache.lucene.util.hnsw.NeighborArray; +import org.apache.lucene.util.hnsw.OnHeapHnswGraph; +import org.apache.lucene.util.hnsw.RandomAccessVectorValues; +import org.apache.lucene.util.hnsw.RandomVectorScorerSupplier; +import org.apache.lucene.util.packed.DirectMonotonicWriter; + +/** + * Writes vector values and knn graphs to index segments. + * + * @lucene.experimental + */ +public final class Lucene99HnswVectorsWriter extends KnnVectorsWriter { + + private final SegmentWriteState segmentWriteState; + private final IndexOutput meta, vectorData, quantizedVectorData, vectorIndex; + private final int M; + private final int beamWidth; + private final Lucene99ScalarQuantizedVectorsWriter quantizedVectorsWriter; + + private final List<FieldWriter<?>> fields = new ArrayList<>(); + private boolean finished; + + Lucene99HnswVectorsWriter( + SegmentWriteState state, + int M, + int beamWidth, + Lucene99ScalarQuantizedVectorsFormat quantizedVectorsFormat) + throws IOException { + this.M = M; + this.beamWidth = beamWidth; + segmentWriteState = state; + String metaFileName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, state.segmentSuffix, Lucene99HnswVectorsFormat.META_EXTENSION); + + String vectorDataFileName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, + state.segmentSuffix, + Lucene99HnswVectorsFormat.VECTOR_DATA_EXTENSION); + + String indexDataFileName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, + state.segmentSuffix, + Lucene99HnswVectorsFormat.VECTOR_INDEX_EXTENSION); + + final String quantizedVectorDataFileName = + quantizedVectorsFormat != null + ? IndexFileNames.segmentFileName( + state.segmentInfo.name, + state.segmentSuffix, + Lucene99ScalarQuantizedVectorsFormat.QUANTIZED_VECTOR_DATA_EXTENSION) + : null; + boolean success = false; + try { + meta = state.directory.createOutput(metaFileName, state.context); + vectorData = state.directory.createOutput(vectorDataFileName, state.context); + vectorIndex = state.directory.createOutput(indexDataFileName, state.context); + + CodecUtil.writeIndexHeader( + meta, + Lucene99HnswVectorsFormat.META_CODEC_NAME, + Lucene99HnswVectorsFormat.VERSION_CURRENT, + state.segmentInfo.getId(), + state.segmentSuffix); + CodecUtil.writeIndexHeader( + vectorData, + Lucene99HnswVectorsFormat.VECTOR_DATA_CODEC_NAME, + Lucene99HnswVectorsFormat.VERSION_CURRENT, + state.segmentInfo.getId(), + state.segmentSuffix); + CodecUtil.writeIndexHeader( + vectorIndex, + Lucene99HnswVectorsFormat.VECTOR_INDEX_CODEC_NAME, + Lucene99HnswVectorsFormat.VERSION_CURRENT, + state.segmentInfo.getId(), + state.segmentSuffix); + if (quantizedVectorDataFileName != null) { + quantizedVectorData = + state.directory.createOutput(quantizedVectorDataFileName, state.context); + CodecUtil.writeIndexHeader( + quantizedVectorData, + Lucene99ScalarQuantizedVectorsFormat.QUANTIZED_VECTOR_DATA_CODEC_NAME, + Lucene99ScalarQuantizedVectorsFormat.VERSION_CURRENT, + state.segmentInfo.getId(), + state.segmentSuffix); + quantizedVectorsWriter = + new Lucene99ScalarQuantizedVectorsWriter( + quantizedVectorData, quantizedVectorsFormat.quantile); + } else { + quantizedVectorData = null; + quantizedVectorsWriter = null; + } + success = true; + } finally { + if (success == false) { + IOUtils.closeWhileHandlingException(this); + } + } + } + + @Override + public KnnFieldVectorsWriter<?> addField(FieldInfo fieldInfo) throws IOException { + Lucene99ScalarQuantizedVectorsWriter.QuantizationVectorWriter quantizedVectorFieldWriter = null; + // Quantization only supports FLOAT32 for now + if (quantizedVectorsWriter != null + && fieldInfo.getVectorEncoding().equals(VectorEncoding.FLOAT32)) { + quantizedVectorFieldWriter = quantizedVectorsWriter.addField(fieldInfo); + } + FieldWriter<?> newField = + FieldWriter.create( + fieldInfo, M, beamWidth, segmentWriteState.infoStream, quantizedVectorFieldWriter); + fields.add(newField); + return newField; + } + + @Override + public void flush(int maxDoc, Sorter.DocMap sortMap) throws IOException { + for (FieldWriter<?> field : fields) { + long[] quantizedVectorOffsetAndLen = null; + if (field.quantizedWriter != null) { + assert quantizedVectorsWriter != null; + quantizedVectorOffsetAndLen = + quantizedVectorsWriter.flush(sortMap, field.quantizedWriter, field.docsWithField); + } + if (sortMap == null) { + writeField(field, maxDoc, quantizedVectorOffsetAndLen); + } else { + writeSortingField(field, maxDoc, sortMap, quantizedVectorOffsetAndLen); + } + } + } + + @Override + public void finish() throws IOException { + if (finished) { + throw new IllegalStateException("already finished"); + } + finished = true; + if (quantizedVectorsWriter != null) { + quantizedVectorsWriter.finish(); + } + + if (meta != null) { + // write end of fields marker + meta.writeInt(-1); + CodecUtil.writeFooter(meta); + } + if (vectorData != null) { + CodecUtil.writeFooter(vectorData); + CodecUtil.writeFooter(vectorIndex); + } + } + + @Override + public long ramBytesUsed() { + long total = 0; + for (FieldWriter<?> field : fields) { + total += field.ramBytesUsed(); + } + return total; + } + + private void writeField(FieldWriter<?> fieldData, int maxDoc, long[] quantizedVecOffsetAndLen) + throws IOException { + // write vector values + long vectorDataOffset = vectorData.alignFilePointer(Float.BYTES); + switch (fieldData.fieldInfo.getVectorEncoding()) { + case BYTE -> writeByteVectors(fieldData); + case FLOAT32 -> writeFloat32Vectors(fieldData); + } + long vectorDataLength = vectorData.getFilePointer() - vectorDataOffset; + + // write graph + long vectorIndexOffset = vectorIndex.getFilePointer(); + OnHeapHnswGraph graph = fieldData.getGraph(); + int[][] graphLevelNodeOffsets = writeGraph(graph); + long vectorIndexLength = vectorIndex.getFilePointer() - vectorIndexOffset; + + writeMeta( + fieldData.isQuantized(), + fieldData.fieldInfo, + maxDoc, + fieldData.getConfiguredQuantile(), + fieldData.getMinQuantile(), + fieldData.getMaxQuantile(), + quantizedVecOffsetAndLen, + vectorDataOffset, + vectorDataLength, + vectorIndexOffset, + vectorIndexLength, + fieldData.docsWithField, + graph, + graphLevelNodeOffsets); + } + + private void writeFloat32Vectors(FieldWriter<?> fieldData) throws IOException { + final ByteBuffer buffer = + ByteBuffer.allocate(fieldData.dim * Float.BYTES).order(ByteOrder.LITTLE_ENDIAN); + for (Object v : fieldData.vectors) { + buffer.asFloatBuffer().put((float[]) v); + vectorData.writeBytes(buffer.array(), buffer.array().length); + } + } + + private void writeByteVectors(FieldWriter<?> fieldData) throws IOException { + for (Object v : fieldData.vectors) { + byte[] vector = (byte[]) v; + vectorData.writeBytes(vector, vector.length); + } + } + + private void writeSortingField( + FieldWriter<?> fieldData, + int maxDoc, + Sorter.DocMap sortMap, + long[] quantizedVectorOffsetAndLen) + throws IOException { + final int[] docIdOffsets = new int[sortMap.size()]; + int offset = 1; // 0 means no vector for this (field, document) + DocIdSetIterator iterator = fieldData.docsWithField.iterator(); + for (int docID = iterator.nextDoc(); + docID != DocIdSetIterator.NO_MORE_DOCS; + docID = iterator.nextDoc()) { + int newDocID = sortMap.oldToNew(docID); + docIdOffsets[newDocID] = offset++; + } + DocsWithFieldSet newDocsWithField = new DocsWithFieldSet(); + final int[] ordMap = new int[offset - 1]; // new ord to old ord + final int[] oldOrdMap = new int[offset - 1]; // old ord to new ord + int ord = 0; + int doc = 0; + for (int docIdOffset : docIdOffsets) { + if (docIdOffset != 0) { + ordMap[ord] = docIdOffset - 1; + oldOrdMap[docIdOffset - 1] = ord; + newDocsWithField.add(doc); + ord++; + } + doc++; + } + + // write vector values + long vectorDataOffset = + switch (fieldData.fieldInfo.getVectorEncoding()) { + case BYTE -> writeSortedByteVectors(fieldData, ordMap); + case FLOAT32 -> writeSortedFloat32Vectors(fieldData, ordMap); + }; + long vectorDataLength = vectorData.getFilePointer() - vectorDataOffset; + + // write graph + long vectorIndexOffset = vectorIndex.getFilePointer(); + OnHeapHnswGraph graph = fieldData.getGraph(); + int[][] graphLevelNodeOffsets = graph == null ? new int[0][] : new int[graph.numLevels()][]; + HnswGraph mockGraph = reconstructAndWriteGraph(graph, ordMap, oldOrdMap, graphLevelNodeOffsets); + long vectorIndexLength = vectorIndex.getFilePointer() - vectorIndexOffset; + + writeMeta( + fieldData.isQuantized(), + fieldData.fieldInfo, + maxDoc, + fieldData.getConfiguredQuantile(), + fieldData.getMinQuantile(), + fieldData.getMaxQuantile(), + quantizedVectorOffsetAndLen, + vectorDataOffset, + vectorDataLength, + vectorIndexOffset, + vectorIndexLength, + newDocsWithField, + mockGraph, + graphLevelNodeOffsets); + } + + private long writeSortedFloat32Vectors(FieldWriter<?> fieldData, int[] ordMap) + throws IOException { + long vectorDataOffset = vectorData.alignFilePointer(Float.BYTES); + final ByteBuffer buffer = + ByteBuffer.allocate(fieldData.dim * Float.BYTES).order(ByteOrder.LITTLE_ENDIAN); + for (int ordinal : ordMap) { + float[] vector = (float[]) fieldData.vectors.get(ordinal); + buffer.asFloatBuffer().put(vector); + vectorData.writeBytes(buffer.array(), buffer.array().length); + } + return vectorDataOffset; + } + + private long writeSortedByteVectors(FieldWriter<?> fieldData, int[] ordMap) throws IOException { + long vectorDataOffset = vectorData.alignFilePointer(Float.BYTES); + for (int ordinal : ordMap) { + byte[] vector = (byte[]) fieldData.vectors.get(ordinal); + vectorData.writeBytes(vector, vector.length); + } + return vectorDataOffset; + } + + /** + * Reconstructs the graph given the old and new node ids. + * + * <p>Additionally, the graph node connections are written to the vectorIndex. + * + * @param graph The current on heap graph + * @param newToOldMap the new node ids indexed to the old node ids + * @param oldToNewMap the old node ids indexed to the new node ids + * @param levelNodeOffsets where to place the new offsets for the nodes in the vector index. + * @return The graph + * @throws IOException if writing to vectorIndex fails + */ + private HnswGraph reconstructAndWriteGraph( + OnHeapHnswGraph graph, int[] newToOldMap, int[] oldToNewMap, int[][] levelNodeOffsets) + throws IOException { + if (graph == null) return null; + + List<int[]> nodesByLevel = new ArrayList<>(graph.numLevels()); + nodesByLevel.add(null); + + int maxOrd = graph.size(); + NodesIterator nodesOnLevel0 = graph.getNodesOnLevel(0); + levelNodeOffsets[0] = new int[nodesOnLevel0.size()]; + while (nodesOnLevel0.hasNext()) { + int node = nodesOnLevel0.nextInt(); + NeighborArray neighbors = graph.getNeighbors(0, newToOldMap[node]); + long offset = vectorIndex.getFilePointer(); + reconstructAndWriteNeigbours(neighbors, oldToNewMap, maxOrd); + levelNodeOffsets[0][node] = Math.toIntExact(vectorIndex.getFilePointer() - offset); + } + + for (int level = 1; level < graph.numLevels(); level++) { + NodesIterator nodesOnLevel = graph.getNodesOnLevel(level); + int[] newNodes = new int[nodesOnLevel.size()]; + for (int n = 0; nodesOnLevel.hasNext(); n++) { + newNodes[n] = oldToNewMap[nodesOnLevel.nextInt()]; + } + Arrays.sort(newNodes); + nodesByLevel.add(newNodes); + levelNodeOffsets[level] = new int[newNodes.length]; + int nodeOffsetIndex = 0; + for (int node : newNodes) { + NeighborArray neighbors = graph.getNeighbors(level, newToOldMap[node]); + long offset = vectorIndex.getFilePointer(); + reconstructAndWriteNeigbours(neighbors, oldToNewMap, maxOrd); + levelNodeOffsets[level][nodeOffsetIndex++] = + Math.toIntExact(vectorIndex.getFilePointer() - offset); + } + } + return new HnswGraph() { + @Override + public int nextNeighbor() { + throw new UnsupportedOperationException("Not supported on a mock graph"); + } + + @Override + public void seek(int level, int target) { + throw new UnsupportedOperationException("Not supported on a mock graph"); + } + + @Override + public int size() { + return graph.size(); + } + + @Override + public int numLevels() { + return graph.numLevels(); + } + + @Override + public int entryNode() { + throw new UnsupportedOperationException("Not supported on a mock graph"); + } + + @Override + public NodesIterator getNodesOnLevel(int level) { + if (level == 0) { + return graph.getNodesOnLevel(0); + } else { + return new ArrayNodesIterator(nodesByLevel.get(level), nodesByLevel.get(level).length); + } + } + }; + } + + private void reconstructAndWriteNeigbours(NeighborArray neighbors, int[] oldToNewMap, int maxOrd) + throws IOException { + int size = neighbors.size(); + vectorIndex.writeVInt(size); + + // Destructively modify; it's ok we are discarding it after this + int[] nnodes = neighbors.node(); + for (int i = 0; i < size; i++) { + nnodes[i] = oldToNewMap[nnodes[i]]; + } + Arrays.sort(nnodes, 0, size); + // Now that we have sorted, do delta encoding to minimize the required bits to store the + // information + for (int i = size - 1; i > 0; --i) { + assert nnodes[i] < maxOrd : "node too large: " + nnodes[i] + ">=" + maxOrd; + nnodes[i] -= nnodes[i - 1]; + } + for (int i = 0; i < size; i++) { + vectorIndex.writeVInt(nnodes[i]); + } + } + + @Override + public void mergeOneField(FieldInfo fieldInfo, MergeState mergeState) throws IOException { + long vectorDataOffset = vectorData.alignFilePointer(Float.BYTES); + IndexOutput tempVectorData = + segmentWriteState.directory.createTempOutput( + vectorData.getName(), "temp", segmentWriteState.context); + IndexInput vectorDataInput = null; + IndexInput quantizationDataInput = null; + Lucene99ScalarQuantizedVectorsWriter.MergedQuantileState quantizationState = null; + boolean success = false; + try { + long[] quantizedVectorDataOffsetAndLength = null; + if (quantizedVectorsWriter != null) { + quantizationState = + quantizedVectorsWriter.mergeOneField(segmentWriteState, fieldInfo, mergeState); + if (quantizationState != null) { + quantizedVectorDataOffsetAndLength = new long[2]; + quantizedVectorDataOffsetAndLength[0] = quantizedVectorData.alignFilePointer(Float.BYTES); + quantizationDataInput = + segmentWriteState.directory.openInput( + quantizationState.tempVectorFileName, segmentWriteState.context); + quantizedVectorData.copyBytes( + quantizationDataInput, quantizationDataInput.length() - CodecUtil.footerLength()); + quantizedVectorDataOffsetAndLength[1] = + quantizedVectorData.getFilePointer() - quantizedVectorDataOffsetAndLength[0]; + CodecUtil.retrieveChecksum(quantizationDataInput); + } Review Comment: For merging and reading, I need to: - merge the quantile information and determine if we should recalculate quantiles and requantize - Merge the quantized files into a single file - provide some way to then read the merged file using the merged quantization settings for graph building. This seemed like the best way at the time (return the temp-file name and allow the hnsw writer read and delete it later). ########## lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsReader.java: ########## @@ -0,0 +1,739 @@ +/* + * 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. + */ + +package org.apache.lucene.codecs.lucene99; + +import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.HnswGraphProvider; +import org.apache.lucene.codecs.KnnVectorsReader; +import org.apache.lucene.codecs.lucene90.IndexedDISI; +import org.apache.lucene.index.ByteVectorValues; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.index.FloatVectorValues; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.SegmentReadState; +import org.apache.lucene.index.VectorEncoding; +import org.apache.lucene.index.VectorSimilarityFunction; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.search.KnnCollector; +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.Bits; +import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.RamUsageEstimator; +import org.apache.lucene.util.ScalarQuantizer; +import org.apache.lucene.util.hnsw.HnswGraph; +import org.apache.lucene.util.hnsw.HnswGraphSearcher; +import org.apache.lucene.util.hnsw.OrdinalTranslatedKnnCollector; +import org.apache.lucene.util.hnsw.RandomVectorScorer; +import org.apache.lucene.util.packed.DirectMonotonicReader; + +/** + * Reads vectors from the index segments along with index data structures supporting KNN search. + * + * @lucene.experimental + */ +public final class Lucene99HnswVectorsReader extends KnnVectorsReader + implements QuantizedVectorsReader, HnswGraphProvider { + + private static final long SHALLOW_SIZE = + RamUsageEstimator.shallowSizeOfInstance(Lucene99HnswVectorsFormat.class); + + private final FieldInfos fieldInfos; + private final Map<String, FieldEntry> fields = new HashMap<>(); + private final IndexInput vectorData; + private final IndexInput vectorIndex; + private final IndexInput quantizedVectorData; + private final Lucene99ScalarQuantizedVectorsReader quantizedVectorsReader; + + Lucene99HnswVectorsReader(SegmentReadState state) throws IOException { + this.fieldInfos = state.fieldInfos; + int versionMeta = readMetadata(state); + boolean success = false; + try { + vectorData = + openDataInput( + state, + versionMeta, + Lucene99HnswVectorsFormat.VECTOR_DATA_EXTENSION, + Lucene99HnswVectorsFormat.VECTOR_DATA_CODEC_NAME); + vectorIndex = + openDataInput( + state, + versionMeta, + Lucene99HnswVectorsFormat.VECTOR_INDEX_EXTENSION, + Lucene99HnswVectorsFormat.VECTOR_INDEX_CODEC_NAME); + if (fields.values().stream().anyMatch(FieldEntry::hasQuantizedVectors)) { + quantizedVectorData = + openDataInput( + state, + versionMeta, + Lucene99ScalarQuantizedVectorsFormat.QUANTIZED_VECTOR_DATA_EXTENSION, + Lucene99ScalarQuantizedVectorsFormat.QUANTIZED_VECTOR_DATA_CODEC_NAME); + quantizedVectorsReader = new Lucene99ScalarQuantizedVectorsReader(quantizedVectorData); + } else { + quantizedVectorData = null; + quantizedVectorsReader = null; + } + success = true; + } finally { + if (success == false) { + IOUtils.closeWhileHandlingException(this); + } + } + } + + private int readMetadata(SegmentReadState state) throws IOException { + String metaFileName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, state.segmentSuffix, Lucene99HnswVectorsFormat.META_EXTENSION); + int versionMeta = -1; + try (ChecksumIndexInput meta = state.directory.openChecksumInput(metaFileName)) { + Throwable priorE = null; + try { + versionMeta = + CodecUtil.checkIndexHeader( + meta, + Lucene99HnswVectorsFormat.META_CODEC_NAME, + Lucene99HnswVectorsFormat.VERSION_START, + Lucene99HnswVectorsFormat.VERSION_CURRENT, + state.segmentInfo.getId(), + state.segmentSuffix); + readFields(meta, state.fieldInfos); + } catch (Throwable exception) { + priorE = exception; + } finally { + CodecUtil.checkFooter(meta, priorE); + } + } + return versionMeta; + } + + private static IndexInput openDataInput( + SegmentReadState state, int versionMeta, String fileExtension, String codecName) + throws IOException { + String fileName = + IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, fileExtension); + IndexInput in = state.directory.openInput(fileName, state.context); + boolean success = false; + try { + int versionVectorData = + CodecUtil.checkIndexHeader( + in, + codecName, + Lucene99HnswVectorsFormat.VERSION_START, + Lucene99HnswVectorsFormat.VERSION_CURRENT, + state.segmentInfo.getId(), + state.segmentSuffix); + if (versionMeta != versionVectorData) { + throw new CorruptIndexException( + "Format versions mismatch: meta=" + + versionMeta + + ", " + + codecName + + "=" + + versionVectorData, + in); + } + CodecUtil.retrieveChecksum(in); + success = true; + return in; + } finally { + if (success == false) { + IOUtils.closeWhileHandlingException(in); + } + } + } + + private void readFields(ChecksumIndexInput meta, FieldInfos infos) throws IOException { + for (int fieldNumber = meta.readInt(); fieldNumber != -1; fieldNumber = meta.readInt()) { + FieldInfo info = infos.fieldInfo(fieldNumber); + if (info == null) { + throw new CorruptIndexException("Invalid field number: " + fieldNumber, meta); + } + FieldEntry fieldEntry = readField(meta); + validateFieldEntry(info, fieldEntry); + fields.put(info.name, fieldEntry); + } + } + + private void validateFieldEntry(FieldInfo info, FieldEntry fieldEntry) { + int dimension = info.getVectorDimension(); + if (dimension != fieldEntry.dimension) { + throw new IllegalStateException( + "Inconsistent vector dimension for field=\"" + + info.name + + "\"; " + + dimension + + " != " + + fieldEntry.dimension); + } + + int byteSize = + switch (info.getVectorEncoding()) { + case BYTE -> Byte.BYTES; + case FLOAT32 -> Float.BYTES; + }; + long vectorBytes = Math.multiplyExact((long) dimension, byteSize); + long numBytes = Math.multiplyExact(vectorBytes, fieldEntry.size); + if (numBytes != fieldEntry.vectorDataLength) { + throw new IllegalStateException( + "Vector data length " + + fieldEntry.vectorDataLength + + " not matching size=" + + fieldEntry.size + + " * dim=" + + dimension + + " * byteSize=" + + byteSize + + " = " + + numBytes); + } + if (fieldEntry.hasQuantizedVectors()) { + Lucene99ScalarQuantizedVectorsReader.validateFieldEntry( + info, fieldEntry.dimension, fieldEntry.size, fieldEntry.quantizedVectorDataLength); + } + } + + private VectorSimilarityFunction readSimilarityFunction(DataInput input) throws IOException { + int similarityFunctionId = input.readInt(); + if (similarityFunctionId < 0 + || similarityFunctionId >= VectorSimilarityFunction.values().length) { + throw new CorruptIndexException( + "Invalid similarity function id: " + similarityFunctionId, input); + } + return VectorSimilarityFunction.values()[similarityFunctionId]; + } + + private VectorEncoding readVectorEncoding(DataInput input) throws IOException { + int encodingId = input.readInt(); + if (encodingId < 0 || encodingId >= VectorEncoding.values().length) { + throw new CorruptIndexException("Invalid vector encoding id: " + encodingId, input); + } + return VectorEncoding.values()[encodingId]; + } + + private FieldEntry readField(IndexInput input) throws IOException { + VectorEncoding vectorEncoding = readVectorEncoding(input); + VectorSimilarityFunction similarityFunction = readSimilarityFunction(input); + return new FieldEntry(input, vectorEncoding, similarityFunction); + } + + @Override + public long ramBytesUsed() { + return Lucene99HnswVectorsReader.SHALLOW_SIZE + + RamUsageEstimator.sizeOfMap( + fields, RamUsageEstimator.shallowSizeOfInstance(FieldEntry.class)); + } + + @Override + public void checkIntegrity() throws IOException { + CodecUtil.checksumEntireFile(vectorData); + CodecUtil.checksumEntireFile(vectorIndex); + if (quantizedVectorsReader != null) { + quantizedVectorsReader.checkIntegrity(); + } + } + + @Override + public FloatVectorValues getFloatVectorValues(String field) throws IOException { + FieldEntry fieldEntry = fields.get(field); + if (fieldEntry.vectorEncoding != VectorEncoding.FLOAT32) { + throw new IllegalArgumentException( + "field=\"" + + field + + "\" is encoded as: " + + fieldEntry.vectorEncoding + + " expected: " + + VectorEncoding.FLOAT32); + } + return OffHeapFloatVectorValues.load(fieldEntry, vectorData); + } + + @Override + public ByteVectorValues getByteVectorValues(String field) throws IOException { + FieldEntry fieldEntry = fields.get(field); + if (fieldEntry.vectorEncoding != VectorEncoding.BYTE) { + throw new IllegalArgumentException( + "field=\"" + + field + + "\" is encoded as: " + + fieldEntry.vectorEncoding + + " expected: " + + VectorEncoding.FLOAT32); + } + return OffHeapByteVectorValues.load(fieldEntry, vectorData); + } + + @Override + public void search(String field, float[] target, KnnCollector knnCollector, Bits acceptDocs) + throws IOException { + FieldEntry fieldEntry = fields.get(field); + + if (fieldEntry.size() == 0 + || knnCollector.k() == 0 + || fieldEntry.vectorEncoding != VectorEncoding.FLOAT32) { + return; + } + if (quantizedVectorsReader != null) { + OffHeapQuantizedByteVectorValues vectorValues = getQuantizedVectorValues(field); + if (vectorValues == null) { + return; + } + RandomVectorScorer scorer = + new ScalarQuantizedRandomVectorScorer( + fieldEntry.similarityFunction, fieldEntry.scalarQuantizer, vectorValues, target); + HnswGraphSearcher.search( + scorer, + new OrdinalTranslatedKnnCollector(knnCollector, vectorValues::ordToDoc), + getGraph(fieldEntry), + vectorValues.getAcceptOrds(acceptDocs)); + } else { + OffHeapFloatVectorValues vectorValues = OffHeapFloatVectorValues.load(fieldEntry, vectorData); + RandomVectorScorer scorer = + RandomVectorScorer.createFloats(vectorValues, fieldEntry.similarityFunction, target); + HnswGraphSearcher.search( + scorer, + new OrdinalTranslatedKnnCollector(knnCollector, vectorValues::ordToDoc), + getGraph(fieldEntry), + vectorValues.getAcceptOrds(acceptDocs)); + } + } + + @Override + public void search(String field, byte[] target, KnnCollector knnCollector, Bits acceptDocs) + throws IOException { + FieldEntry fieldEntry = fields.get(field); + + if (fieldEntry.size() == 0 + || knnCollector.k() == 0 + || fieldEntry.vectorEncoding != VectorEncoding.BYTE) { + return; + } + + OffHeapByteVectorValues vectorValues = OffHeapByteVectorValues.load(fieldEntry, vectorData); + RandomVectorScorer scorer = + RandomVectorScorer.createBytes(vectorValues, fieldEntry.similarityFunction, target); + HnswGraphSearcher.search( + scorer, + new OrdinalTranslatedKnnCollector(knnCollector, vectorValues::ordToDoc), + getGraph(fieldEntry), + vectorValues.getAcceptOrds(acceptDocs)); + } + + @Override + public HnswGraph getGraph(String field) throws IOException { + FieldInfo info = fieldInfos.fieldInfo(field); + if (info == null) { + throw new IllegalArgumentException("No such field '" + field + "'"); + } + FieldEntry entry = fields.get(field); + if (entry != null && entry.vectorIndexLength > 0) { + return getGraph(entry); + } else { + return HnswGraph.EMPTY; + } + } + + private HnswGraph getGraph(FieldEntry entry) throws IOException { + return new OffHeapHnswGraph(entry, vectorIndex); + } + + @Override + public void close() throws IOException { + IOUtils.close(vectorData, vectorIndex, quantizedVectorData); + } + + @Override + public OffHeapQuantizedByteVectorValues getQuantizedVectorValues(String field) + throws IOException { + FieldEntry fieldEntry = fields.get(field); + if (fieldEntry == null || fieldEntry.hasQuantizedVectors() == false) { + return null; + } + assert quantizedVectorsReader != null && fieldEntry.quantizedOrdToDoc != null; + return quantizedVectorsReader.getQuantizedVectorValues( + fieldEntry.quantizedOrdToDoc, + fieldEntry.dimension, + fieldEntry.size, + fieldEntry.quantizedVectorDataOffset, + fieldEntry.quantizedVectorDataLength); + } + + @Override + public ScalarQuantizer getQuantizationState(String fieldName) { + FieldEntry field = fields.get(fieldName); + if (field == null || field.hasQuantizedVectors() == false) { + return null; + } + return field.scalarQuantizer; + } + + static class FieldEntry implements Accountable { + private static final long SHALLOW_SIZE = + RamUsageEstimator.shallowSizeOfInstance(FieldEntry.class); + final VectorSimilarityFunction similarityFunction; + final VectorEncoding vectorEncoding; + final long vectorDataOffset; + final long vectorDataLength; + final long vectorIndexOffset; + final long vectorIndexLength; + final int M; + final int numLevels; + final int dimension; + final int size; + final int[][] nodesByLevel; + // for each level the start offsets in vectorIndex file from where to read neighbours + final DirectMonotonicReader.Meta offsetsMeta; + final long offsetsOffset; + final int offsetsBlockShift; + final long offsetsLength; + final OrdToDocDISReaderConfiguration ordToDoc; + + final float configuredQuantile, lowerQuantile, upperQuantile; + final long quantizedVectorDataOffset, quantizedVectorDataLength; + final ScalarQuantizer scalarQuantizer; + final boolean isQuantized; + final OrdToDocDISReaderConfiguration quantizedOrdToDoc; + + FieldEntry( + IndexInput input, + VectorEncoding vectorEncoding, + VectorSimilarityFunction similarityFunction) + throws IOException { + this.similarityFunction = similarityFunction; + this.vectorEncoding = vectorEncoding; + this.isQuantized = input.readByte() == 1; + // Has int8 quantization + if (isQuantized) { + configuredQuantile = Float.intBitsToFloat(input.readInt()); + lowerQuantile = Float.intBitsToFloat(input.readInt()); + upperQuantile = Float.intBitsToFloat(input.readInt()); + quantizedVectorDataOffset = input.readVLong(); + quantizedVectorDataLength = input.readVLong(); + scalarQuantizer = new ScalarQuantizer(lowerQuantile, upperQuantile); + } else { + configuredQuantile = -1; + lowerQuantile = -1; + upperQuantile = -1; + quantizedVectorDataOffset = -1; + quantizedVectorDataLength = -1; + scalarQuantizer = null; + } + vectorDataOffset = input.readVLong(); + vectorDataLength = input.readVLong(); + vectorIndexOffset = input.readVLong(); + vectorIndexLength = input.readVLong(); + dimension = input.readVInt(); + size = input.readInt(); + if (isQuantized) { + quantizedOrdToDoc = OrdToDocDISReaderConfiguration.fromStoredMeta(input, size); + } else { + quantizedOrdToDoc = null; + } + ordToDoc = OrdToDocDISReaderConfiguration.fromStoredMeta(input, size); + + /*docsWithFieldOffset = input.readLong(); Review Comment: dang, will delete, unnecessary code commented out. ########## lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsReader.java: ########## @@ -0,0 +1,739 @@ +/* + * 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. + */ + +package org.apache.lucene.codecs.lucene99; + +import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.HnswGraphProvider; +import org.apache.lucene.codecs.KnnVectorsReader; +import org.apache.lucene.codecs.lucene90.IndexedDISI; +import org.apache.lucene.index.ByteVectorValues; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.index.FloatVectorValues; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.SegmentReadState; +import org.apache.lucene.index.VectorEncoding; +import org.apache.lucene.index.VectorSimilarityFunction; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.search.KnnCollector; +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.Bits; +import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.RamUsageEstimator; +import org.apache.lucene.util.ScalarQuantizer; +import org.apache.lucene.util.hnsw.HnswGraph; +import org.apache.lucene.util.hnsw.HnswGraphSearcher; +import org.apache.lucene.util.hnsw.OrdinalTranslatedKnnCollector; +import org.apache.lucene.util.hnsw.RandomVectorScorer; +import org.apache.lucene.util.packed.DirectMonotonicReader; + +/** + * Reads vectors from the index segments along with index data structures supporting KNN search. + * + * @lucene.experimental + */ +public final class Lucene99HnswVectorsReader extends KnnVectorsReader + implements QuantizedVectorsReader, HnswGraphProvider { + + private static final long SHALLOW_SIZE = + RamUsageEstimator.shallowSizeOfInstance(Lucene99HnswVectorsFormat.class); + + private final FieldInfos fieldInfos; + private final Map<String, FieldEntry> fields = new HashMap<>(); + private final IndexInput vectorData; + private final IndexInput vectorIndex; + private final IndexInput quantizedVectorData; + private final Lucene99ScalarQuantizedVectorsReader quantizedVectorsReader; + + Lucene99HnswVectorsReader(SegmentReadState state) throws IOException { + this.fieldInfos = state.fieldInfos; + int versionMeta = readMetadata(state); + boolean success = false; + try { + vectorData = + openDataInput( + state, + versionMeta, + Lucene99HnswVectorsFormat.VECTOR_DATA_EXTENSION, + Lucene99HnswVectorsFormat.VECTOR_DATA_CODEC_NAME); + vectorIndex = + openDataInput( + state, + versionMeta, + Lucene99HnswVectorsFormat.VECTOR_INDEX_EXTENSION, + Lucene99HnswVectorsFormat.VECTOR_INDEX_CODEC_NAME); + if (fields.values().stream().anyMatch(FieldEntry::hasQuantizedVectors)) { + quantizedVectorData = + openDataInput( + state, + versionMeta, + Lucene99ScalarQuantizedVectorsFormat.QUANTIZED_VECTOR_DATA_EXTENSION, + Lucene99ScalarQuantizedVectorsFormat.QUANTIZED_VECTOR_DATA_CODEC_NAME); + quantizedVectorsReader = new Lucene99ScalarQuantizedVectorsReader(quantizedVectorData); + } else { + quantizedVectorData = null; + quantizedVectorsReader = null; + } + success = true; + } finally { + if (success == false) { + IOUtils.closeWhileHandlingException(this); + } + } + } + + private int readMetadata(SegmentReadState state) throws IOException { + String metaFileName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, state.segmentSuffix, Lucene99HnswVectorsFormat.META_EXTENSION); + int versionMeta = -1; + try (ChecksumIndexInput meta = state.directory.openChecksumInput(metaFileName)) { + Throwable priorE = null; + try { + versionMeta = + CodecUtil.checkIndexHeader( + meta, + Lucene99HnswVectorsFormat.META_CODEC_NAME, + Lucene99HnswVectorsFormat.VERSION_START, + Lucene99HnswVectorsFormat.VERSION_CURRENT, + state.segmentInfo.getId(), + state.segmentSuffix); + readFields(meta, state.fieldInfos); + } catch (Throwable exception) { + priorE = exception; + } finally { + CodecUtil.checkFooter(meta, priorE); + } + } + return versionMeta; + } + + private static IndexInput openDataInput( + SegmentReadState state, int versionMeta, String fileExtension, String codecName) + throws IOException { + String fileName = + IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, fileExtension); + IndexInput in = state.directory.openInput(fileName, state.context); + boolean success = false; + try { + int versionVectorData = + CodecUtil.checkIndexHeader( + in, + codecName, + Lucene99HnswVectorsFormat.VERSION_START, + Lucene99HnswVectorsFormat.VERSION_CURRENT, + state.segmentInfo.getId(), + state.segmentSuffix); + if (versionMeta != versionVectorData) { + throw new CorruptIndexException( + "Format versions mismatch: meta=" + + versionMeta + + ", " + + codecName + + "=" + + versionVectorData, + in); + } + CodecUtil.retrieveChecksum(in); + success = true; + return in; + } finally { + if (success == false) { + IOUtils.closeWhileHandlingException(in); + } + } + } + + private void readFields(ChecksumIndexInput meta, FieldInfos infos) throws IOException { + for (int fieldNumber = meta.readInt(); fieldNumber != -1; fieldNumber = meta.readInt()) { + FieldInfo info = infos.fieldInfo(fieldNumber); + if (info == null) { + throw new CorruptIndexException("Invalid field number: " + fieldNumber, meta); + } + FieldEntry fieldEntry = readField(meta); + validateFieldEntry(info, fieldEntry); + fields.put(info.name, fieldEntry); + } + } + + private void validateFieldEntry(FieldInfo info, FieldEntry fieldEntry) { + int dimension = info.getVectorDimension(); + if (dimension != fieldEntry.dimension) { + throw new IllegalStateException( + "Inconsistent vector dimension for field=\"" + + info.name + + "\"; " + + dimension + + " != " + + fieldEntry.dimension); + } + + int byteSize = + switch (info.getVectorEncoding()) { + case BYTE -> Byte.BYTES; + case FLOAT32 -> Float.BYTES; + }; + long vectorBytes = Math.multiplyExact((long) dimension, byteSize); + long numBytes = Math.multiplyExact(vectorBytes, fieldEntry.size); + if (numBytes != fieldEntry.vectorDataLength) { + throw new IllegalStateException( + "Vector data length " + + fieldEntry.vectorDataLength + + " not matching size=" + + fieldEntry.size + + " * dim=" + + dimension + + " * byteSize=" + + byteSize + + " = " + + numBytes); + } + if (fieldEntry.hasQuantizedVectors()) { + Lucene99ScalarQuantizedVectorsReader.validateFieldEntry( + info, fieldEntry.dimension, fieldEntry.size, fieldEntry.quantizedVectorDataLength); + } + } + + private VectorSimilarityFunction readSimilarityFunction(DataInput input) throws IOException { + int similarityFunctionId = input.readInt(); + if (similarityFunctionId < 0 + || similarityFunctionId >= VectorSimilarityFunction.values().length) { + throw new CorruptIndexException( + "Invalid similarity function id: " + similarityFunctionId, input); + } + return VectorSimilarityFunction.values()[similarityFunctionId]; + } + + private VectorEncoding readVectorEncoding(DataInput input) throws IOException { + int encodingId = input.readInt(); + if (encodingId < 0 || encodingId >= VectorEncoding.values().length) { + throw new CorruptIndexException("Invalid vector encoding id: " + encodingId, input); + } + return VectorEncoding.values()[encodingId]; + } + + private FieldEntry readField(IndexInput input) throws IOException { + VectorEncoding vectorEncoding = readVectorEncoding(input); + VectorSimilarityFunction similarityFunction = readSimilarityFunction(input); + return new FieldEntry(input, vectorEncoding, similarityFunction); + } + + @Override + public long ramBytesUsed() { + return Lucene99HnswVectorsReader.SHALLOW_SIZE + + RamUsageEstimator.sizeOfMap( + fields, RamUsageEstimator.shallowSizeOfInstance(FieldEntry.class)); + } + + @Override + public void checkIntegrity() throws IOException { + CodecUtil.checksumEntireFile(vectorData); + CodecUtil.checksumEntireFile(vectorIndex); + if (quantizedVectorsReader != null) { + quantizedVectorsReader.checkIntegrity(); + } + } + + @Override + public FloatVectorValues getFloatVectorValues(String field) throws IOException { + FieldEntry fieldEntry = fields.get(field); + if (fieldEntry.vectorEncoding != VectorEncoding.FLOAT32) { + throw new IllegalArgumentException( + "field=\"" + + field + + "\" is encoded as: " + + fieldEntry.vectorEncoding + + " expected: " + + VectorEncoding.FLOAT32); + } + return OffHeapFloatVectorValues.load(fieldEntry, vectorData); + } + + @Override + public ByteVectorValues getByteVectorValues(String field) throws IOException { + FieldEntry fieldEntry = fields.get(field); + if (fieldEntry.vectorEncoding != VectorEncoding.BYTE) { + throw new IllegalArgumentException( + "field=\"" + + field + + "\" is encoded as: " + + fieldEntry.vectorEncoding + + " expected: " + + VectorEncoding.FLOAT32); + } + return OffHeapByteVectorValues.load(fieldEntry, vectorData); + } + + @Override + public void search(String field, float[] target, KnnCollector knnCollector, Bits acceptDocs) + throws IOException { + FieldEntry fieldEntry = fields.get(field); + + if (fieldEntry.size() == 0 + || knnCollector.k() == 0 + || fieldEntry.vectorEncoding != VectorEncoding.FLOAT32) { + return; + } + if (quantizedVectorsReader != null) { + OffHeapQuantizedByteVectorValues vectorValues = getQuantizedVectorValues(field); + if (vectorValues == null) { + return; + } + RandomVectorScorer scorer = + new ScalarQuantizedRandomVectorScorer( + fieldEntry.similarityFunction, fieldEntry.scalarQuantizer, vectorValues, target); + HnswGraphSearcher.search( + scorer, + new OrdinalTranslatedKnnCollector(knnCollector, vectorValues::ordToDoc), + getGraph(fieldEntry), + vectorValues.getAcceptOrds(acceptDocs)); + } else { + OffHeapFloatVectorValues vectorValues = OffHeapFloatVectorValues.load(fieldEntry, vectorData); + RandomVectorScorer scorer = + RandomVectorScorer.createFloats(vectorValues, fieldEntry.similarityFunction, target); + HnswGraphSearcher.search( + scorer, + new OrdinalTranslatedKnnCollector(knnCollector, vectorValues::ordToDoc), + getGraph(fieldEntry), + vectorValues.getAcceptOrds(acceptDocs)); + } + } + + @Override + public void search(String field, byte[] target, KnnCollector knnCollector, Bits acceptDocs) + throws IOException { + FieldEntry fieldEntry = fields.get(field); + + if (fieldEntry.size() == 0 + || knnCollector.k() == 0 + || fieldEntry.vectorEncoding != VectorEncoding.BYTE) { + return; + } + + OffHeapByteVectorValues vectorValues = OffHeapByteVectorValues.load(fieldEntry, vectorData); + RandomVectorScorer scorer = + RandomVectorScorer.createBytes(vectorValues, fieldEntry.similarityFunction, target); + HnswGraphSearcher.search( + scorer, + new OrdinalTranslatedKnnCollector(knnCollector, vectorValues::ordToDoc), + getGraph(fieldEntry), + vectorValues.getAcceptOrds(acceptDocs)); + } + + @Override + public HnswGraph getGraph(String field) throws IOException { + FieldInfo info = fieldInfos.fieldInfo(field); + if (info == null) { + throw new IllegalArgumentException("No such field '" + field + "'"); + } + FieldEntry entry = fields.get(field); + if (entry != null && entry.vectorIndexLength > 0) { + return getGraph(entry); + } else { + return HnswGraph.EMPTY; + } + } + + private HnswGraph getGraph(FieldEntry entry) throws IOException { + return new OffHeapHnswGraph(entry, vectorIndex); + } + + @Override + public void close() throws IOException { + IOUtils.close(vectorData, vectorIndex, quantizedVectorData); + } + + @Override + public OffHeapQuantizedByteVectorValues getQuantizedVectorValues(String field) + throws IOException { + FieldEntry fieldEntry = fields.get(field); + if (fieldEntry == null || fieldEntry.hasQuantizedVectors() == false) { + return null; + } + assert quantizedVectorsReader != null && fieldEntry.quantizedOrdToDoc != null; + return quantizedVectorsReader.getQuantizedVectorValues( + fieldEntry.quantizedOrdToDoc, + fieldEntry.dimension, + fieldEntry.size, + fieldEntry.quantizedVectorDataOffset, + fieldEntry.quantizedVectorDataLength); + } + + @Override + public ScalarQuantizer getQuantizationState(String fieldName) { + FieldEntry field = fields.get(fieldName); + if (field == null || field.hasQuantizedVectors() == false) { + return null; + } + return field.scalarQuantizer; + } + + static class FieldEntry implements Accountable { + private static final long SHALLOW_SIZE = + RamUsageEstimator.shallowSizeOfInstance(FieldEntry.class); + final VectorSimilarityFunction similarityFunction; + final VectorEncoding vectorEncoding; + final long vectorDataOffset; + final long vectorDataLength; + final long vectorIndexOffset; + final long vectorIndexLength; + final int M; + final int numLevels; + final int dimension; + final int size; + final int[][] nodesByLevel; + // for each level the start offsets in vectorIndex file from where to read neighbours + final DirectMonotonicReader.Meta offsetsMeta; + final long offsetsOffset; + final int offsetsBlockShift; + final long offsetsLength; + final OrdToDocDISReaderConfiguration ordToDoc; + + final float configuredQuantile, lowerQuantile, upperQuantile; + final long quantizedVectorDataOffset, quantizedVectorDataLength; + final ScalarQuantizer scalarQuantizer; + final boolean isQuantized; + final OrdToDocDISReaderConfiguration quantizedOrdToDoc; + + FieldEntry( + IndexInput input, + VectorEncoding vectorEncoding, + VectorSimilarityFunction similarityFunction) + throws IOException { + this.similarityFunction = similarityFunction; + this.vectorEncoding = vectorEncoding; + this.isQuantized = input.readByte() == 1; + // Has int8 quantization + if (isQuantized) { + configuredQuantile = Float.intBitsToFloat(input.readInt()); + lowerQuantile = Float.intBitsToFloat(input.readInt()); + upperQuantile = Float.intBitsToFloat(input.readInt()); + quantizedVectorDataOffset = input.readVLong(); + quantizedVectorDataLength = input.readVLong(); + scalarQuantizer = new ScalarQuantizer(lowerQuantile, upperQuantile); + } else { + configuredQuantile = -1; + lowerQuantile = -1; + upperQuantile = -1; + quantizedVectorDataOffset = -1; + quantizedVectorDataLength = -1; + scalarQuantizer = null; + } + vectorDataOffset = input.readVLong(); + vectorDataLength = input.readVLong(); + vectorIndexOffset = input.readVLong(); + vectorIndexLength = input.readVLong(); + dimension = input.readVInt(); + size = input.readInt(); + if (isQuantized) { + quantizedOrdToDoc = OrdToDocDISReaderConfiguration.fromStoredMeta(input, size); + } else { + quantizedOrdToDoc = null; + } + ordToDoc = OrdToDocDISReaderConfiguration.fromStoredMeta(input, size); + + /*docsWithFieldOffset = input.readLong(); + docsWithFieldLength = input.readLong(); + jumpTableEntryCount = input.readShort(); + denseRankPower = input.readByte(); + + // dense or empty + if (docsWithFieldOffset == -1 || docsWithFieldOffset == -2) { + addressesOffset = 0; + blockShift = 0; + meta = null; + addressesLength = 0; + } else { + // sparse + if (isQuantized) { + + } + addressesOffset = input.readLong(); + blockShift = input.readVInt(); + meta = DirectMonotonicReader.loadMeta(input, size, blockShift); + addressesLength = input.readLong(); + }*/ + + // read nodes by level + M = input.readVInt(); + numLevels = input.readVInt(); + nodesByLevel = new int[numLevels][]; + long numberOfOffsets = 0; + for (int level = 0; level < numLevels; level++) { + if (level > 0) { + int numNodesOnLevel = input.readVInt(); + numberOfOffsets += numNodesOnLevel; + nodesByLevel[level] = new int[numNodesOnLevel]; + nodesByLevel[level][0] = input.readVInt(); + for (int i = 1; i < numNodesOnLevel; i++) { + nodesByLevel[level][i] = nodesByLevel[level][i - 1] + input.readVInt(); + } + } else { + numberOfOffsets += size; + } + } + if (numberOfOffsets > 0) { + offsetsOffset = input.readLong(); + offsetsBlockShift = input.readVInt(); + offsetsMeta = DirectMonotonicReader.loadMeta(input, numberOfOffsets, offsetsBlockShift); + offsetsLength = input.readLong(); + } else { + offsetsOffset = 0; + offsetsBlockShift = 0; + offsetsMeta = null; + offsetsLength = 0; + } + } + + int size() { + return size; + } + + boolean hasQuantizedVectors() { + return isQuantized; + } + + @Override + public long ramBytesUsed() { + return SHALLOW_SIZE + + Arrays.stream(nodesByLevel).mapToLong(nodes -> RamUsageEstimator.sizeOf(nodes)).sum() + + RamUsageEstimator.sizeOf(ordToDoc) + + (quantizedOrdToDoc == null ? 0 : RamUsageEstimator.sizeOf(quantizedOrdToDoc)) + + RamUsageEstimator.sizeOf(offsetsMeta); + } + } + + static final class OrdToDocDISReaderConfiguration implements Accountable { Review Comment: this is all the information needed to iterate sparse vector docs, translate ords, etc. refactoring to allow quantization to use the same code. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org