diff --git a/lucene/backward-codecs/src/java/module-info.java b/lucene/backward-codecs/src/java/module-info.java index 5dfe4efb4ab..ae4bd84fa62 100644 --- a/lucene/backward-codecs/src/java/module-info.java +++ b/lucene/backward-codecs/src/java/module-info.java @@ -29,6 +29,7 @@ module org.apache.lucene.backward_codecs { exports org.apache.lucene.backward_codecs.lucene84; exports org.apache.lucene.backward_codecs.lucene86; exports org.apache.lucene.backward_codecs.lucene87; + exports org.apache.lucene.backward_codecs.lucene90; exports org.apache.lucene.backward_codecs.packed; exports org.apache.lucene.backward_codecs.store; @@ -37,9 +38,12 @@ module org.apache.lucene.backward_codecs { provides org.apache.lucene.codecs.PostingsFormat with org.apache.lucene.backward_codecs.lucene50.Lucene50PostingsFormat, org.apache.lucene.backward_codecs.lucene84.Lucene84PostingsFormat; + provides org.apache.lucene.codecs.KnnVectorsFormat with + org.apache.lucene.backward_codecs.lucene90.Lucene90HnswVectorsFormat; provides org.apache.lucene.codecs.Codec with org.apache.lucene.backward_codecs.lucene80.Lucene80Codec, org.apache.lucene.backward_codecs.lucene84.Lucene84Codec, org.apache.lucene.backward_codecs.lucene86.Lucene86Codec, - org.apache.lucene.backward_codecs.lucene87.Lucene87Codec; + org.apache.lucene.backward_codecs.lucene87.Lucene87Codec, + org.apache.lucene.backward_codecs.lucene90.Lucene90Codec; } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90Codec.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90Codec.java similarity index 90% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90Codec.java rename to lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90Codec.java index 97b8c1b7f34..40ba06606bd 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90Codec.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90Codec.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene90; +package org.apache.lucene.backward_codecs.lucene90; import java.util.Objects; import org.apache.lucene.codecs.Codec; @@ -30,6 +30,16 @@ 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.Lucene90FieldInfosFormat; +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.perfield.PerFieldDocValuesFormat; import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat; import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat; diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswGraph.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswGraph.java new file mode 100644 index 00000000000..d8d28eca16b --- /dev/null +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswGraph.java @@ -0,0 +1,216 @@ +/* + * 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.backward_codecs.lucene90; + +import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.SplittableRandom; +import org.apache.lucene.index.KnnGraphValues; +import org.apache.lucene.index.RandomAccessVectorValues; +import org.apache.lucene.index.VectorSimilarityFunction; +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.SparseFixedBitSet; +import org.apache.lucene.util.hnsw.BoundsChecker; +import org.apache.lucene.util.hnsw.NeighborArray; +import org.apache.lucene.util.hnsw.NeighborQueue; + +/** + * Navigable Small-world graph. Provides efficient approximate nearest neighbor search for high + * dimensional vectors. See Approximate nearest + * neighbor algorithm based on navigable small world graphs [2014] and this paper [2018] for details. + * + *

The nomenclature is a bit different here from what's used in those papers: + * + *

Hyperparameters

+ * + * + * + *

Note: The graph may be searched by multiple threads concurrently, but updates are not + * thread-safe. Also note: there is no notion of deletions. Document searching built on top of this + * must do its own deletion-filtering. + * + *

Graph building logic is preserved here only for tests. + */ +public final class Lucene90HnswGraph extends KnnGraphValues { + + private final int maxConn; + + // Each entry lists the top maxConn neighbors of a node. The nodes correspond to vectors added to + // HnswBuilder, and the + // node values are the ordinals of those vectors. + private final List graph; + + // KnnGraphValues iterator members + private int upto; + private NeighborArray cur; + + Lucene90HnswGraph(int maxConn) { + graph = new ArrayList<>(); + // Typically with diversity criteria we see nodes not fully occupied; average fanout seems to be + // about 1/2 maxConn. There is some indexing time penalty for under-allocating, but saves RAM + graph.add(new NeighborArray(Math.max(32, maxConn / 4))); + this.maxConn = maxConn; + } + + /** + * Searches for the nearest neighbors of a query vector. + * + * @param query search query vector + * @param topK the number of nodes to be returned + * @param numSeed the size of the queue maintained while searching, and controls the number of + * random entry points to sample + * @param vectors vector values + * @param graphValues the graph values. May represent the entire graph, or a level in a + * hierarchical graph. + * @param acceptOrds {@link Bits} that represents the allowed document ordinals to match, or + * {@code null} if they are all allowed to match. + * @param random a source of randomness, used for generating entry points to the graph + * @return a priority queue holding the closest neighbors found + */ + public static NeighborQueue search( + float[] query, + int topK, + int numSeed, + RandomAccessVectorValues vectors, + VectorSimilarityFunction similarityFunction, + KnnGraphValues graphValues, + Bits acceptOrds, + SplittableRandom random) + throws IOException { + int size = graphValues.size(); + + // MIN heap, holding the top results + NeighborQueue results = new NeighborQueue(numSeed, similarityFunction.reversed); + // MAX heap, from which to pull the candidate nodes + NeighborQueue candidates = new NeighborQueue(numSeed, !similarityFunction.reversed); + + // set of ordinals that have been visited by search on this layer, used to avoid backtracking + SparseFixedBitSet visited = new SparseFixedBitSet(size); + // get initial candidates at random + int boundedNumSeed = Math.min(numSeed, 2 * size); + for (int i = 0; i < boundedNumSeed; i++) { + int entryPoint = random.nextInt(size); + if (visited.getAndSet(entryPoint) == false) { + // explore the topK starting points of some random numSeed probes + float score = similarityFunction.compare(query, vectors.vectorValue(entryPoint)); + candidates.add(entryPoint, score); + if (acceptOrds == null || acceptOrds.get(entryPoint)) { + results.add(entryPoint, score); + } + } + } + + // Set the bound to the worst current result and below reject any newly-generated candidates + // failing + // to exceed this bound + BoundsChecker bound = BoundsChecker.create(similarityFunction.reversed); + bound.set(results.topScore()); + while (candidates.size() > 0) { + // get the best candidate (closest or best scoring) + float topCandidateScore = candidates.topScore(); + if (results.size() >= topK) { + if (bound.check(topCandidateScore)) { + break; + } + } + int topCandidateNode = candidates.pop(); + graphValues.seek(0, topCandidateNode); + int friendOrd; + while ((friendOrd = graphValues.nextNeighbor()) != NO_MORE_DOCS) { + assert friendOrd < size : "friendOrd=" + friendOrd + "; size=" + size; + if (visited.getAndSet(friendOrd)) { + continue; + } + + float score = similarityFunction.compare(query, vectors.vectorValue(friendOrd)); + if (results.size() < numSeed || bound.check(score) == false) { + candidates.add(friendOrd, score); + if (acceptOrds == null || acceptOrds.get(friendOrd)) { + results.insertWithOverflow(friendOrd, score); + bound.set(results.topScore()); + } + } + } + } + while (results.size() > topK) { + results.pop(); + } + results.setVisitedCount(visited.approximateCardinality()); + return results; + } + + /** + * Returns the {@link NeighborQueue} connected to the given node. + * + * @param node the node whose neighbors are returned + */ + public NeighborArray getNeighbors(int node) { + return graph.get(node); + } + + @Override + public int size() { + return graph.size(); + } + + int addNode() { + graph.add(new NeighborArray(maxConn + 1)); + return graph.size() - 1; + } + + @Override + public void seek(int level, int targetNode) { + cur = getNeighbors(targetNode); + upto = -1; + } + + @Override + public int nextNeighbor() { + if (++upto < cur.size()) { + return cur.node()[upto]; + } + return NO_MORE_DOCS; + } + + @Override + public int numLevels() { + throw new UnsupportedOperationException(); + } + + @Override + public int entryNode() { + throw new UnsupportedOperationException(); + } + + @Override + public NodesIterator getNodesOnLevel(int level) { + throw new UnsupportedOperationException(); + } +} diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswGraphBuilder.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswGraphBuilder.java new file mode 100644 index 00000000000..7fda65e23c9 --- /dev/null +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswGraphBuilder.java @@ -0,0 +1,276 @@ +/* + * 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.backward_codecs.lucene90; + +import java.io.IOException; +import java.util.Locale; +import java.util.Objects; +import java.util.SplittableRandom; +import org.apache.lucene.index.RandomAccessVectorValues; +import org.apache.lucene.index.RandomAccessVectorValuesProducer; +import org.apache.lucene.index.VectorSimilarityFunction; +import org.apache.lucene.util.InfoStream; +import org.apache.lucene.util.hnsw.BoundsChecker; +import org.apache.lucene.util.hnsw.NeighborArray; +import org.apache.lucene.util.hnsw.NeighborQueue; + +/** + * Builder for HNSW graph. See {@link Lucene90HnswGraph} for a gloss on the algorithm and the + * meaning of the hyperparameters. + * + *

This class is preserved here only for tests. + */ +public final class Lucene90HnswGraphBuilder { + + /** Default random seed for level generation * */ + private static final long DEFAULT_RAND_SEED = System.currentTimeMillis(); + /** A name for the HNSW component for the info-stream * */ + public static final String HNSW_COMPONENT = "HNSW"; + + /** Random seed for level generation; public to expose for testing * */ + public static long randSeed = DEFAULT_RAND_SEED; + + private final int maxConn; + private final int beamWidth; + private final NeighborArray scratch; + + private final VectorSimilarityFunction similarityFunction; + private final RandomAccessVectorValues vectorValues; + private final SplittableRandom random; + private final BoundsChecker bound; + final Lucene90HnswGraph hnsw; + + private InfoStream infoStream = InfoStream.getDefault(); + + // we need two sources of vectors in order to perform diversity check comparisons without + // colliding + private RandomAccessVectorValues buildVectors; + + /** + * Reads all the vectors from a VectorValues, builds a graph connecting them by their dense + * ordinals, using the given hyperparameter settings, and returns the resulting graph. + * + * @param vectors the vectors whose relations are represented by the graph - must provide a + * different view over those vectors than the one used to add via addGraphNode. + * @param maxConn the number of connections to make when adding a new graph node; roughly speaking + * the graph fanout. + * @param beamWidth the size of the beam search to use when finding nearest neighbors. + * @param seed the seed for a random number generator used during graph construction. Provide this + * to ensure repeatable construction. + */ + public Lucene90HnswGraphBuilder( + RandomAccessVectorValuesProducer vectors, + VectorSimilarityFunction similarityFunction, + int maxConn, + int beamWidth, + long seed) { + vectorValues = vectors.randomAccess(); + buildVectors = vectors.randomAccess(); + this.similarityFunction = Objects.requireNonNull(similarityFunction); + if (maxConn <= 0) { + throw new IllegalArgumentException("maxConn must be positive"); + } + if (beamWidth <= 0) { + throw new IllegalArgumentException("beamWidth must be positive"); + } + this.maxConn = maxConn; + this.beamWidth = beamWidth; + this.hnsw = new Lucene90HnswGraph(maxConn); + bound = BoundsChecker.create(similarityFunction.reversed); + random = new SplittableRandom(seed); + scratch = new NeighborArray(Math.max(beamWidth, maxConn + 1)); + } + + /** + * Reads all the vectors from two copies of a random access VectorValues. Providing two copies + * enables efficient retrieval without extra data copying, while avoiding collision of the + * returned values. + * + * @param vectors the vectors for which to build a nearest neighbors graph. Must be an independet + * accessor for the vectors + */ + public Lucene90HnswGraph build(RandomAccessVectorValues vectors) throws IOException { + if (vectors == vectorValues) { + throw new IllegalArgumentException( + "Vectors to build must be independent of the source of vectors provided to HnswGraphBuilder()"); + } + if (infoStream.isEnabled(HNSW_COMPONENT)) { + infoStream.message(HNSW_COMPONENT, "build graph from " + vectors.size() + " vectors"); + } + long start = System.nanoTime(), t = start; + // start at node 1! node 0 is added implicitly, in the constructor + for (int node = 1; node < vectors.size(); node++) { + addGraphNode(vectors.vectorValue(node)); + if (node % 10000 == 0) { + if (infoStream.isEnabled(HNSW_COMPONENT)) { + long now = System.nanoTime(); + infoStream.message( + HNSW_COMPONENT, + String.format( + Locale.ROOT, + "built %d in %d/%d ms", + node, + ((now - t) / 1_000_000), + ((now - start) / 1_000_000))); + t = now; + } + } + } + return hnsw; + } + + /** Set info-stream to output debugging information * */ + public void setInfoStream(InfoStream infoStream) { + this.infoStream = infoStream; + } + + /** Inserts a doc with vector value to the graph */ + void addGraphNode(float[] value) throws IOException { + // We pass 'null' for acceptOrds because there are no deletions while building the graph + NeighborQueue candidates = + Lucene90HnswGraph.search( + value, beamWidth, beamWidth, vectorValues, similarityFunction, hnsw, null, random); + + int node = hnsw.addNode(); + + /* connect neighbors to the new node, using a diversity heuristic that chooses successive + * nearest neighbors that are closer to the new node than they are to the previously-selected + * neighbors + */ + addDiverseNeighbors(node, candidates); + } + + /* TODO: we are not maintaining nodes in strict score order; the forward links + * are added in sorted order, but the reverse implicit ones are not. Diversity heuristic should + * work better if we keep the neighbor arrays sorted. Possibly we should switch back to a heap? + * But first we should just see if sorting makes a significant difference. + */ + private void addDiverseNeighbors(int node, NeighborQueue candidates) throws IOException { + /* For each of the beamWidth nearest candidates (going from best to worst), select it only if it + * is closer to target than it is to any of the already-selected neighbors (ie selected in this method, + * since the node is new and has no prior neighbors). + */ + NeighborArray neighbors = hnsw.getNeighbors(node); + assert neighbors.size() == 0; // new node + popToScratch(candidates); + selectDiverse(neighbors, scratch); + + // Link the selected nodes to the new node, and the new node to the selected nodes (again + // applying diversity heuristic) + int size = neighbors.size(); + for (int i = 0; i < size; i++) { + int nbr = neighbors.node()[i]; + NeighborArray nbrNbr = hnsw.getNeighbors(nbr); + nbrNbr.add(node, neighbors.score()[i]); + if (nbrNbr.size() > maxConn) { + diversityUpdate(nbrNbr); + } + } + } + + private void selectDiverse(NeighborArray neighbors, NeighborArray candidates) throws IOException { + // Select the best maxConn neighbors of the new node, applying the diversity heuristic + for (int i = candidates.size() - 1; neighbors.size() < maxConn && i >= 0; i--) { + // compare each neighbor (in distance order) against the closer neighbors selected so far, + // only adding it if it is closer to the target than to any of the other selected neighbors + int cNode = candidates.node()[i]; + float cScore = candidates.score()[i]; + assert cNode < hnsw.size(); + if (diversityCheck(vectorValues.vectorValue(cNode), cScore, neighbors, buildVectors)) { + neighbors.add(cNode, cScore); + } + } + } + + private void popToScratch(NeighborQueue candidates) { + scratch.clear(); + int candidateCount = candidates.size(); + // extract all the Neighbors from the queue into an array; these will now be + // sorted from worst to best + for (int i = 0; i < candidateCount; i++) { + float score = candidates.topScore(); + scratch.add(candidates.pop(), score); + } + } + + /** + * @param candidate the vector of a new candidate neighbor of a node n + * @param score the score of the new candidate and node n, to be compared with scores of the + * candidate and n's neighbors + * @param neighbors the neighbors selected so far + * @param vectorValues source of values used for making comparisons between candidate and existing + * neighbors + * @return whether the candidate is diverse given the existing neighbors + */ + private boolean diversityCheck( + float[] candidate, + float score, + NeighborArray neighbors, + RandomAccessVectorValues vectorValues) + throws IOException { + bound.set(score); + for (int i = 0; i < neighbors.size(); i++) { + float diversityCheck = + similarityFunction.compare(candidate, vectorValues.vectorValue(neighbors.node()[i])); + if (bound.check(diversityCheck) == false) { + return false; + } + } + return true; + } + + private void diversityUpdate(NeighborArray neighbors) throws IOException { + assert neighbors.size() == maxConn + 1; + int replacePoint = findNonDiverse(neighbors); + if (replacePoint == -1) { + // none found; check score against worst existing neighbor + bound.set(neighbors.score()[0]); + if (bound.check(neighbors.score()[maxConn])) { + // drop the new neighbor; it is not competitive and there were no diversity failures + neighbors.removeLast(); + return; + } else { + replacePoint = 0; + } + } + neighbors.node()[replacePoint] = neighbors.node()[maxConn]; + neighbors.score()[replacePoint] = neighbors.score()[maxConn]; + neighbors.removeLast(); + } + + // scan neighbors looking for diversity violations + private int findNonDiverse(NeighborArray neighbors) throws IOException { + for (int i = neighbors.size() - 1; i >= 0; i--) { + // check each neighbor against its better-scoring neighbors. If it fails diversity check with + // them, drop it + int nbrNode = neighbors.node()[i]; + bound.set(neighbors.score()[i]); + float[] nbrVector = vectorValues.vectorValue(nbrNode); + for (int j = maxConn; j > i; j--) { + float diversityCheck = + similarityFunction.compare(nbrVector, buildVectors.vectorValue(neighbors.node()[j])); + if (bound.check(diversityCheck) == false) { + // node j is too similar to node i given its score relative to the base node + // replace it with the new node, which is at [maxConn] + return i; + } + } + } + return -1; + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90HnswVectorsFormat.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswVectorsFormat.java similarity index 88% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90HnswVectorsFormat.java rename to lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswVectorsFormat.java index 90875d9e3f5..c622d9cef6f 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90HnswVectorsFormat.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswVectorsFormat.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.lucene.codecs.lucene90; +package org.apache.lucene.backward_codecs.lucene90; import java.io.IOException; import org.apache.lucene.codecs.KnnVectorsFormat; @@ -65,7 +65,7 @@ import org.apache.lucene.util.hnsw.HnswGraph; * * @lucene.experimental */ -public final class Lucene90HnswVectorsFormat extends KnnVectorsFormat { +public class Lucene90HnswVectorsFormat extends KnnVectorsFormat { static final String META_CODEC_NAME = "Lucene90HnswVectorsFormatMeta"; static final String VECTOR_DATA_CODEC_NAME = "Lucene90HnswVectorsFormatData"; @@ -77,26 +77,33 @@ public final class Lucene90HnswVectorsFormat extends KnnVectorsFormat { static final int VERSION_START = 0; static final int VERSION_CURRENT = VERSION_START; + /** Default number of maximum connections per node */ public static final int DEFAULT_MAX_CONN = 16; + /** + * Default number of the size of the queue maintained while searching and the number of random + * entry points to sample during a graph construction. + */ public static final int DEFAULT_BEAM_WIDTH = 100; /** * Controls how many of the nearest neighbor candidates are connected to the new node. Defaults to * {@link Lucene90HnswVectorsFormat#DEFAULT_MAX_CONN}. See {@link HnswGraph} for more details. */ - private final int maxConn; + final int maxConn; /** * The number of candidate neighbors to track while searching the graph for each newly inserted * node. Defaults to to {@link Lucene90HnswVectorsFormat#DEFAULT_BEAM_WIDTH}. See {@link * HnswGraph} for details. */ - private final int beamWidth; + final int beamWidth; + /** A constructor for vectors format with default parameters */ public Lucene90HnswVectorsFormat() { this(DEFAULT_MAX_CONN, DEFAULT_BEAM_WIDTH); } + /** A constructor for vectors format */ public Lucene90HnswVectorsFormat(int maxConn, int beamWidth) { super("Lucene90HnswVectorsFormat"); this.maxConn = maxConn; @@ -105,7 +112,7 @@ public final class Lucene90HnswVectorsFormat extends KnnVectorsFormat { @Override public KnnVectorsWriter fieldsWriter(SegmentWriteState state) throws IOException { - return new Lucene90HnswVectorsWriter(state, maxConn, beamWidth); + throw new UnsupportedOperationException("Old codecs may only be used for reading"); } @Override diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90HnswVectorsReader.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswVectorsReader.java similarity index 96% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90HnswVectorsReader.java rename to lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswVectorsReader.java index bb62ab9dd95..7669a8d38ff 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90HnswVectorsReader.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswVectorsReader.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.lucene.codecs.lucene90; +package org.apache.lucene.backward_codecs.lucene90; import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; @@ -47,7 +47,6 @@ import org.apache.lucene.util.Bits; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.RamUsageEstimator; -import org.apache.lucene.util.hnsw.HnswGraph; import org.apache.lucene.util.hnsw.NeighborQueue; /** @@ -244,7 +243,7 @@ public final class Lucene90HnswVectorsReader extends KnnVectorsReader { // use a seed that is fixed for the index so we get reproducible results for the same query final SplittableRandom random = new SplittableRandom(checksumSeed); NeighborQueue results = - HnswGraph.search( + Lucene90HnswGraph.search( target, k, k, @@ -291,6 +290,7 @@ public final class Lucene90HnswVectorsReader extends KnnVectorsReader { }; } + /** Get knn graph values; used for testing */ public KnnGraphValues getGraphValues(String field) throws IOException { FieldInfo info = fieldInfos.fieldInfo(field); if (info == null) { @@ -480,7 +480,7 @@ public final class Lucene90HnswVectorsReader extends KnnVectorsReader { } @Override - public void seek(int targetOrd) throws IOException { + public void seek(int level, int targetOrd) throws IOException { // unsafe; no bounds checking dataIn.seek(entry.ordOffsets[targetOrd]); arcCount = dataIn.readInt(); @@ -502,5 +502,20 @@ public final class Lucene90HnswVectorsReader extends KnnVectorsReader { arc += dataIn.readVInt(); return arc; } + + @Override + public int numLevels() { + throw new UnsupportedOperationException(); + } + + @Override + public int entryNode() { + throw new UnsupportedOperationException(); + } + + @Override + public NodesIterator getNodesOnLevel(int level) { + throw new UnsupportedOperationException(); + } } } diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/package-info.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/package-info.java new file mode 100644 index 00000000000..5ad2dbcc82c --- /dev/null +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/package-info.java @@ -0,0 +1,19 @@ +/* + * 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. + */ + +/** Lucene 9.0 file format. */ +package org.apache.lucene.backward_codecs.lucene90; diff --git a/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec index c9a6322f0b1..72e05ab3198 100644 --- a/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec +++ b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec @@ -17,3 +17,4 @@ org.apache.lucene.backward_codecs.lucene80.Lucene80Codec org.apache.lucene.backward_codecs.lucene84.Lucene84Codec org.apache.lucene.backward_codecs.lucene86.Lucene86Codec org.apache.lucene.backward_codecs.lucene87.Lucene87Codec +org.apache.lucene.backward_codecs.lucene90.Lucene90Codec diff --git a/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.KnnVectorsFormat b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.KnnVectorsFormat new file mode 100644 index 00000000000..17d89f3be7f --- /dev/null +++ b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.KnnVectorsFormat @@ -0,0 +1,16 @@ +# 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. + +org.apache.lucene.backward_codecs.lucene90.Lucene90HnswVectorsFormat diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90HnswVectorsWriter.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswVectorsWriter.java similarity index 96% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90HnswVectorsWriter.java rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswVectorsWriter.java index 0fbd1be23dd..d76e5efb635 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90HnswVectorsWriter.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswVectorsWriter.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.lucene.codecs.lucene90; +package org.apache.lucene.backward_codecs.lucene90; import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; @@ -35,8 +35,6 @@ import org.apache.lucene.store.IndexOutput; import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.IOUtils; -import org.apache.lucene.util.hnsw.HnswGraph; -import org.apache.lucene.util.hnsw.HnswGraphBuilder; import org.apache.lucene.util.hnsw.NeighborArray; /** @@ -235,11 +233,15 @@ public final class Lucene90HnswVectorsWriter extends KnnVectorsWriter { int maxConn, int beamWidth) throws IOException { - HnswGraphBuilder hnswGraphBuilder = - new HnswGraphBuilder( - vectorValues, similarityFunction, maxConn, beamWidth, HnswGraphBuilder.randSeed); + Lucene90HnswGraphBuilder hnswGraphBuilder = + new Lucene90HnswGraphBuilder( + vectorValues, + similarityFunction, + maxConn, + beamWidth, + Lucene90HnswGraphBuilder.randSeed); hnswGraphBuilder.setInfoStream(segmentWriteState.infoStream); - HnswGraph graph = hnswGraphBuilder.build(vectorValues.randomAccess()); + Lucene90HnswGraph graph = hnswGraphBuilder.build(vectorValues.randomAccess()); for (int ord = 0; ord < offsets.length; ord++) { // write graph diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene90/Lucene90RWHnswVectorsFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene90/Lucene90RWHnswVectorsFormat.java new file mode 100644 index 00000000000..dd720c18319 --- /dev/null +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene90/Lucene90RWHnswVectorsFormat.java @@ -0,0 +1,43 @@ +/* + * 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.backward_codecs.lucene90; + +import java.io.IOException; +import org.apache.lucene.codecs.KnnVectorsWriter; +import org.apache.lucene.index.SegmentWriteState; + +public class Lucene90RWHnswVectorsFormat extends Lucene90HnswVectorsFormat { + + public Lucene90RWHnswVectorsFormat(int maxConn, int beamWidth) { + super(maxConn, beamWidth); + } + + @Override + public KnnVectorsWriter fieldsWriter(SegmentWriteState state) throws IOException { + return new Lucene90HnswVectorsWriter(state, maxConn, beamWidth); + } + + @Override + public String toString() { + return "Lucene90RWHnswVectorsFormat(name = Lucene90RWHnswVectorsFormat, maxConn = " + + maxConn + + ", beamWidth=" + + beamWidth + + ")"; + } +} diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90HnswVectorsFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene90/TestLucene90HnswVectorsFormat.java similarity index 82% rename from lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90HnswVectorsFormat.java rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene90/TestLucene90HnswVectorsFormat.java index 7baca8ca540..8fe6dc24624 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90HnswVectorsFormat.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene90/TestLucene90HnswVectorsFormat.java @@ -14,11 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene90; +package org.apache.lucene.backward_codecs.lucene90; import static com.carrotsearch.randomizedtesting.RandomizedTest.randomIntBetween; -import static org.apache.lucene.codecs.lucene90.Lucene90HnswVectorsFormat.DEFAULT_BEAM_WIDTH; -import static org.apache.lucene.codecs.lucene90.Lucene90HnswVectorsFormat.DEFAULT_MAX_CONN; +import static org.apache.lucene.backward_codecs.lucene90.Lucene90HnswVectorsFormat.DEFAULT_BEAM_WIDTH; +import static org.apache.lucene.backward_codecs.lucene90.Lucene90HnswVectorsFormat.DEFAULT_MAX_CONN; import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.KnnVectorsFormat; @@ -38,11 +38,11 @@ public class TestLucene90HnswVectorsFormat extends BaseKnnVectorsFormatTestCase new Lucene90Codec() { @Override public KnnVectorsFormat getKnnVectorsFormatForField(String field) { - return new Lucene90HnswVectorsFormat(maxConn, beamWidth); + return new Lucene90RWHnswVectorsFormat(maxConn, beamWidth); } }; String expectedString = - "Lucene90HnswVectorsFormat(name = Lucene90HnswVectorsFormat, maxConn = " + "Lucene90RWHnswVectorsFormat(name = Lucene90RWHnswVectorsFormat, maxConn = " + maxConn + ", beamWidth=" + beamWidth diff --git a/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java b/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java index d7ae013f381..1f76e829676 100644 --- a/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java +++ b/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java @@ -27,7 +27,7 @@ import org.apache.lucene.benchmark.byTask.PerfRunData; import org.apache.lucene.benchmark.byTask.utils.Config; import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.PostingsFormat; -import org.apache.lucene.codecs.lucene90.Lucene90Codec; +import org.apache.lucene.codecs.lucene91.Lucene91Codec; import org.apache.lucene.index.ConcurrentMergeScheduler; import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.IndexDeletionPolicy; @@ -152,7 +152,7 @@ public class CreateIndexTask extends PerfTask { try { final PostingsFormat postingsFormatChosen = PostingsFormat.forName(postingsFormat); iwConf.setCodec( - new Lucene90Codec() { + new Lucene91Codec() { @Override public PostingsFormat getPostingsFormatForField(String field) { return postingsFormatChosen; diff --git a/lucene/core/src/java/module-info.java b/lucene/core/src/java/module-info.java index 02fc3ff0821..f4139e7eaa3 100644 --- a/lucene/core/src/java/module-info.java +++ b/lucene/core/src/java/module-info.java @@ -27,6 +27,7 @@ module org.apache.lucene.core { exports org.apache.lucene.analysis.tokenattributes; exports org.apache.lucene.codecs; exports org.apache.lucene.codecs.compressing; + exports org.apache.lucene.codecs.lucene91; exports org.apache.lucene.codecs.lucene90; exports org.apache.lucene.codecs.lucene90.blocktree; exports org.apache.lucene.codecs.lucene90.compressing; @@ -59,11 +60,11 @@ module org.apache.lucene.core { provides org.apache.lucene.analysis.TokenizerFactory with org.apache.lucene.analysis.standard.StandardTokenizerFactory; provides org.apache.lucene.codecs.Codec with - org.apache.lucene.codecs.lucene90.Lucene90Codec; + org.apache.lucene.codecs.lucene91.Lucene91Codec; provides org.apache.lucene.codecs.DocValuesFormat with org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat; provides org.apache.lucene.codecs.KnnVectorsFormat with - org.apache.lucene.codecs.lucene90.Lucene90HnswVectorsFormat; + org.apache.lucene.codecs.lucene91.Lucene91HnswVectorsFormat; provides org.apache.lucene.codecs.PostingsFormat with org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat; provides org.apache.lucene.index.SortFieldProvider with diff --git a/lucene/core/src/java/org/apache/lucene/codecs/Codec.java b/lucene/core/src/java/org/apache/lucene/codecs/Codec.java index 69ecde2e26c..176cc57cfb5 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/Codec.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/Codec.java @@ -55,7 +55,7 @@ public abstract class Codec implements NamedSPILoader.NamedSPI { return LOADER; } - static Codec defaultCodec = LOADER.lookup("Lucene90"); + static Codec defaultCodec = LOADER.lookup("Lucene91"); } private final String name; diff --git a/lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsFormat.java index 4b58f2dc6c0..69b1a426535 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsFormat.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsFormat.java @@ -85,7 +85,7 @@ public abstract class KnnVectorsFormat implements NamedSPILoader.NamedSPI { @Override public KnnVectorsWriter fieldsWriter(SegmentWriteState state) { throw new UnsupportedOperationException( - "Attempt to write EMPTY VectorValues: maybe you forgot to use codec=Lucene90"); + "Attempt to write EMPTY VectorValues: maybe you forgot to use codec=Lucene91"); } @Override diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/package-info.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/package-info.java index d8c6828263f..e8fdeb41080 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/package-info.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/package-info.java @@ -15,405 +15,5 @@ * limitations under the License. */ -/** - * Lucene 9.0 file format. - * - *

Apache Lucene - Index File Formats

- * - *
- * - * - * - *
- * - *

Introduction

- * - *
- * - *

This document defines the index file formats used in this version of Lucene. If you are using - * a different version of Lucene, please consult the copy of docs/ that was distributed - * with the version you are using. - * - *

This document attempts to provide a high-level definition of the Apache Lucene file formats. - *

- * - *

Definitions

- * - *
- * - *

The fundamental concepts in Lucene are index, document, field and term. - * - *

An index contains a sequence of documents. - * - *

- * - *

The same sequence of bytes in two different fields is considered a different term. Thus terms - * are represented as a pair: the string naming the field, and the bytes within the field. - * - *

Inverted Indexing

- * - *

Lucene's index stores terms and statistics about those terms in order to make term-based - * search more efficient. Lucene's terms index falls into the family of indexes known as an - * inverted index. This is because it can list, for a term, the documents that contain it. - * This is the inverse of the natural relationship, in which documents list terms. - * - *

Types of Fields

- * - *

In Lucene, fields may be stored, in which case their text is stored in the index - * literally, in a non-inverted manner. Fields that are inverted are called indexed. A field - * may be both stored and indexed. - * - *

The text of a field may be tokenized into terms to be indexed, or the text of a field - * may be used literally as a term to be indexed. Most fields are tokenized, but sometimes it is - * useful for certain identifier fields to be indexed literally. - * - *

See the {@link org.apache.lucene.document.Field Field} java docs for more information on - * Fields. - * - *

Segments

- * - *

Lucene indexes may be composed of multiple sub-indexes, or segments. Each segment is a - * fully independent index, which could be searched separately. Indexes evolve by: - * - *

    - *
  1. Creating new segments for newly added documents. - *
  2. Merging existing segments. - *
- * - *

Searches may involve multiple segments and/or multiple indexes, each index potentially - * composed of a set of segments. - * - *

Document Numbers

- * - *

Internally, Lucene refers to documents by an integer document number. The first - * document added to an index is numbered zero, and each subsequent document added gets a number one - * greater than the previous. - * - *

Note that a document's number may change, so caution should be taken when storing these - * numbers outside of Lucene. In particular, numbers may change in the following situations: - * - *

- * - *
- * - *

Index Structure Overview

- * - *
- * - *

Each segment index maintains the following: - * - *

- * - *

Details on each of these are provided in their linked pages.

- * - *

File Naming

- * - *
- * - *

All files belonging to a segment have the same name with varying extensions. The extensions - * correspond to the different file formats described below. When using the Compound File format - * (default for small segments) these files (except for the Segment info file, the Lock file, and - * Deleted documents file) are collapsed into a single .cfs file (see below for details) - * - *

Typically, all segments in an index are stored in a single directory, although this is not - * required. - * - *

File names are never re-used. That is, when any file is saved to the Directory it is given a - * never before used filename. This is achieved using a simple generations approach. For example, - * the first segments file is segments_1, then segments_2, etc. The generation is a sequential long - * integer represented in alpha-numeric (base 36) form.

- * - *

Summary of File Extensions

- * - *
- * - *

The following table summarizes the names and extensions of the files in Lucene: - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - *
lucene filenames by extension
NameExtensionBrief Description
{@link org.apache.lucene.index.SegmentInfos Segments File}segments_NStores information about a commit point
Lock Filewrite.lockThe Write lock prevents multiple IndexWriters from writing to the same - * file.
{@link org.apache.lucene.codecs.lucene90.Lucene90SegmentInfoFormat Segment Info}.siStores metadata about a segment
{@link org.apache.lucene.codecs.lucene90.Lucene90CompoundFormat Compound File}.cfs, .cfeAn optional "virtual" file consisting of all the other index files for - * systems that frequently run out of file handles.
{@link org.apache.lucene.codecs.lucene90.Lucene90FieldInfosFormat Fields}.fnmStores information about the fields
{@link org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat Field Index}.fdxContains pointers to field data
{@link org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat Field Data}.fdtThe stored fields for documents
{@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat Term Dictionary}.timThe term dictionary, stores term info
{@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat Term Index}.tipThe index into the Term Dictionary
{@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat Frequencies}.docContains the list of docs which contain each term along with frequency
{@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat Positions}.posStores position information about where a term occurs in the index
{@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat Payloads}.payStores additional per-position metadata information such as character offsets and user payloads
{@link org.apache.lucene.codecs.lucene90.Lucene90NormsFormat Norms}.nvd, .nvmEncodes length and boost factors for docs and fields
{@link org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat Per-Document Values}.dvd, .dvmEncodes additional scoring factors or other per-document information.
{@link org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat Term Vector Index}.tvxStores offset into the document data file
{@link org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat Term Vector Data}.tvdContains term vector data.
{@link org.apache.lucene.codecs.lucene90.Lucene90LiveDocsFormat Live Documents}.livInfo about what documents are live
{@link org.apache.lucene.codecs.lucene90.Lucene90PointsFormat Point values}.dii, .dimHolds indexed points
{@link org.apache.lucene.codecs.lucene90.Lucene90HnswVectorsFormat Vector values}.vec, .vemHolds indexed vectors; .vec files contain the raw vector data, and - * .vem the vector metadata
- * - *

- * - *

Lock File

- * - * The write lock, which is stored in the index directory by default, is named "write.lock". If the - * lock directory is different from the index directory then the write lock will be named - * "XXXX-write.lock" where XXXX is a unique prefix derived from the full path to the index - * directory. When this file is present, a writer is currently modifying the index (adding or - * removing documents). This lock file ensures that only one writer is modifying the index at a - * time. - * - *

History

- * - *

Compatibility notes are provided in this document, describing how file formats have changed - * from prior versions: - * - *

- * - * - * - *

Limitations

- * - *
- * - *

Lucene uses a Java int to refer to document numbers, and the index file format - * uses an Int32 on-disk to store document numbers. This is a limitation of both the - * index file format and the current implementation. Eventually these should be replaced with either - * UInt64 values, or better yet, {@link org.apache.lucene.store.DataOutput#writeVInt - * VInt} values which have no limit.

- */ +/** Lucene 9.0 file format. */ package org.apache.lucene.codecs.lucene90; diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene91/Lucene91Codec.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene91/Lucene91Codec.java new file mode 100644 index 00000000000..2c289a888c1 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene91/Lucene91Codec.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.lucene91; + +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.Lucene90FieldInfosFormat; +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.perfield.PerFieldDocValuesFormat; +import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat; +import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat; + +/** + * Implements the Lucene 9.1 index format + * + *

If you want to reuse functionality of this codec in another codec, extend {@link FilterCodec}. + * + * @see org.apache.lucene.codecs.lucene91 package documentation for file format details. + * @lucene.experimental + */ +public class Lucene91Codec extends Codec { + + /** Configuration option for the codec. */ + public enum Mode { + /** Trade compression ratio for retrieval speed. */ + BEST_SPEED(Lucene90StoredFieldsFormat.Mode.BEST_SPEED), + /** Trade retrieval speed for compression ratio. */ + BEST_COMPRESSION(Lucene90StoredFieldsFormat.Mode.BEST_COMPRESSION); + + private final Lucene90StoredFieldsFormat.Mode storedMode; + + private Mode(Lucene90StoredFieldsFormat.Mode storedMode) { + this.storedMode = Objects.requireNonNull(storedMode); + } + } + + private final TermVectorsFormat vectorsFormat = new Lucene90TermVectorsFormat(); + private final FieldInfosFormat fieldInfosFormat = new Lucene90FieldInfosFormat(); + private final SegmentInfoFormat segmentInfosFormat = new Lucene90SegmentInfoFormat(); + private final LiveDocsFormat liveDocsFormat = new Lucene90LiveDocsFormat(); + private final CompoundFormat compoundFormat = new Lucene90CompoundFormat(); + private final NormsFormat normsFormat = new Lucene90NormsFormat(); + + private final PostingsFormat defaultPostingsFormat; + private final PostingsFormat postingsFormat = + new PerFieldPostingsFormat() { + @Override + public PostingsFormat getPostingsFormatForField(String field) { + return Lucene91Codec.this.getPostingsFormatForField(field); + } + }; + + private final DocValuesFormat defaultDVFormat; + private final DocValuesFormat docValuesFormat = + new PerFieldDocValuesFormat() { + @Override + public DocValuesFormat getDocValuesFormatForField(String field) { + return Lucene91Codec.this.getDocValuesFormatForField(field); + } + }; + + private final KnnVectorsFormat defaultKnnVectorsFormat; + private final KnnVectorsFormat knnVectorsFormat = + new PerFieldKnnVectorsFormat() { + @Override + public KnnVectorsFormat getKnnVectorsFormatForField(String field) { + return Lucene91Codec.this.getKnnVectorsFormatForField(field); + } + }; + + private final StoredFieldsFormat storedFieldsFormat; + + /** Instantiates a new codec. */ + public Lucene91Codec() { + this(Mode.BEST_SPEED); + } + + /** + * Instantiates a new codec, specifying the stored fields compression mode to use. + * + * @param mode stored fields compression mode to use for newly flushed/merged segments. + */ + public Lucene91Codec(Mode mode) { + super("Lucene91"); + this.storedFieldsFormat = + new Lucene90StoredFieldsFormat(Objects.requireNonNull(mode).storedMode); + this.defaultPostingsFormat = new Lucene90PostingsFormat(); + this.defaultDVFormat = new Lucene90DocValuesFormat(); + this.defaultKnnVectorsFormat = new Lucene91HnswVectorsFormat(); + } + + @Override + public final StoredFieldsFormat storedFieldsFormat() { + return storedFieldsFormat; + } + + @Override + public final TermVectorsFormat termVectorsFormat() { + return vectorsFormat; + } + + @Override + public final PostingsFormat postingsFormat() { + return postingsFormat; + } + + @Override + public final FieldInfosFormat fieldInfosFormat() { + return fieldInfosFormat; + } + + @Override + public final SegmentInfoFormat segmentInfoFormat() { + return segmentInfosFormat; + } + + @Override + public final LiveDocsFormat liveDocsFormat() { + return liveDocsFormat; + } + + @Override + public final CompoundFormat compoundFormat() { + return compoundFormat; + } + + @Override + public final PointsFormat pointsFormat() { + return new Lucene90PointsFormat(); + } + + @Override + public final KnnVectorsFormat knnVectorsFormat() { + return knnVectorsFormat; + } + + /** + * Returns the postings format that should be used for writing new segments of field. + * + *

The default implementation always returns "Lucene90". + * + *

WARNING: if you subclass, you are responsible for index backwards compatibility: + * future version of Lucene are only guaranteed to be able to read the default implementation, + */ + public PostingsFormat getPostingsFormatForField(String field) { + return defaultPostingsFormat; + } + + /** + * Returns the docvalues format that should be used for writing new segments of field + * . + * + *

The default implementation always returns "Lucene90". + * + *

WARNING: if you subclass, you are responsible for index backwards compatibility: + * future version of Lucene are only guaranteed to be able to read the default implementation. + */ + public DocValuesFormat getDocValuesFormatForField(String field) { + return defaultDVFormat; + } + + /** + * Returns the vectors format that should be used for writing new segments of field + * + *

The default implementation always returns "Lucene91". + * + *

WARNING: if you subclass, you are responsible for index backwards compatibility: + * future version of Lucene are only guaranteed to be able to read the default implementation. + */ + public KnnVectorsFormat getKnnVectorsFormatForField(String field) { + return defaultKnnVectorsFormat; + } + + @Override + public final DocValuesFormat docValuesFormat() { + return docValuesFormat; + } + + @Override + public final NormsFormat normsFormat() { + return normsFormat; + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene91/Lucene91HnswVectorsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene91/Lucene91HnswVectorsFormat.java new file mode 100644 index 00000000000..2e3fc6c298f --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene91/Lucene91HnswVectorsFormat.java @@ -0,0 +1,143 @@ +/* + * 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.lucene91; + +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.index.SegmentReadState; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.util.hnsw.HnswGraph; + +/** + * Lucene 9.0 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: + * + *

.vec (vector data) file

+ * + *

This file stores all the floating-point vector data ordered by field, document ordinal, and + * vector dimension. The floats are stored in little-endian byte order. + * + *

.vex (vector index)

+ * + *

Stores graphs connecting the documents for each field organized as a list of nodes' neighbours + * as following: + * + *

+ * + *

.vem (vector metadata) file

+ * + *

For each field: + * + *

+ * + * @lucene.experimental + */ +public final class Lucene91HnswVectorsFormat extends KnnVectorsFormat { + + static final String META_CODEC_NAME = "Lucene91HnswVectorsFormatMeta"; + static final String VECTOR_DATA_CODEC_NAME = "Lucene91HnswVectorsFormatData"; + static final String VECTOR_INDEX_CODEC_NAME = "Lucene91HnswVectorsFormatIndex"; + static final String META_EXTENSION = "vem"; + static final String VECTOR_DATA_EXTENSION = "vec"; + static final String VECTOR_INDEX_EXTENSION = "vex"; + + static final int VERSION_START = 0; + static final int VERSION_CURRENT = VERSION_START; + + /** Default number of maximum connections per node */ + public static final int DEFAULT_MAX_CONN = 16; + /** + * Default number of the size of the queue maintained while searching during a graph construction. + */ + public static final int DEFAULT_BEAM_WIDTH = 100; + + /** + * Controls how many of the nearest neighbor candidates are connected to the new node. Defaults to + * {@link Lucene91HnswVectorsFormat#DEFAULT_MAX_CONN}. See {@link HnswGraph} for more details. + */ + private final int maxConn; + + /** + * The number of candidate neighbors to track while searching the graph for each newly inserted + * node. Defaults to to {@link Lucene91HnswVectorsFormat#DEFAULT_BEAM_WIDTH}. See {@link + * HnswGraph} for details. + */ + private final int beamWidth; + + public Lucene91HnswVectorsFormat() { + this(DEFAULT_MAX_CONN, DEFAULT_BEAM_WIDTH); + } + + public Lucene91HnswVectorsFormat(int maxConn, int beamWidth) { + super("Lucene91HnswVectorsFormat"); + this.maxConn = maxConn; + this.beamWidth = beamWidth; + } + + @Override + public KnnVectorsWriter fieldsWriter(SegmentWriteState state) throws IOException { + return new Lucene91HnswVectorsWriter(state, maxConn, beamWidth); + } + + @Override + public KnnVectorsReader fieldsReader(SegmentReadState state) throws IOException { + return new Lucene91HnswVectorsReader(state); + } + + @Override + public String toString() { + return "Lucene91HnswVectorsFormat(name = Lucene91HnswVectorsFormat, maxConn = " + + maxConn + + ", beamWidth=" + + beamWidth + + ")"; + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene91/Lucene91HnswVectorsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene91/Lucene91HnswVectorsReader.java new file mode 100644 index 00000000000..92135f41aca --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene91/Lucene91HnswVectorsReader.java @@ -0,0 +1,554 @@ +/* + * 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.lucene91; + +import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.KnnVectorsReader; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.KnnGraphValues; +import org.apache.lucene.index.RandomAccessVectorValues; +import org.apache.lucene.index.RandomAccessVectorValuesProducer; +import org.apache.lucene.index.SegmentReadState; +import org.apache.lucene.index.VectorSimilarityFunction; +import org.apache.lucene.index.VectorValues; +import org.apache.lucene.search.ScoreDoc; +import org.apache.lucene.search.TopDocs; +import org.apache.lucene.search.TotalHits; +import org.apache.lucene.store.ChecksumIndexInput; +import org.apache.lucene.store.DataInput; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.RamUsageEstimator; +import org.apache.lucene.util.hnsw.HnswGraph; +import org.apache.lucene.util.hnsw.NeighborQueue; + +/** + * Reads vectors from the index segments along with index data structures supporting KNN search. + * + * @lucene.experimental + */ +public final class Lucene91HnswVectorsReader extends KnnVectorsReader { + + private final FieldInfos fieldInfos; + private final Map fields = new HashMap<>(); + private final IndexInput vectorData; + private final IndexInput vectorIndex; + + Lucene91HnswVectorsReader(SegmentReadState state) throws IOException { + this.fieldInfos = state.fieldInfos; + int versionMeta = readMetadata(state); + boolean success = false; + try { + vectorData = + openDataInput( + state, + versionMeta, + Lucene91HnswVectorsFormat.VECTOR_DATA_EXTENSION, + Lucene91HnswVectorsFormat.VECTOR_DATA_CODEC_NAME); + vectorIndex = + openDataInput( + state, + versionMeta, + Lucene91HnswVectorsFormat.VECTOR_INDEX_EXTENSION, + Lucene91HnswVectorsFormat.VECTOR_INDEX_CODEC_NAME); + 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, Lucene91HnswVectorsFormat.META_EXTENSION); + int versionMeta = -1; + try (ChecksumIndexInput meta = state.directory.openChecksumInput(metaFileName, state.context)) { + Throwable priorE = null; + try { + versionMeta = + CodecUtil.checkIndexHeader( + meta, + Lucene91HnswVectorsFormat.META_CODEC_NAME, + Lucene91HnswVectorsFormat.VERSION_START, + Lucene91HnswVectorsFormat.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); + int versionVectorData = + CodecUtil.checkIndexHeader( + in, + codecName, + Lucene91HnswVectorsFormat.VERSION_START, + Lucene91HnswVectorsFormat.VERSION_CURRENT, + state.segmentInfo.getId(), + state.segmentSuffix); + if (versionMeta != versionVectorData) { + throw new CorruptIndexException( + "Format versions mismatch: meta=" + + versionMeta + + ", " + + codecName + + "=" + + versionVectorData, + in); + } + CodecUtil.retrieveChecksum(in); + return 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); + } + + long numBytes = (long) fieldEntry.size() * dimension * Float.BYTES; + if (numBytes != fieldEntry.vectorDataLength) { + throw new IllegalStateException( + "Vector data length " + + fieldEntry.vectorDataLength + + " not matching size=" + + fieldEntry.size() + + " * dim=" + + dimension + + " * 4 = " + + numBytes); + } + } + + 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 FieldEntry readField(DataInput input) throws IOException { + VectorSimilarityFunction similarityFunction = readSimilarityFunction(input); + return new FieldEntry(input, similarityFunction); + } + + @Override + public long ramBytesUsed() { + long totalBytes = RamUsageEstimator.shallowSizeOfInstance(Lucene91HnswVectorsFormat.class); + totalBytes += + RamUsageEstimator.sizeOfMap( + fields, RamUsageEstimator.shallowSizeOfInstance(FieldEntry.class)); + for (FieldEntry entry : fields.values()) { + totalBytes += RamUsageEstimator.sizeOf(entry.ordToDoc); + } + return totalBytes; + } + + @Override + public void checkIntegrity() throws IOException { + CodecUtil.checksumEntireFile(vectorData); + CodecUtil.checksumEntireFile(vectorIndex); + } + + @Override + public VectorValues getVectorValues(String field) throws IOException { + FieldEntry fieldEntry = fields.get(field); + return getOffHeapVectorValues(fieldEntry); + } + + @Override + public TopDocs search(String field, float[] target, int k, Bits acceptDocs) throws IOException { + FieldEntry fieldEntry = fields.get(field); + + if (fieldEntry.size() == 0) { + return new TopDocs(new TotalHits(0, TotalHits.Relation.EQUAL_TO), new ScoreDoc[0]); + } + + // bound k by total number of vectors to prevent oversizing data structures + k = Math.min(k, fieldEntry.size()); + + OffHeapVectorValues vectorValues = getOffHeapVectorValues(fieldEntry); + // use a seed that is fixed for the index so we get reproducible results for the same query + NeighborQueue results = + HnswGraph.search( + target, + k, + vectorValues, + fieldEntry.similarityFunction, + getGraphValues(fieldEntry), + getAcceptOrds(acceptDocs, fieldEntry)); + int i = 0; + ScoreDoc[] scoreDocs = new ScoreDoc[Math.min(results.size(), k)]; + while (results.size() > 0) { + int node = results.topNode(); + float score = fieldEntry.similarityFunction.convertToScore(results.topScore()); + results.pop(); + scoreDocs[scoreDocs.length - ++i] = new ScoreDoc(fieldEntry.ordToDoc[node], score); + } + // always return >= the case where we can assert == is only when there are fewer than topK + // vectors in the index + return new TopDocs( + new TotalHits(results.visitedCount(), TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO), + scoreDocs); + } + + private OffHeapVectorValues getOffHeapVectorValues(FieldEntry fieldEntry) throws IOException { + IndexInput bytesSlice = + vectorData.slice("vector-data", fieldEntry.vectorDataOffset, fieldEntry.vectorDataLength); + return new OffHeapVectorValues(fieldEntry.dimension, fieldEntry.ordToDoc, bytesSlice); + } + + private Bits getAcceptOrds(Bits acceptDocs, FieldEntry fieldEntry) { + if (acceptDocs == null) { + return null; + } + return new Bits() { + @Override + public boolean get(int index) { + return acceptDocs.get(fieldEntry.ordToDoc[index]); + } + + @Override + public int length() { + return fieldEntry.ordToDoc.length; + } + }; + } + + /** Get knn graph values; used for testing */ + public KnnGraphValues getGraphValues(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 getGraphValues(entry); + } else { + return KnnGraphValues.EMPTY; + } + } + + private KnnGraphValues getGraphValues(FieldEntry entry) throws IOException { + IndexInput bytesSlice = + vectorIndex.slice("graph-data", entry.vectorIndexOffset, entry.vectorIndexLength); + return new IndexedKnnGraphReader(entry, bytesSlice); + } + + @Override + public void close() throws IOException { + IOUtils.close(vectorData, vectorIndex); + } + + private static class FieldEntry { + + final VectorSimilarityFunction similarityFunction; + final long vectorDataOffset; + final long vectorDataLength; + final long vectorIndexOffset; + final long vectorIndexLength; + final int maxConn; + final int numLevels; + final int dimension; + final int[] ordToDoc; + final int[][] nodesByLevel; + // for each level the start offsets in vectorIndex file from where to read neighbours + final long[] graphOffsetsByLevel; + + FieldEntry(DataInput input, VectorSimilarityFunction similarityFunction) throws IOException { + this.similarityFunction = similarityFunction; + vectorDataOffset = input.readVLong(); + vectorDataLength = input.readVLong(); + vectorIndexOffset = input.readVLong(); + vectorIndexLength = input.readVLong(); + dimension = input.readInt(); + int size = input.readInt(); + ordToDoc = new int[size]; + for (int i = 0; i < size; i++) { + int doc = input.readVInt(); + ordToDoc[i] = doc; + } + + // read nodes by level + maxConn = input.readInt(); + numLevels = input.readInt(); + nodesByLevel = new int[numLevels][]; + for (int level = 0; level < numLevels; level++) { + int numNodesOnLevel = input.readInt(); + if (level == 0) { + // we don't store nodes for level 0th, as this level contains all nodes + assert numNodesOnLevel == size; + nodesByLevel[0] = null; + } else { + nodesByLevel[level] = new int[numNodesOnLevel]; + for (int i = 0; i < numNodesOnLevel; i++) { + nodesByLevel[level][i] = input.readVInt(); + } + } + } + + // calculate for each level the start offsets in vectorIndex file from where to read + // neighbours + graphOffsetsByLevel = new long[numLevels]; + for (int level = 0; level < numLevels; level++) { + if (level == 0) { + graphOffsetsByLevel[level] = 0; + } else { + int numNodesOnPrevLevel = level == 1 ? size : nodesByLevel[level - 1].length; + graphOffsetsByLevel[level] = + graphOffsetsByLevel[level - 1] + (1 + maxConn) * Integer.BYTES * numNodesOnPrevLevel; + } + } + } + + int size() { + return ordToDoc.length; + } + } + + /** Read the vector values from the index input. This supports both iterated and random access. */ + static class OffHeapVectorValues extends VectorValues + implements RandomAccessVectorValues, RandomAccessVectorValuesProducer { + + final int dimension; + final int[] ordToDoc; + final IndexInput dataIn; + + final BytesRef binaryValue; + final ByteBuffer byteBuffer; + final int byteSize; + final float[] value; + + int ord = -1; + int doc = -1; + + OffHeapVectorValues(int dimension, int[] ordToDoc, IndexInput dataIn) { + this.dimension = dimension; + this.ordToDoc = ordToDoc; + this.dataIn = dataIn; + byteSize = Float.BYTES * dimension; + byteBuffer = ByteBuffer.allocate(byteSize); + value = new float[dimension]; + binaryValue = new BytesRef(byteBuffer.array(), byteBuffer.arrayOffset(), byteSize); + } + + @Override + public int dimension() { + return dimension; + } + + @Override + public int size() { + return ordToDoc.length; + } + + @Override + public float[] vectorValue() throws IOException { + dataIn.seek((long) ord * byteSize); + dataIn.readFloats(value, 0, value.length); + return value; + } + + @Override + public BytesRef binaryValue() throws IOException { + dataIn.seek((long) ord * byteSize); + dataIn.readBytes(byteBuffer.array(), byteBuffer.arrayOffset(), byteSize, false); + return binaryValue; + } + + @Override + public int docID() { + return doc; + } + + @Override + public int nextDoc() { + if (++ord >= size()) { + doc = NO_MORE_DOCS; + } else { + doc = ordToDoc[ord]; + } + return doc; + } + + @Override + public int advance(int target) { + assert docID() < target; + ord = Arrays.binarySearch(ordToDoc, ord + 1, ordToDoc.length, target); + if (ord < 0) { + ord = -(ord + 1); + } + assert ord <= ordToDoc.length; + if (ord == ordToDoc.length) { + doc = NO_MORE_DOCS; + } else { + doc = ordToDoc[ord]; + } + return doc; + } + + @Override + public long cost() { + return ordToDoc.length; + } + + @Override + public RandomAccessVectorValues randomAccess() { + return new OffHeapVectorValues(dimension, ordToDoc, dataIn.clone()); + } + + @Override + public float[] vectorValue(int targetOrd) throws IOException { + dataIn.seek((long) targetOrd * byteSize); + dataIn.readFloats(value, 0, value.length); + return value; + } + + @Override + public BytesRef binaryValue(int targetOrd) throws IOException { + readValue(targetOrd); + return binaryValue; + } + + private void readValue(int targetOrd) throws IOException { + dataIn.seek((long) targetOrd * byteSize); + dataIn.readBytes(byteBuffer.array(), byteBuffer.arrayOffset(), byteSize); + } + } + + /** Read the nearest-neighbors graph from the index input */ + private static final class IndexedKnnGraphReader extends KnnGraphValues { + + final IndexInput dataIn; + final int[][] nodesByLevel; + final long[] graphOffsetsByLevel; + final int numLevels; + final int entryNode; + final int size; + final long bytesForConns; + + int arcCount; + int arcUpTo; + int arc; + + IndexedKnnGraphReader(FieldEntry entry, IndexInput dataIn) { + this.dataIn = dataIn; + this.nodesByLevel = entry.nodesByLevel; + this.numLevels = entry.numLevels; + this.entryNode = numLevels > 1 ? nodesByLevel[numLevels - 1][0] : 0; + this.size = entry.size(); + this.graphOffsetsByLevel = entry.graphOffsetsByLevel; + this.bytesForConns = ((long) entry.maxConn + 1) * Integer.BYTES; + } + + @Override + public void seek(int level, int targetOrd) throws IOException { + int targetIndex = + level == 0 + ? targetOrd + : Arrays.binarySearch(nodesByLevel[level], 0, nodesByLevel[level].length, targetOrd); + assert targetIndex >= 0; + long graphDataOffset = graphOffsetsByLevel[level] + targetIndex * bytesForConns; + // unsafe; no bounds checking + dataIn.seek(graphDataOffset); + arcCount = dataIn.readInt(); + arc = -1; + arcUpTo = 0; + } + + @Override + public int size() { + return size; + } + + @Override + public int nextNeighbor() throws IOException { + if (arcUpTo >= arcCount) { + return NO_MORE_DOCS; + } + ++arcUpTo; + arc = dataIn.readInt(); + return arc; + } + + @Override + public int numLevels() throws IOException { + return numLevels; + } + + @Override + public int entryNode() throws IOException { + return entryNode; + } + + @Override + public NodesIterator getNodesOnLevel(int level) { + if (level == 0) { + return new NodesIterator(size()); + } else { + return new NodesIterator(nodesByLevel[level], nodesByLevel[level].length); + } + } + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene91/Lucene91HnswVectorsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene91/Lucene91HnswVectorsWriter.java new file mode 100644 index 00000000000..f56f1d72e25 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene91/Lucene91HnswVectorsWriter.java @@ -0,0 +1,295 @@ +/* + * 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.lucene91; + +import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; + +import java.io.IOException; +import java.util.Arrays; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.KnnVectorsReader; +import org.apache.lucene.codecs.KnnVectorsWriter; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.KnnGraphValues.NodesIterator; +import org.apache.lucene.index.RandomAccessVectorValuesProducer; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.index.VectorSimilarityFunction; +import org.apache.lucene.index.VectorValues; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.hnsw.HnswGraph; +import org.apache.lucene.util.hnsw.HnswGraphBuilder; +import org.apache.lucene.util.hnsw.NeighborArray; + +/** + * Writes vector values and knn graphs to index segments. + * + * @lucene.experimental + */ +public final class Lucene91HnswVectorsWriter extends KnnVectorsWriter { + + private final SegmentWriteState segmentWriteState; + private final IndexOutput meta, vectorData, vectorIndex; + + private final int maxConn; + private final int beamWidth; + private boolean finished; + + Lucene91HnswVectorsWriter(SegmentWriteState state, int maxConn, int beamWidth) + throws IOException { + this.maxConn = maxConn; + this.beamWidth = beamWidth; + + assert state.fieldInfos.hasVectorValues(); + segmentWriteState = state; + + String metaFileName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, state.segmentSuffix, Lucene91HnswVectorsFormat.META_EXTENSION); + + String vectorDataFileName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, + state.segmentSuffix, + Lucene91HnswVectorsFormat.VECTOR_DATA_EXTENSION); + + String indexDataFileName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, + state.segmentSuffix, + Lucene91HnswVectorsFormat.VECTOR_INDEX_EXTENSION); + + 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, + Lucene91HnswVectorsFormat.META_CODEC_NAME, + Lucene91HnswVectorsFormat.VERSION_CURRENT, + state.segmentInfo.getId(), + state.segmentSuffix); + CodecUtil.writeIndexHeader( + vectorData, + Lucene91HnswVectorsFormat.VECTOR_DATA_CODEC_NAME, + Lucene91HnswVectorsFormat.VERSION_CURRENT, + state.segmentInfo.getId(), + state.segmentSuffix); + CodecUtil.writeIndexHeader( + vectorIndex, + Lucene91HnswVectorsFormat.VECTOR_INDEX_CODEC_NAME, + Lucene91HnswVectorsFormat.VERSION_CURRENT, + state.segmentInfo.getId(), + state.segmentSuffix); + success = true; + } finally { + if (success == false) { + IOUtils.closeWhileHandlingException(this); + } + } + } + + @Override + public void writeField(FieldInfo fieldInfo, KnnVectorsReader knnVectorsReader) + throws IOException { + long vectorDataOffset = vectorData.alignFilePointer(Float.BYTES); + VectorValues vectors = knnVectorsReader.getVectorValues(fieldInfo.name); + + IndexOutput tempVectorData = + segmentWriteState.directory.createTempOutput( + vectorData.getName(), "temp", segmentWriteState.context); + IndexInput vectorDataInput = null; + boolean success = false; + try { + // write the vector data to a temporary file + // TODO - use a better data structure; a bitset? DocsWithFieldSet is p.p. in o.a.l.index + int[] docIds = writeVectorData(tempVectorData, vectors); + CodecUtil.writeFooter(tempVectorData); + IOUtils.close(tempVectorData); + + // copy the temporary file vectors to the actual data file + vectorDataInput = + segmentWriteState.directory.openInput( + tempVectorData.getName(), segmentWriteState.context); + vectorData.copyBytes(vectorDataInput, vectorDataInput.length() - CodecUtil.footerLength()); + CodecUtil.retrieveChecksum(vectorDataInput); + long vectorDataLength = vectorData.getFilePointer() - vectorDataOffset; + + long vectorIndexOffset = vectorIndex.getFilePointer(); + // build the graph using the temporary vector data + Lucene91HnswVectorsReader.OffHeapVectorValues offHeapVectors = + new Lucene91HnswVectorsReader.OffHeapVectorValues( + vectors.dimension(), docIds, vectorDataInput); + HnswGraph graph = + offHeapVectors.size() == 0 + ? null + : writeGraph(offHeapVectors, fieldInfo.getVectorSimilarityFunction()); + long vectorIndexLength = vectorIndex.getFilePointer() - vectorIndexOffset; + writeMeta( + fieldInfo, + vectorDataOffset, + vectorDataLength, + vectorIndexOffset, + vectorIndexLength, + docIds, + graph); + success = true; + } finally { + IOUtils.close(vectorDataInput); + if (success) { + segmentWriteState.directory.deleteFile(tempVectorData.getName()); + } else { + IOUtils.closeWhileHandlingException(tempVectorData); + IOUtils.deleteFilesIgnoringExceptions( + segmentWriteState.directory, tempVectorData.getName()); + } + } + } + + /** + * Writes the vector values to the output and returns a mapping from dense ordinals to document + * IDs. The length of the returned array matches the total number of documents with a vector + * (which excludes deleted documents), so it may be less than {@link VectorValues#size()}. + */ + private static int[] writeVectorData(IndexOutput output, VectorValues vectors) + throws IOException { + int[] docIds = new int[vectors.size()]; + int count = 0; + for (int docV = vectors.nextDoc(); docV != NO_MORE_DOCS; docV = vectors.nextDoc(), count++) { + // write vector + BytesRef binaryValue = vectors.binaryValue(); + assert binaryValue.length == vectors.dimension() * Float.BYTES; + output.writeBytes(binaryValue.bytes, binaryValue.offset, binaryValue.length); + docIds[count] = docV; + } + + if (docIds.length > count) { + return ArrayUtil.copyOfSubArray(docIds, 0, count); + } + return docIds; + } + + private void writeMeta( + FieldInfo field, + long vectorDataOffset, + long vectorDataLength, + long vectorIndexOffset, + long vectorIndexLength, + int[] docIds, + HnswGraph graph) + throws IOException { + meta.writeInt(field.number); + meta.writeInt(field.getVectorSimilarityFunction().ordinal()); + meta.writeVLong(vectorDataOffset); + meta.writeVLong(vectorDataLength); + meta.writeVLong(vectorIndexOffset); + meta.writeVLong(vectorIndexLength); + meta.writeInt(field.getVectorDimension()); + meta.writeInt(docIds.length); + for (int docId : docIds) { + // TODO: delta-encode, or write as bitset + meta.writeVInt(docId); + } + + meta.writeInt(maxConn); + + // write graph nodes on each level + if (graph == null) { + meta.writeInt(0); + } else { + meta.writeInt(graph.numLevels()); + for (int level = 0; level < graph.numLevels(); level++) { + NodesIterator nodesOnLevel = graph.getNodesOnLevel(level); + meta.writeInt(nodesOnLevel.size()); // number of nodes on a level + if (level > 0) { + while (nodesOnLevel.hasNext()) { + int node = nodesOnLevel.nextInt(); + meta.writeVInt(node); // list of nodes on a level + } + } + } + } + } + + private HnswGraph writeGraph( + RandomAccessVectorValuesProducer vectorValues, VectorSimilarityFunction similarityFunction) + throws IOException { + + // build graph + HnswGraphBuilder hnswGraphBuilder = + new HnswGraphBuilder( + vectorValues, similarityFunction, maxConn, beamWidth, HnswGraphBuilder.randSeed); + hnswGraphBuilder.setInfoStream(segmentWriteState.infoStream); + HnswGraph graph = hnswGraphBuilder.build(vectorValues.randomAccess()); + + // write vectors' neighbours on each level into the vectorIndex file + int countOnLevel0 = graph.size(); + for (int level = 0; level < graph.numLevels(); level++) { + NodesIterator nodesOnLevel = graph.getNodesOnLevel(level); + while (nodesOnLevel.hasNext()) { + int node = nodesOnLevel.nextInt(); + NeighborArray neighbors = graph.getNeighbors(level, node); + int size = neighbors.size(); + vectorIndex.writeInt(size); + // Destructively modify; it's ok we are discarding it after this + int[] nnodes = neighbors.node(); + Arrays.sort(nnodes, 0, size); + for (int i = 0; i < size; i++) { + int nnode = nnodes[i]; + assert nnode < countOnLevel0 : "node too large: " + nnode + ">=" + countOnLevel0; + vectorIndex.writeInt(nnode); + } + // if number of connections < maxConn, add bogus values up to maxConn to have predictable + // offsets + for (int i = size; i < maxConn; i++) { + vectorIndex.writeInt(0); + } + } + } + return graph; + } + + @Override + public void finish() throws IOException { + if (finished) { + throw new IllegalStateException("already finished"); + } + finished = true; + + 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 void close() throws IOException { + IOUtils.close(meta, vectorData, vectorIndex); + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene91/package-info.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene91/package-info.java new file mode 100644 index 00000000000..04603320664 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene91/package-info.java @@ -0,0 +1,420 @@ +/* + * 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. + */ + +/** + * Lucene 9.1 file format. + * + *

Apache Lucene - Index File Formats

+ * + *
+ * + * + * + *
+ * + *

Introduction

+ * + *
+ * + *

This document defines the index file formats used in this version of Lucene. If you are using + * a different version of Lucene, please consult the copy of docs/ that was distributed + * with the version you are using. + * + *

This document attempts to provide a high-level definition of the Apache Lucene file formats. + *

+ * + *

Definitions

+ * + *
+ * + *

The fundamental concepts in Lucene are index, document, field and term. + * + *

An index contains a sequence of documents. + * + *

+ * + *

The same sequence of bytes in two different fields is considered a different term. Thus terms + * are represented as a pair: the string naming the field, and the bytes within the field. + * + *

Inverted Indexing

+ * + *

Lucene's index stores terms and statistics about those terms in order to make term-based + * search more efficient. Lucene's terms index falls into the family of indexes known as an + * inverted index. This is because it can list, for a term, the documents that contain it. + * This is the inverse of the natural relationship, in which documents list terms. + * + *

Types of Fields

+ * + *

In Lucene, fields may be stored, in which case their text is stored in the index + * literally, in a non-inverted manner. Fields that are inverted are called indexed. A field + * may be both stored and indexed. + * + *

The text of a field may be tokenized into terms to be indexed, or the text of a field + * may be used literally as a term to be indexed. Most fields are tokenized, but sometimes it is + * useful for certain identifier fields to be indexed literally. + * + *

See the {@link org.apache.lucene.document.Field Field} java docs for more information on + * Fields. + * + *

Segments

+ * + *

Lucene indexes may be composed of multiple sub-indexes, or segments. Each segment is a + * fully independent index, which could be searched separately. Indexes evolve by: + * + *

    + *
  1. Creating new segments for newly added documents. + *
  2. Merging existing segments. + *
+ * + *

Searches may involve multiple segments and/or multiple indexes, each index potentially + * composed of a set of segments. + * + *

Document Numbers

+ * + *

Internally, Lucene refers to documents by an integer document number. The first + * document added to an index is numbered zero, and each subsequent document added gets a number one + * greater than the previous. + * + *

Note that a document's number may change, so caution should be taken when storing these + * numbers outside of Lucene. In particular, numbers may change in the following situations: + * + *

+ * + *
+ * + *

Index Structure Overview

+ * + *
+ * + *

Each segment index maintains the following: + * + *

+ * + *

Details on each of these are provided in their linked pages.

+ * + *

File Naming

+ * + *
+ * + *

All files belonging to a segment have the same name with varying extensions. The extensions + * correspond to the different file formats described below. When using the Compound File format + * (default for small segments) these files (except for the Segment info file, the Lock file, and + * Deleted documents file) are collapsed into a single .cfs file (see below for details) + * + *

Typically, all segments in an index are stored in a single directory, although this is not + * required. + * + *

File names are never re-used. That is, when any file is saved to the Directory it is given a + * never before used filename. This is achieved using a simple generations approach. For example, + * the first segments file is segments_1, then segments_2, etc. The generation is a sequential long + * integer represented in alpha-numeric (base 36) form.

+ * + *

Summary of File Extensions

+ * + *
+ * + *

The following table summarizes the names and extensions of the files in Lucene: + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + *
lucene filenames by extension
NameExtensionBrief Description
{@link org.apache.lucene.index.SegmentInfos Segments File}segments_NStores information about a commit point
Lock Filewrite.lockThe Write lock prevents multiple IndexWriters from writing to the same + * file.
{@link org.apache.lucene.codecs.lucene90.Lucene90SegmentInfoFormat Segment Info}.siStores metadata about a segment
{@link org.apache.lucene.codecs.lucene90.Lucene90CompoundFormat Compound File}.cfs, .cfeAn optional "virtual" file consisting of all the other index files for + * systems that frequently run out of file handles.
{@link org.apache.lucene.codecs.lucene90.Lucene90FieldInfosFormat Fields}.fnmStores information about the fields
{@link org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat Field Index}.fdxContains pointers to field data
{@link org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat Field Data}.fdtThe stored fields for documents
{@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat Term Dictionary}.timThe term dictionary, stores term info
{@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat Term Index}.tipThe index into the Term Dictionary
{@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat Frequencies}.docContains the list of docs which contain each term along with frequency
{@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat Positions}.posStores position information about where a term occurs in the index
{@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat Payloads}.payStores additional per-position metadata information such as character offsets and user payloads
{@link org.apache.lucene.codecs.lucene90.Lucene90NormsFormat Norms}.nvd, .nvmEncodes length and boost factors for docs and fields
{@link org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat Per-Document Values}.dvd, .dvmEncodes additional scoring factors or other per-document information.
{@link org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat Term Vector Index}.tvxStores offset into the document data file
{@link org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat Term Vector Data}.tvdContains term vector data.
{@link org.apache.lucene.codecs.lucene90.Lucene90LiveDocsFormat Live Documents}.livInfo about what documents are live
{@link org.apache.lucene.codecs.lucene90.Lucene90PointsFormat Point values}.dii, .dimHolds indexed points
{@link org.apache.lucene.codecs.lucene91.Lucene91HnswVectorsFormat Vector values}.vec, .vemHolds indexed vectors; .vec files contain the raw vector data, and + * .vem the vector metadata
+ * + *

+ * + *

Lock File

+ * + * The write lock, which is stored in the index directory by default, is named "write.lock". If the + * lock directory is different from the index directory then the write lock will be named + * "XXXX-write.lock" where XXXX is a unique prefix derived from the full path to the index + * directory. When this file is present, a writer is currently modifying the index (adding or + * removing documents). This lock file ensures that only one writer is modifying the index at a + * time. + * + *

History

+ * + *

Compatibility notes are provided in this document, describing how file formats have changed + * from prior versions: + * + *

+ * + * + * + *

Limitations

+ * + *
+ * + *

Lucene uses a Java int to refer to document numbers, and the index file format + * uses an Int32 on-disk to store document numbers. This is a limitation of both the + * index file format and the current implementation. Eventually these should be replaced with either + * UInt64 values, or better yet, {@link org.apache.lucene.store.DataOutput#writeVInt + * VInt} values which have no limit.

+ */ +package org.apache.lucene.codecs.lucene91; diff --git a/lucene/core/src/java/org/apache/lucene/index/KnnGraphValues.java b/lucene/core/src/java/org/apache/lucene/index/KnnGraphValues.java index f8f175acd04..07feb2a8136 100644 --- a/lucene/core/src/java/org/apache/lucene/index/KnnGraphValues.java +++ b/lucene/core/src/java/org/apache/lucene/index/KnnGraphValues.java @@ -20,6 +20,8 @@ package org.apache.lucene.index; import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; import java.io.IOException; +import java.util.NoSuchElementException; +import java.util.PrimitiveIterator; /** * Access to per-document neighbor lists in a (hierarchical) knn search graph. @@ -32,25 +34,40 @@ public abstract class KnnGraphValues { protected KnnGraphValues() {} /** - * Move the pointer to exactly {@code target}, the id of a node in the graph. After this method + * Move the pointer to exactly the given {@code level}'s {@code target}. After this method * returns, call {@link #nextNeighbor()} to return successive (ordered) connected node ordinals. * - * @param target must be a valid node in the graph, ie. ≥ 0 and < {@link + * @param level level of the graph + * @param target ordinal of a node in the graph, must be ≥ 0 and < {@link * VectorValues#size()}. */ - public abstract void seek(int target) throws IOException; + public abstract void seek(int level, int target) throws IOException; /** Returns the number of nodes in the graph */ public abstract int size(); /** * Iterates over the neighbor list. It is illegal to call this method after it returns - * NO_MORE_DOCS without calling {@link #seek(int)}, which resets the iterator. + * NO_MORE_DOCS without calling {@link #seek(int, int)}, which resets the iterator. * * @return a node ordinal in the graph, or NO_MORE_DOCS if the iteration is complete. */ public abstract int nextNeighbor() throws IOException; + /** Returns the number of levels of the graph */ + public abstract int numLevels() throws IOException; + + /** Returns graph's entry point on the top level * */ + public abstract int entryNode() throws IOException; + + /** + * Get all nodes on a given level as node 0th ordinals + * + * @param level level for which to get all nodes + * @return an iterator over nodes where {@code nextInt} returns a next node on the level + */ + public abstract NodesIterator getNodesOnLevel(int level) throws IOException; + /** Empty graph value */ public static KnnGraphValues EMPTY = new KnnGraphValues() { @@ -61,11 +78,74 @@ public abstract class KnnGraphValues { } @Override - public void seek(int target) {} + public void seek(int level, int target) {} @Override public int size() { return 0; } + + @Override + public int numLevels() { + return 0; + } + + @Override + public int entryNode() { + return 0; + } + + @Override + public NodesIterator getNodesOnLevel(int level) { + return NodesIterator.EMPTY; + } }; + + /** + * Iterator over the graph nodes on a certain level, Iterator also provides the size – the total + * number of nodes to be iterated over. + */ + public static final class NodesIterator implements PrimitiveIterator.OfInt { + static NodesIterator EMPTY = new NodesIterator(0); + + private final int[] nodes; + private final int size; + int cur = 0; + + /** Constructor for iterator based on the nodes array up to the size */ + public NodesIterator(int[] nodes, int size) { + assert nodes != null; + assert size <= nodes.length; + this.nodes = nodes; + this.size = size; + } + + /** Constructor for iterator based on the size */ + public NodesIterator(int size) { + this.nodes = null; + this.size = size; + } + + @Override + public int nextInt() { + if (hasNext() == false) { + throw new NoSuchElementException(); + } + if (nodes == null) { + return cur++; + } else { + return nodes[cur++]; + } + } + + @Override + public boolean hasNext() { + return cur < size; + } + + /** The number of elements in this iterator * */ + public int size() { + return size; + } + } } diff --git a/lucene/core/src/java/org/apache/lucene/util/hnsw/BoundsChecker.java b/lucene/core/src/java/org/apache/lucene/util/hnsw/BoundsChecker.java index ce152043836..9cde17db421 100644 --- a/lucene/core/src/java/org/apache/lucene/util/hnsw/BoundsChecker.java +++ b/lucene/core/src/java/org/apache/lucene/util/hnsw/BoundsChecker.java @@ -17,22 +17,26 @@ package org.apache.lucene.util.hnsw; -abstract class BoundsChecker { +/** + * A helper class for an hnsw graph that serves as a comparator of the currently set bound value + * with a new value. + */ +public abstract class BoundsChecker { float bound; /** Update the bound if sample is better */ - abstract void update(float sample); + public abstract void update(float sample); /** Update the bound unconditionally */ - void set(float sample) { + public void set(float sample) { bound = sample; } /** @return whether the sample exceeds (is worse than) the bound */ - abstract boolean check(float sample); + public abstract boolean check(float sample); - static BoundsChecker create(boolean reversed) { + public static BoundsChecker create(boolean reversed) { if (reversed) { return new Min(); } else { @@ -40,39 +44,47 @@ abstract class BoundsChecker { } } - static class Max extends BoundsChecker { + /** + * A helper class for an hnsw graph that serves as a comparator of the currently set maximum value + * with a new value. + */ + public static class Max extends BoundsChecker { Max() { bound = Float.NEGATIVE_INFINITY; } @Override - void update(float sample) { + public void update(float sample) { if (sample > bound) { bound = sample; } } @Override - boolean check(float sample) { + public boolean check(float sample) { return sample < bound; } } - static class Min extends BoundsChecker { + /** + * A helper class for an hnsw graph that serves as a comparator of the currently set minimum value + * with a new value. + */ + public static class Min extends BoundsChecker { Min() { bound = Float.POSITIVE_INFINITY; } @Override - void update(float sample) { + public void update(float sample) { if (sample < bound) { bound = sample; } } @Override - boolean check(float sample) { + public boolean check(float sample) { return sample > bound; } } diff --git a/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraph.java b/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraph.java index 9af10170803..6678bb47d8e 100644 --- a/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraph.java +++ b/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraph.java @@ -21,32 +21,31 @@ import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; -import java.util.SplittableRandom; import org.apache.lucene.index.KnnGraphValues; import org.apache.lucene.index.RandomAccessVectorValues; import org.apache.lucene.index.VectorSimilarityFunction; +import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.Bits; import org.apache.lucene.util.SparseFixedBitSet; /** - * Navigable Small-world graph. Provides efficient approximate nearest neighbor search for high - * dimensional vectors. See Approximate nearest - * neighbor algorithm based on navigable small world graphs [2014] and this paper [2018] for details. + * Hierarchical Navigable Small World graph. Provides efficient approximate nearest neighbor search + * for high dimensional vectors. See Efficient and robust + * approximate nearest neighbor search using Hierarchical Navigable Small World graphs [2018] + * paper for details. * - *

The nomenclature is a bit different here from what's used in those papers: + *

The nomenclature is a bit different here from what's used in the paper: * *

Hyperparameters

* * * *

Note: The graph may be searched by multiple threads concurrently, but updates are not @@ -56,75 +55,120 @@ import org.apache.lucene.util.SparseFixedBitSet; public final class HnswGraph extends KnnGraphValues { private final int maxConn; + private int numLevels; // the current number of levels in the graph + private int entryNode; // the current graph entry node on the top level - // Each entry lists the top maxConn neighbors of a node. The nodes correspond to vectors added to - // HnswBuilder, and the - // node values are the ordinals of those vectors. - private final List graph; + // Nodes by level expressed as the level 0's nodes' ordinals. + // As level 0 contains all nodes, nodesByLevel.get(0) is null. + private final List nodesByLevel; + + // graph is a list of graph levels. + // Each level is represented as List – nodes' connections on this level. + // Each entry in the list has the top maxConn neighbors of a node. The nodes correspond to vectors + // added to HnswBuilder, and the node values are the ordinals of those vectors. + // Thus, on all levels, neighbors expressed as the level 0's nodes' ordinals. + private final List> graph; // KnnGraphValues iterator members private int upto; private NeighborArray cur; - HnswGraph(int maxConn) { - graph = new ArrayList<>(); - // Typically with diversity criteria we see nodes not fully occupied; average fanout seems to be - // about 1/2 maxConn. There is some indexing time penalty for under-allocating, but saves RAM - graph.add(new NeighborArray(Math.max(32, maxConn / 4))); + HnswGraph(int maxConn, int levelOfFirstNode) { this.maxConn = maxConn; + this.numLevels = levelOfFirstNode + 1; + this.graph = new ArrayList<>(numLevels); + this.entryNode = 0; + for (int i = 0; i < numLevels; i++) { + graph.add(new ArrayList<>()); + // Typically with diversity criteria we see nodes not fully occupied; + // average fanout seems to be about 1/2 maxConn. + // There is some indexing time penalty for under-allocating, but saves RAM + graph.get(i).add(new NeighborArray(Math.max(32, maxConn / 4))); + } + + this.nodesByLevel = new ArrayList<>(numLevels); + nodesByLevel.add(null); // we don't need this for 0th level, as it contains all nodes + for (int l = 1; l < numLevels; l++) { + nodesByLevel.add(new int[] {0}); + } } /** - * Searches for the nearest neighbors of a query vector. + * Searches HNSW graph for the nearest neighbors of a query vector. * * @param query search query vector * @param topK the number of nodes to be returned - * @param numSeed the size of the queue maintained while searching, and controls the number of - * random entry points to sample * @param vectors vector values * @param graphValues the graph values. May represent the entire graph, or a level in a * hierarchical graph. * @param acceptOrds {@link Bits} that represents the allowed document ordinals to match, or * {@code null} if they are all allowed to match. - * @param random a source of randomness, used for generating entry points to the graph * @return a priority queue holding the closest neighbors found */ public static NeighborQueue search( float[] query, int topK, - int numSeed, RandomAccessVectorValues vectors, VectorSimilarityFunction similarityFunction, KnnGraphValues graphValues, - Bits acceptOrds, - SplittableRandom random) + Bits acceptOrds) throws IOException { + + NeighborQueue results; + int[] eps = new int[] {graphValues.entryNode()}; + for (int level = graphValues.numLevels() - 1; level >= 1; level--) { + results = searchLevel(query, 1, level, eps, vectors, similarityFunction, graphValues, null); + eps[0] = results.pop(); + } + results = + searchLevel(query, topK, 0, eps, vectors, similarityFunction, graphValues, acceptOrds); + return results; + } + + /** + * Searches for the nearest neighbors of a query vector in a given level + * + * @param query search query vector + * @param topK the number of nearest to query results to return + * @param level level to search + * @param eps the entry points for search at this level expressed as level 0th ordinals + * @param vectors vector values + * @param similarityFunction similarity function + * @param graphValues the graph values + * @param acceptOrds {@link Bits} that represents the allowed document ordinals to match, or + * {@code null} if they are all allowed to match. + * @return a priority queue holding the closest neighbors found + */ + static NeighborQueue searchLevel( + float[] query, + int topK, + int level, + final int[] eps, + RandomAccessVectorValues vectors, + VectorSimilarityFunction similarityFunction, + KnnGraphValues graphValues, + Bits acceptOrds) + throws IOException { + int size = graphValues.size(); - // MIN heap, holding the top results - NeighborQueue results = new NeighborQueue(numSeed, similarityFunction.reversed); + NeighborQueue results = new NeighborQueue(topK, similarityFunction.reversed); // MAX heap, from which to pull the candidate nodes - NeighborQueue candidates = new NeighborQueue(numSeed, !similarityFunction.reversed); - + NeighborQueue candidates = new NeighborQueue(topK, !similarityFunction.reversed); // set of ordinals that have been visited by search on this layer, used to avoid backtracking SparseFixedBitSet visited = new SparseFixedBitSet(size); - // get initial candidates at random - int boundedNumSeed = Math.min(numSeed, 2 * size); - for (int i = 0; i < boundedNumSeed; i++) { - int entryPoint = random.nextInt(size); - if (visited.getAndSet(entryPoint) == false) { - // explore the topK starting points of some random numSeed probes - float score = similarityFunction.compare(query, vectors.vectorValue(entryPoint)); - candidates.add(entryPoint, score); - if (acceptOrds == null || acceptOrds.get(entryPoint)) { - results.add(entryPoint, score); + for (int ep : eps) { + if (visited.getAndSet(ep) == false) { + float score = similarityFunction.compare(query, vectors.vectorValue(ep)); + candidates.add(ep, score); + if (acceptOrds == null || acceptOrds.get(ep)) { + results.add(ep, score); } } } // Set the bound to the worst current result and below reject any newly-generated candidates - // failing - // to exceed this bound + // failing to exceed this bound BoundsChecker bound = BoundsChecker.create(similarityFunction.reversed); bound.set(results.topScore()); while (candidates.size() > 0) { @@ -136,7 +180,7 @@ public final class HnswGraph extends KnnGraphValues { } } int topCandidateNode = candidates.pop(); - graphValues.seek(topCandidateNode); + graphValues.seek(level, topCandidateNode); int friendOrd; while ((friendOrd = graphValues.nextNeighbor()) != NO_MORE_DOCS) { assert friendOrd < size : "friendOrd=" + friendOrd + "; size=" + size; @@ -145,7 +189,7 @@ public final class HnswGraph extends KnnGraphValues { } float score = similarityFunction.compare(query, vectors.vectorValue(friendOrd)); - if (results.size() < numSeed || bound.check(score) == false) { + if (results.size() < topK || bound.check(score) == false) { candidates.add(friendOrd, score); if (acceptOrds == null || acceptOrds.get(friendOrd)) { results.insertWithOverflow(friendOrd, score); @@ -164,25 +208,60 @@ public final class HnswGraph extends KnnGraphValues { /** * Returns the {@link NeighborQueue} connected to the given node. * - * @param node the node whose neighbors are returned + * @param level level of the graph + * @param node the node whose neighbors are returned, represented as an ordinal on the level 0. */ - public NeighborArray getNeighbors(int node) { - return graph.get(node); + public NeighborArray getNeighbors(int level, int node) { + if (level == 0) { + return graph.get(level).get(node); + } + int nodeIndex = Arrays.binarySearch(nodesByLevel.get(level), 0, graph.get(level).size(), node); + assert nodeIndex >= 0; + return graph.get(level).get(nodeIndex); } @Override public int size() { - return graph.size(); + return graph.get(0).size(); // all nodes are located on the 0th level } - int addNode() { - graph.add(new NeighborArray(maxConn + 1)); - return graph.size() - 1; + /** + * Add node on the given level + * + * @param level level to add a node on + * @param node the node to add, represented as an ordinal on the level 0. + */ + public void addNode(int level, int node) { + if (level > 0) { + // if the new node introduces a new level, add more levels to the graph, + // and make this node the graph's new entry point + if (level >= numLevels) { + for (int i = numLevels; i <= level; i++) { + graph.add(new ArrayList<>()); + nodesByLevel.add(new int[] {node}); + } + numLevels = level + 1; + entryNode = node; + } else { + // Add this node id to this level's nodes + int[] nodes = nodesByLevel.get(level); + int idx = graph.get(level).size(); + if (idx < nodes.length) { + nodes[idx] = node; + } else { + nodes = ArrayUtil.grow(nodes); + nodes[idx] = node; + nodesByLevel.set(level, nodes); + } + } + } + + graph.get(level).add(new NeighborArray(maxConn + 1)); } @Override - public void seek(int targetNode) { - cur = getNeighbors(targetNode); + public void seek(int level, int targetNode) { + cur = getNeighbors(level, targetNode); upto = -1; } @@ -193,4 +272,34 @@ public final class HnswGraph extends KnnGraphValues { } return NO_MORE_DOCS; } + + /** + * Returns the current number of levels in the graph + * + * @return the current number of levels in the graph + */ + @Override + public int numLevels() { + return numLevels; + } + + /** + * Returns the graph's current entry node on the top level shown as ordinals of the nodes on 0th + * level + * + * @return the graph's current entry node on the top level + */ + @Override + public int entryNode() { + return entryNode; + } + + @Override + public NodesIterator getNodesOnLevel(int level) { + if (level == 0) { + return new NodesIterator(size()); + } else { + return new NodesIterator(nodesByLevel.get(level), graph.get(level).size()); + } + } } diff --git a/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java b/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java index f5cfc6a854f..041fd181a1e 100644 --- a/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java +++ b/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java @@ -17,6 +17,8 @@ package org.apache.lucene.util.hnsw; +import static java.lang.Math.log; + import java.io.IOException; import java.util.Locale; import java.util.Objects; @@ -32,15 +34,17 @@ import org.apache.lucene.util.InfoStream; */ public final class HnswGraphBuilder { - // default random seed for level generation + /** Default random seed for level generation * */ private static final long DEFAULT_RAND_SEED = System.currentTimeMillis(); + /** A name for the HNSW component for the info-stream * */ public static final String HNSW_COMPONENT = "HNSW"; - // expose for testing. + /** Random seed for level generation; public to expose for testing * */ public static long randSeed = DEFAULT_RAND_SEED; private final int maxConn; private final int beamWidth; + private final double ml; private final NeighborArray scratch; private final VectorSimilarityFunction similarityFunction; @@ -84,9 +88,12 @@ public final class HnswGraphBuilder { } this.maxConn = maxConn; this.beamWidth = beamWidth; - this.hnsw = new HnswGraph(maxConn); + // normalization factor for level generation; currently not configurable + this.ml = 1 / Math.log(1.0 * maxConn); + this.random = new SplittableRandom(seed); + int levelOfFirstNode = getRandomGraphLevel(ml, random); + this.hnsw = new HnswGraph(maxConn, levelOfFirstNode); bound = BoundsChecker.create(similarityFunction.reversed); - random = new SplittableRandom(seed); scratch = new NeighborArray(Math.max(beamWidth, maxConn + 1)); } @@ -109,43 +116,58 @@ public final class HnswGraphBuilder { long start = System.nanoTime(), t = start; // start at node 1! node 0 is added implicitly, in the constructor for (int node = 1; node < vectors.size(); node++) { - addGraphNode(vectors.vectorValue(node)); - if (node % 10000 == 0) { - if (infoStream.isEnabled(HNSW_COMPONENT)) { - long now = System.nanoTime(); - infoStream.message( - HNSW_COMPONENT, - String.format( - Locale.ROOT, - "built %d in %d/%d ms", - node, - ((now - t) / 1_000_000), - ((now - start) / 1_000_000))); - t = now; - } + addGraphNode(node, vectors.vectorValue(node)); + if ((node % 10000 == 0) && infoStream.isEnabled(HNSW_COMPONENT)) { + t = printGraphBuildStatus(node, start, t); } } return hnsw; } + /** Set info-stream to output debugging information * */ public void setInfoStream(InfoStream infoStream) { this.infoStream = infoStream; } /** Inserts a doc with vector value to the graph */ - void addGraphNode(float[] value) throws IOException { - // We pass 'null' for acceptOrds because there are no deletions while building the graph - NeighborQueue candidates = - HnswGraph.search( - value, beamWidth, beamWidth, vectorValues, similarityFunction, hnsw, null, random); + void addGraphNode(int node, float[] value) throws IOException { + NeighborQueue candidates; + final int nodeLevel = getRandomGraphLevel(ml, random); + int curMaxLevel = hnsw.numLevels() - 1; + int[] eps = new int[] {hnsw.entryNode()}; - int node = hnsw.addNode(); + // if a node introduces new levels to the graph, add this new node on new levels + for (int level = nodeLevel; level > curMaxLevel; level--) { + hnsw.addNode(level, node); + } + // for levels > nodeLevel search with topk = 1 + for (int level = curMaxLevel; level > nodeLevel; level--) { + candidates = + HnswGraph.searchLevel(value, 1, level, eps, vectorValues, similarityFunction, hnsw, null); + eps = new int[] {candidates.pop()}; + } + // for levels <= nodeLevel search with topk = beamWidth, and add connections + for (int level = Math.min(nodeLevel, curMaxLevel); level >= 0; level--) { + candidates = + HnswGraph.searchLevel( + value, beamWidth, level, eps, vectorValues, similarityFunction, hnsw, null); + eps = candidates.nodes(); + hnsw.addNode(level, node); + addDiverseNeighbors(level, node, candidates); + } + } - /* connect neighbors to the new node, using a diversity heuristic that chooses successive - * nearest neighbors that are closer to the new node than they are to the previously-selected - * neighbors - */ - addDiverseNeighbors(node, candidates); + private long printGraphBuildStatus(int node, long start, long t) { + long now = System.nanoTime(); + infoStream.message( + HNSW_COMPONENT, + String.format( + Locale.ROOT, + "built %d in %d/%d ms", + node, + ((now - t) / 1_000_000), + ((now - start) / 1_000_000))); + return now; } /* TODO: we are not maintaining nodes in strict score order; the forward links @@ -153,12 +175,13 @@ public final class HnswGraphBuilder { * work better if we keep the neighbor arrays sorted. Possibly we should switch back to a heap? * But first we should just see if sorting makes a significant difference. */ - private void addDiverseNeighbors(int node, NeighborQueue candidates) throws IOException { + private void addDiverseNeighbors(int level, int node, NeighborQueue candidates) + throws IOException { /* For each of the beamWidth nearest candidates (going from best to worst), select it only if it * is closer to target than it is to any of the already-selected neighbors (ie selected in this method, * since the node is new and has no prior neighbors). */ - NeighborArray neighbors = hnsw.getNeighbors(node); + NeighborArray neighbors = hnsw.getNeighbors(level, node); assert neighbors.size() == 0; // new node popToScratch(candidates); selectDiverse(neighbors, scratch); @@ -168,7 +191,7 @@ public final class HnswGraphBuilder { int size = neighbors.size(); for (int i = 0; i < size; i++) { int nbr = neighbors.node[i]; - NeighborArray nbrNbr = hnsw.getNeighbors(nbr); + NeighborArray nbrNbr = hnsw.getNeighbors(level, nbr); nbrNbr.add(node, neighbors.score[i]); if (nbrNbr.size() > maxConn) { diversityUpdate(nbrNbr); @@ -266,4 +289,12 @@ public final class HnswGraphBuilder { } return -1; } + + private static int getRandomGraphLevel(double ml, SplittableRandom random) { + double randDouble; + do { + randDouble = random.nextDouble(); // avoid 0 value, as log(0) is undefined + } while (randDouble == 0.0); + return ((int) (-log(randDouble) * ml)); + } } diff --git a/lucene/core/src/java/org/apache/lucene/util/hnsw/NeighborArray.java b/lucene/core/src/java/org/apache/lucene/util/hnsw/NeighborArray.java index 9deaa64113c..40125750309 100644 --- a/lucene/core/src/java/org/apache/lucene/util/hnsw/NeighborArray.java +++ b/lucene/core/src/java/org/apache/lucene/util/hnsw/NeighborArray.java @@ -32,7 +32,7 @@ public class NeighborArray { float[] score; int[] node; - NeighborArray(int maxSize) { + public NeighborArray(int maxSize) { node = new int[maxSize]; score = new float[maxSize]; } @@ -60,11 +60,15 @@ public class NeighborArray { return node; } + public float[] score() { + return score; + } + public void clear() { size = 0; } - void removeLast() { + public void removeLast() { size--; } diff --git a/lucene/core/src/java/org/apache/lucene/util/hnsw/NeighborQueue.java b/lucene/core/src/java/org/apache/lucene/util/hnsw/NeighborQueue.java index 0d2b94d1c5f..6f08a7bf329 100644 --- a/lucene/core/src/java/org/apache/lucene/util/hnsw/NeighborQueue.java +++ b/lucene/core/src/java/org/apache/lucene/util/hnsw/NeighborQueue.java @@ -54,7 +54,7 @@ public class NeighborQueue { // Used to track the number of neighbors visited during a single graph traversal private int visitedCount; - NeighborQueue(int initialSize, boolean reversed) { + public NeighborQueue(int initialSize, boolean reversed) { this.heap = new LongHeap(initialSize); this.order = reversed ? Order.REVERSED : Order.NATURAL; } @@ -119,7 +119,7 @@ public class NeighborQueue { return visitedCount; } - void setVisitedCount(int visitedCount) { + public void setVisitedCount(int visitedCount) { this.visitedCount = visitedCount; } diff --git a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec b/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec index 7eec415dffa..6e977e42f1e 100644 --- a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec +++ b/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec @@ -13,4 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.lucene.codecs.lucene90.Lucene90Codec +org.apache.lucene.codecs.lucene91.Lucene91Codec diff --git a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.KnnVectorsFormat b/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.KnnVectorsFormat index e7158f195c1..692145891fb 100644 --- a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.KnnVectorsFormat +++ b/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.KnnVectorsFormat @@ -13,4 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.lucene.codecs.lucene90.Lucene90HnswVectorsFormat \ No newline at end of file +org.apache.lucene.codecs.lucene91.Lucene91HnswVectorsFormat diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90StoredFieldsFormatHighCompression.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90StoredFieldsFormatHighCompression.java index a1cb543bc05..6b747ea00a1 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90StoredFieldsFormatHighCompression.java +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90StoredFieldsFormatHighCompression.java @@ -18,7 +18,8 @@ package org.apache.lucene.codecs.lucene90; import com.carrotsearch.randomizedtesting.generators.RandomPicks; import org.apache.lucene.codecs.Codec; -import org.apache.lucene.codecs.lucene90.Lucene90Codec.Mode; +import org.apache.lucene.codecs.lucene91.Lucene91Codec; +import org.apache.lucene.codecs.lucene91.Lucene91Codec.Mode; import org.apache.lucene.document.Document; import org.apache.lucene.document.StoredField; import org.apache.lucene.index.DirectoryReader; @@ -30,7 +31,7 @@ import org.apache.lucene.tests.index.BaseStoredFieldsFormatTestCase; public class TestLucene90StoredFieldsFormatHighCompression extends BaseStoredFieldsFormatTestCase { @Override protected Codec getCodec() { - return new Lucene90Codec(Mode.BEST_COMPRESSION); + return new Lucene91Codec(Mode.BEST_COMPRESSION); } /** @@ -40,7 +41,7 @@ public class TestLucene90StoredFieldsFormatHighCompression extends BaseStoredFie Directory dir = newDirectory(); for (int i = 0; i < 10; i++) { IndexWriterConfig iwc = newIndexWriterConfig(); - iwc.setCodec(new Lucene90Codec(RandomPicks.randomFrom(random(), Mode.values()))); + iwc.setCodec(new Lucene91Codec(RandomPicks.randomFrom(random(), Mode.values()))); IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig()); Document doc = new Document(); doc.add(new StoredField("field1", "value1")); @@ -69,7 +70,7 @@ public class TestLucene90StoredFieldsFormatHighCompression extends BaseStoredFie expectThrows( NullPointerException.class, () -> { - new Lucene90Codec(null); + new Lucene91Codec(null); }); expectThrows( diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene91/TestLucene91HnswVectorsFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene91/TestLucene91HnswVectorsFormat.java new file mode 100644 index 00000000000..fe828f018cb --- /dev/null +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene91/TestLucene91HnswVectorsFormat.java @@ -0,0 +1,54 @@ +/* + * 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.lucene91; + +import static com.carrotsearch.randomizedtesting.RandomizedTest.randomIntBetween; +import static org.apache.lucene.codecs.lucene91.Lucene91HnswVectorsFormat.DEFAULT_BEAM_WIDTH; +import static org.apache.lucene.codecs.lucene91.Lucene91HnswVectorsFormat.DEFAULT_MAX_CONN; + +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.codecs.KnnVectorsFormat; +import org.apache.lucene.tests.index.BaseKnnVectorsFormatTestCase; +import org.apache.lucene.tests.util.TestUtil; + +public class TestLucene91HnswVectorsFormat extends BaseKnnVectorsFormatTestCase { + @Override + protected Codec getCodec() { + return TestUtil.getDefaultCodec(); + } + + public void testToString() { + int maxConn = randomIntBetween(DEFAULT_MAX_CONN - 10, DEFAULT_MAX_CONN + 10); + int beamWidth = randomIntBetween(DEFAULT_BEAM_WIDTH - 50, DEFAULT_BEAM_WIDTH + 50); + Codec customCodec = + new Lucene91Codec() { + @Override + public KnnVectorsFormat getKnnVectorsFormatForField(String field) { + return new Lucene91HnswVectorsFormat(maxConn, beamWidth); + } + }; + String expectedString = + "Lucene91HnswVectorsFormat(name = Lucene91HnswVectorsFormat, maxConn = " + + maxConn + + ", beamWidth=" + + beamWidth + + ")"; + assertEquals( + expectedString, + ((Lucene91Codec) customCodec).getKnnVectorsFormatForField("bogus_field").toString()); + } +} diff --git a/lucene/core/src/test/org/apache/lucene/index/TestKnnGraph.java b/lucene/core/src/test/org/apache/lucene/index/TestKnnGraph.java index 46b6b6d920b..e2366aab1fb 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestKnnGraph.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestKnnGraph.java @@ -16,6 +16,7 @@ */ package org.apache.lucene.index; +import static com.carrotsearch.randomizedtesting.RandomizedTest.randomIntBetween; import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; import static org.apache.lucene.util.hnsw.HnswGraphBuilder.randSeed; @@ -26,11 +27,12 @@ import java.util.HashSet; import java.util.LinkedList; import java.util.List; import java.util.Set; +import java.util.concurrent.CountDownLatch; import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.KnnVectorsFormat; -import org.apache.lucene.codecs.lucene90.Lucene90Codec; -import org.apache.lucene.codecs.lucene90.Lucene90HnswVectorsFormat; -import org.apache.lucene.codecs.lucene90.Lucene90HnswVectorsReader; +import org.apache.lucene.codecs.lucene91.Lucene91Codec; +import org.apache.lucene.codecs.lucene91.Lucene91HnswVectorsFormat; +import org.apache.lucene.codecs.lucene91.Lucene91HnswVectorsReader; import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; @@ -38,13 +40,19 @@ import org.apache.lucene.document.FieldType; import org.apache.lucene.document.KnnVectorField; import org.apache.lucene.document.SortedDocValuesField; import org.apache.lucene.document.StringField; +import org.apache.lucene.index.KnnGraphValues.NodesIterator; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.KnnVectorQuery; import org.apache.lucene.search.ScoreDoc; +import org.apache.lucene.search.SearcherFactory; +import org.apache.lucene.search.SearcherManager; import org.apache.lucene.search.TopDocs; import org.apache.lucene.store.Directory; import org.apache.lucene.tests.util.LuceneTestCase; import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.Bits; import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.VectorUtil; import org.apache.lucene.util.hnsw.HnswGraphBuilder; import org.junit.After; @@ -55,7 +63,7 @@ public class TestKnnGraph extends LuceneTestCase { private static final String KNN_GRAPH_FIELD = "vector"; - private static int maxConn = Lucene90HnswVectorsFormat.DEFAULT_MAX_CONN; + private static int maxConn = Lucene91HnswVectorsFormat.DEFAULT_MAX_CONN; private Codec codec; private VectorSimilarityFunction similarityFunction; @@ -68,11 +76,11 @@ public class TestKnnGraph extends LuceneTestCase { } codec = - new Lucene90Codec() { + new Lucene91Codec() { @Override public KnnVectorsFormat getKnnVectorsFormatForField(String field) { - return new Lucene90HnswVectorsFormat( - maxConn, Lucene90HnswVectorsFormat.DEFAULT_BEAM_WIDTH); + return new Lucene91HnswVectorsFormat( + maxConn, Lucene91HnswVectorsFormat.DEFAULT_BEAM_WIDTH); } }; @@ -82,7 +90,7 @@ public class TestKnnGraph extends LuceneTestCase { @After public void cleanup() { - maxConn = Lucene90HnswVectorsFormat.DEFAULT_MAX_CONN; + maxConn = Lucene91HnswVectorsFormat.DEFAULT_MAX_CONN; } /** Basic test of creating documents in a graph */ @@ -153,21 +161,63 @@ public class TestKnnGraph extends LuceneTestCase { int dimension = atLeast(10); float[][] values = randomVectors(numDoc, dimension); int mergePoint = random().nextInt(numDoc); - int[][] mergedGraph = getIndexedGraph(values, mergePoint, seed); - int[][] singleSegmentGraph = getIndexedGraph(values, -1, seed); + int[][][] mergedGraph = getIndexedGraph(values, mergePoint, seed); + int[][][] singleSegmentGraph = getIndexedGraph(values, -1, seed); assertGraphEquals(singleSegmentGraph, mergedGraph); } - private void assertGraphEquals(int[][] expected, int[][] actual) { - assertEquals("graph sizes differ", expected.length, actual.length); - for (int i = 0; i < expected.length; i++) { - assertArrayEquals("difference at ord=" + i, expected[i], actual[i]); + /** Test writing and reading of multiple vector fields * */ + public void testMultipleVectorFields() throws Exception { + int numVectorFields = randomIntBetween(2, 5); + int numDoc = atLeast(100); + int[] dims = new int[numVectorFields]; + float[][][] values = new float[numVectorFields][][]; + for (int field = 0; field < numVectorFields; field++) { + dims[field] = atLeast(3); + values[field] = randomVectors(numDoc, dims[field]); + } + + try (Directory dir = newDirectory(); + IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig(null).setCodec(codec))) { + for (int docID = 0; docID < numDoc; docID++) { + Document doc = new Document(); + for (int field = 0; field < numVectorFields; field++) { + float[] vector = values[field][docID]; + if (vector != null) { + FieldType fieldType = KnnVectorField.createFieldType(vector.length, similarityFunction); + doc.add(new KnnVectorField(KNN_GRAPH_FIELD + field, vector, fieldType)); + } + } + String idString = Integer.toString(docID); + doc.add(new StringField("id", idString, Field.Store.YES)); + iw.addDocument(doc); + } + for (int field = 0; field < numVectorFields; field++) { + assertConsistentGraph(iw, values[field], KNN_GRAPH_FIELD + field); + } } } - private int[][] getIndexedGraph(float[][] values, int mergePoint, long seed) throws IOException { + private void assertGraphEquals(int[][][] expected, int[][][] actual) { + assertEquals("graph sizes differ", expected.length, actual.length); + for (int level = 0; level < expected.length; level++) { + for (int node = 0; node < expected[level].length; node++) { + assertArrayEquals("difference at ord=" + node, expected[level][node], actual[level][node]); + } + } + } + + /** + * Return a naive representation of an HNSW graph as a 3 dimensional array: 1st dim represents a + * graph layer. Each layer contains an array of arrays – a list of nodes and for each node a list + * of the node's neighbours. 2nd dim represents a node on a layer, and contains the node's + * neighbourhood, or {@code null} if a node is not present on this layer. 3rd dim represents + * neighbours of a node. + */ + private int[][][] getIndexedGraph(float[][] values, int mergePoint, long seed) + throws IOException { HnswGraphBuilder.randSeed = seed; - int[][] graph; + int[][][] graph; try (Directory dir = newDirectory()) { IndexWriterConfig iwc = newIndexWriterConfig(); iwc.setMergePolicy(new LogDocMergePolicy()); // for predictable segment ordering when merging @@ -186,8 +236,8 @@ public class TestKnnGraph extends LuceneTestCase { PerFieldKnnVectorsFormat.FieldsReader perFieldReader = (PerFieldKnnVectorsFormat.FieldsReader) ((CodecReader) getOnlyLeafReader(reader)).getVectorReader(); - Lucene90HnswVectorsReader vectorReader = - (Lucene90HnswVectorsReader) perFieldReader.getFieldReader(KNN_GRAPH_FIELD); + Lucene91HnswVectorsReader vectorReader = + (Lucene91HnswVectorsReader) perFieldReader.getFieldReader(KNN_GRAPH_FIELD); graph = copyGraph(vectorReader.getGraphValues(KNN_GRAPH_FIELD)); } } @@ -208,18 +258,23 @@ public class TestKnnGraph extends LuceneTestCase { return values; } - int[][] copyGraph(KnnGraphValues values) throws IOException { - int size = values.size(); - int[][] graph = new int[size][]; + int[][][] copyGraph(KnnGraphValues graphValues) throws IOException { + int[][][] graph = new int[graphValues.numLevels()][][]; + int size = graphValues.size(); int[] scratch = new int[maxConn]; - for (int node = 0; node < size; node++) { - int n, count = 0; - values.seek(node); - while ((n = values.nextNeighbor()) != NO_MORE_DOCS) { - scratch[count++] = n; - // graph[node][i++] = n; + + for (int level = 0; level < graphValues.numLevels(); level++) { + NodesIterator nodesItr = graphValues.getNodesOnLevel(level); + graph[level] = new int[size][]; + while (nodesItr.hasNext()) { + int node = nodesItr.nextInt(); + graphValues.seek(level, node); + int n, count = 0; + while ((n = graphValues.nextNeighbor()) != NO_MORE_DOCS) { + scratch[count++] = n; + } + graph[level][node] = ArrayUtil.copyOfSubArray(scratch, 0, count); } - graph[node] = ArrayUtil.copyOfSubArray(scratch, 0, count); } return graph; } @@ -232,31 +287,7 @@ public class TestKnnGraph extends LuceneTestCase { config.setCodec(codec); // test is not compatible with simpletext try (Directory dir = newDirectory(); IndexWriter iw = new IndexWriter(dir, config)) { - // Add a document for every cartesian point in an NxN square so we can - // easily know which are the nearest neighbors to every point. Insert by iterating - // using a prime number that is not a divisor of N*N so that we will hit each point once, - // and chosen so that points will be inserted in a deterministic - // but somewhat distributed pattern - int n = 5, stepSize = 17; - float[][] values = new float[n * n][]; - int index = 0; - for (int i = 0; i < values.length; i++) { - // System.out.printf("%d: (%d, %d)\n", i, index % n, index / n); - int x = index % n, y = index / n; - values[i] = new float[] {x, y}; - index = (index + stepSize) % (n * n); - add(iw, i, values[i]); - if (i == 13) { - // create 2 segments - iw.commit(); - } - } - boolean forceMerge = random().nextBoolean(); - // System.out.println(""); - if (forceMerge) { - iw.forceMerge(1); - } - assertConsistentGraph(iw, values); + indexData(iw); try (DirectoryReader dr = DirectoryReader.open(iw)) { // results are ordered by score (descending) and docid (ascending); // This is the insertion order: @@ -279,6 +310,77 @@ public class TestKnnGraph extends LuceneTestCase { } } + private void indexData(IndexWriter iw) throws IOException { + // Add a document for every cartesian point in an NxN square so we can + // easily know which are the nearest neighbors to every point. Insert by iterating + // using a prime number that is not a divisor of N*N so that we will hit each point once, + // and chosen so that points will be inserted in a deterministic + // but somewhat distributed pattern + int n = 5, stepSize = 17; + float[][] values = new float[n * n][]; + int index = 0; + for (int i = 0; i < values.length; i++) { + // System.out.printf("%d: (%d, %d)\n", i, index % n, index / n); + int x = index % n, y = index / n; + values[i] = new float[] {x, y}; + index = (index + stepSize) % (n * n); + add(iw, i, values[i]); + if (i == 13) { + // create 2 segments + iw.commit(); + } + } + boolean forceMerge = random().nextBoolean(); + if (forceMerge) { + iw.forceMerge(1); + } + assertConsistentGraph(iw, values); + } + + public void testMultiThreadedSearch() throws Exception { + similarityFunction = VectorSimilarityFunction.EUCLIDEAN; + IndexWriterConfig config = newIndexWriterConfig(); + config.setCodec(codec); + Directory dir = newDirectory(); + IndexWriter iw = new IndexWriter(dir, config); + indexData(iw); + + final SearcherManager manager = new SearcherManager(iw, new SearcherFactory()); + Thread[] threads = new Thread[randomIntBetween(2, 5)]; + final CountDownLatch latch = new CountDownLatch(1); + for (int i = 0; i < threads.length; i++) { + threads[i] = + new Thread( + () -> { + try { + latch.await(); + IndexSearcher searcher = manager.acquire(); + try { + KnnVectorQuery query = new KnnVectorQuery("vector", new float[] {0f, 0.1f}, 5); + TopDocs results = searcher.search(query, 5); + for (ScoreDoc doc : results.scoreDocs) { + // map docId to insertion id + doc.doc = + Integer.parseInt(searcher.getIndexReader().document(doc.doc).get("id")); + } + assertResults(new int[] {0, 15, 3, 18, 5}, results); + } finally { + manager.release(searcher); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + threads[i].start(); + } + + latch.countDown(); + for (Thread t : threads) { + t.join(); + } + IOUtils.close(manager, iw, dir); + } + private void assertGraphSearch(int[] expected, float[] vector, IndexReader reader) throws IOException { TopDocs results = doKnnSearch(reader, vector, 5); @@ -310,39 +412,40 @@ public class TestKnnGraph extends LuceneTestCase { } } + private void assertConsistentGraph(IndexWriter iw, float[][] values) throws IOException { + assertConsistentGraph(iw, values, KNN_GRAPH_FIELD); + } + // For each leaf, verify that its graph nodes are 1-1 with vectors, that the vectors are the - // expected values, - // and that the graph is fully connected and symmetric. + // expected values, and that the graph is fully connected and symmetric. // NOTE: when we impose max-fanout on the graph it wil no longer be symmetric, but should still // be fully connected. Is there any other invariant we can test? Well, we can check that max - // fanout - // is respected. We can test *desirable* properties of the graph like small-world (the graph - // diameter - // should be tightly bounded). - private void assertConsistentGraph(IndexWriter iw, float[][] values) throws IOException { - int totalGraphDocs = 0; + // fanout is respected. We can test *desirable* properties of the graph like small-world + // (the graph diameter should be tightly bounded). + private void assertConsistentGraph(IndexWriter iw, float[][] values, String vectorField) + throws IOException { + int numDocsWithVectors = 0; try (DirectoryReader dr = DirectoryReader.open(iw)) { for (LeafReaderContext ctx : dr.leaves()) { LeafReader reader = ctx.reader(); - VectorValues vectorValues = reader.getVectorValues(KNN_GRAPH_FIELD); + VectorValues vectorValues = reader.getVectorValues(vectorField); PerFieldKnnVectorsFormat.FieldsReader perFieldReader = (PerFieldKnnVectorsFormat.FieldsReader) ((CodecReader) reader).getVectorReader(); if (perFieldReader == null) { continue; } - Lucene90HnswVectorsReader vectorReader = - (Lucene90HnswVectorsReader) perFieldReader.getFieldReader(KNN_GRAPH_FIELD); - KnnGraphValues graphValues = vectorReader.getGraphValues(KNN_GRAPH_FIELD); - assertEquals((vectorValues == null), (graphValues == null)); + Lucene91HnswVectorsReader vectorReader = + (Lucene91HnswVectorsReader) perFieldReader.getFieldReader(vectorField); + KnnGraphValues graphValues = vectorReader.getGraphValues(vectorField); if (vectorValues == null) { + assert graphValues == null; continue; } - int[][] graph = new int[reader.maxDoc()][]; - boolean foundOrphan = false; - int graphSize = 0; + + // assert vector values: + // stored vector values are the same as original for (int i = 0; i < reader.maxDoc(); i++) { int nextDocWithVectors = vectorValues.advance(i); - // System.out.println("advanced to " + nextDocWithVectors); while (i < nextDocWithVectors && i < reader.maxDoc()) { int id = Integer.parseInt(reader.document(i).get("id")); assertNull("document " + id + " has no vector, but was expected to", values[id]); @@ -352,7 +455,6 @@ public class TestKnnGraph extends LuceneTestCase { break; } int id = Integer.parseInt(reader.document(i).get("id")); - graphValues.seek(graphSize); // documents with KnnGraphValues have the expected vectors float[] scratch = vectorValues.vectorValue(); assertArrayEquals( @@ -360,54 +462,71 @@ public class TestKnnGraph extends LuceneTestCase { values[id], scratch, 0f); - // We collect neighbors for analysis below - List friends = new ArrayList<>(); - int arc; - while ((arc = graphValues.nextNeighbor()) != NO_MORE_DOCS) { - friends.add(arc); - } - if (friends.size() == 0) { - // System.out.printf("knngraph @%d is singleton (advance returns %d)\n", i, - // nextWithNeighbors); - foundOrphan = true; - } else { - // NOTE: these friends are dense ordinals, not docIds. - int[] friendCopy = new int[friends.size()]; - for (int j = 0; j < friends.size(); j++) { - friendCopy[j] = friends.get(j); - } - graph[graphSize] = friendCopy; - // System.out.printf("knngraph @%d => %s\n", i, Arrays.toString(graph[i])); - } - graphSize++; + numDocsWithVectors++; } assertEquals(NO_MORE_DOCS, vectorValues.nextDoc()); - if (foundOrphan) { - assertEquals("graph is not fully connected", 1, graphSize); - } else { - assertTrue( - "Graph has " + graphSize + " nodes, but one of them has no neighbors", graphSize > 1); + + // assert graph values: + // For each level of the graph assert that: + // 1. There are no orphan nodes without any friends + // 2. If orphans are found, than the level must contain only 0 or a single node + // 3. If the number of nodes on the level doesn't exceed maxConn, assert that the graph is + // fully connected, i.e. any node is reachable from any other node. + // 4. If the number of nodes on the level exceeds maxConn, assert that maxConn is respected. + for (int level = 0; level < graphValues.numLevels(); level++) { + int[][] graphOnLevel = new int[graphValues.size()][]; + int countOnLevel = 0; + boolean foundOrphan = false; + NodesIterator nodesItr = graphValues.getNodesOnLevel(level); + while (nodesItr.hasNext()) { + int node = nodesItr.nextInt(); + graphValues.seek(level, node); + int arc; + List friends = new ArrayList<>(); + while ((arc = graphValues.nextNeighbor()) != NO_MORE_DOCS) { + friends.add(arc); + } + if (friends.size() == 0) { + foundOrphan = true; + } else { + int[] friendsCopy = new int[friends.size()]; + Arrays.setAll(friendsCopy, friends::get); + graphOnLevel[node] = friendsCopy; + } + countOnLevel++; + } + // System.out.println("Level[" + level + "] has [" + nodesCount + "] nodes."); + assertEquals(nodesItr.size(), countOnLevel); + assertFalse("No nodes on level [" + level + "]", countOnLevel == 0); + if (countOnLevel == 1) { + assertTrue( + "Graph with 1 node has unexpected neighbors on level [" + level + "]", foundOrphan); + } else { + assertFalse( + "Graph has orphan nodes with no friends on level [" + level + "]", foundOrphan); + if (maxConn > countOnLevel) { + // assert that the graph is fully connected, + // i.e. any node can be reached from any other node + assertConnected(graphOnLevel); + } else { + // assert that max-connections was respected + assertMaxConn(graphOnLevel, maxConn); + } + } } - if (maxConn > graphSize) { - // assert that the graph in each leaf is connected - assertConnected(graph); - } else { - // assert that max-connections was respected - assertMaxConn(graph, maxConn); - } - totalGraphDocs += graphSize; } } - int expectedCount = 0; - for (float[] friends : values) { - if (friends != null) { - ++expectedCount; + + int expectedNumDocsWithVectors = 0; + for (float[] value : values) { + if (value != null) { + ++expectedNumDocsWithVectors; } } - assertEquals(expectedCount, totalGraphDocs); + assertEquals(expectedNumDocsWithVectors, numDocsWithVectors); } - private void assertMaxConn(int[][] graph, int maxConn) { + public static void assertMaxConn(int[][] graph, int maxConn) { for (int[] ints : graph) { if (ints != null) { assert (ints.length <= maxConn); @@ -418,37 +537,36 @@ public class TestKnnGraph extends LuceneTestCase { } } - private void assertConnected(int[][] graph) { - // every node in the graph is reachable from every other node + /** Assert that every node is reachable from some other node */ + private static void assertConnected(int[][] graph) { + List nodes = new ArrayList<>(); Set visited = new HashSet<>(); List queue = new LinkedList<>(); - int count = 0; - for (int[] entry : graph) { - if (entry != null) { - if (queue.isEmpty()) { - queue.add(entry[0]); // start from any node - // System.out.println("start at " + entry[0]); - } - ++count; + for (int i = 0; i < graph.length; i++) { + if (graph[i] != null) { + nodes.add(i); } } + + // start from any node + int startIdx = random().nextInt(nodes.size()); + queue.add(nodes.get(startIdx)); while (queue.isEmpty() == false) { int i = queue.remove(0); assertNotNull("expected neighbors of " + i, graph[i]); visited.add(i); for (int j : graph[i]) { if (visited.contains(j) == false) { - // System.out.println(" ... " + j); queue.add(j); } } } - for (int i = 0; i < count; i++) { - assertTrue("Attempted to walk entire graph but never visited " + i, visited.contains(i)); + // assert that every node is reachable from some other node as it was visited + for (int node : nodes) { + assertTrue( + "Attempted to walk entire graph but never visited node [" + node + "]", + visited.contains(node)); } - // we visited each node exactly once - assertEquals( - "Attempted to walk entire graph but only visited " + visited.size(), count, visited.size()); } private void add(IndexWriter iw, int id, float[] vector) throws IOException { diff --git a/lucene/core/src/test/org/apache/lucene/util/hnsw/KnnGraphTester.java b/lucene/core/src/test/org/apache/lucene/util/hnsw/KnnGraphTester.java index 339d8f8fecf..be600ae40c5 100644 --- a/lucene/core/src/test/org/apache/lucene/util/hnsw/KnnGraphTester.java +++ b/lucene/core/src/test/org/apache/lucene/util/hnsw/KnnGraphTester.java @@ -37,9 +37,9 @@ import java.util.Locale; import java.util.Set; import org.apache.lucene.codecs.KnnVectorsFormat; import org.apache.lucene.codecs.KnnVectorsReader; -import org.apache.lucene.codecs.lucene90.Lucene90Codec; -import org.apache.lucene.codecs.lucene90.Lucene90HnswVectorsFormat; -import org.apache.lucene.codecs.lucene90.Lucene90HnswVectorsReader; +import org.apache.lucene.codecs.lucene91.Lucene91Codec; +import org.apache.lucene.codecs.lucene91.Lucene91HnswVectorsFormat; +import org.apache.lucene.codecs.lucene91.Lucene91HnswVectorsReader; import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat; import org.apache.lucene.document.Document; import org.apache.lucene.document.FieldType; @@ -253,7 +253,7 @@ public class KnnGraphTester { ((PerFieldKnnVectorsFormat.FieldsReader) ((CodecReader) leafReader).getVectorReader()) .getFieldReader(KNN_FIELD); KnnGraphValues knnValues = - ((Lucene90HnswVectorsReader) vectorsReader).getGraphValues(KNN_FIELD); + ((Lucene91HnswVectorsReader) vectorsReader).getGraphValues(KNN_FIELD); System.out.printf("Leaf %d has %d documents\n", context.ord, leafReader.maxDoc()); printGraphFanout(knnValues, leafReader.maxDoc()); } @@ -267,7 +267,7 @@ public class KnnGraphTester { new HnswGraphBuilder(vectors, similarityFunction, maxConn, beamWidth, 0); // start at node 1 for (int i = 1; i < numDocs; i++) { - builder.addGraphNode(values.vectorValue(i)); + builder.addGraphNode(i, values.vectorValue(i)); System.out.println("\nITERATION " + i); dumpGraph(builder.hnsw); } @@ -276,7 +276,7 @@ public class KnnGraphTester { private void dumpGraph(HnswGraph hnsw) { for (int i = 0; i < hnsw.size(); i++) { - NeighborArray neighbors = hnsw.getNeighbors(i); + NeighborArray neighbors = hnsw.getNeighbors(0, i); System.out.printf(Locale.ROOT, "%5d", i); NeighborArray sorted = new NeighborArray(neighbors.size()); for (int j = 0; j < neighbors.size(); j++) { @@ -308,7 +308,7 @@ public class KnnGraphTester { int count = 0; int[] leafHist = new int[numDocs]; for (int node = 0; node < numDocs; node++) { - knnValues.seek(node); + knnValues.seek(0, node); int n = 0; while (knnValues.nextNeighbor() != NO_MORE_DOCS) { ++n; @@ -580,10 +580,10 @@ public class KnnGraphTester { private int createIndex(Path docsPath, Path indexPath) throws IOException { IndexWriterConfig iwc = new IndexWriterConfig().setOpenMode(IndexWriterConfig.OpenMode.CREATE); iwc.setCodec( - new Lucene90Codec() { + new Lucene91Codec() { @Override public KnnVectorsFormat getKnnVectorsFormatForField(String field) { - return new Lucene90HnswVectorsFormat(maxConn, beamWidth); + return new Lucene91HnswVectorsFormat(maxConn, beamWidth); } }); // iwc.setMergePolicy(NoMergePolicy.INSTANCE); diff --git a/lucene/core/src/test/org/apache/lucene/util/hnsw/TestHnswGraph.java b/lucene/core/src/test/org/apache/lucene/util/hnsw/TestHnswGraph.java index 38bb50a3e97..c46f12d0db5 100644 --- a/lucene/core/src/test/org/apache/lucene/util/hnsw/TestHnswGraph.java +++ b/lucene/core/src/test/org/apache/lucene/util/hnsw/TestHnswGraph.java @@ -24,11 +24,10 @@ import java.util.Arrays; import java.util.HashSet; import java.util.Random; import java.util.Set; -import java.util.SplittableRandom; import org.apache.lucene.codecs.KnnVectorsFormat; -import org.apache.lucene.codecs.lucene90.Lucene90Codec; -import org.apache.lucene.codecs.lucene90.Lucene90HnswVectorsFormat; -import org.apache.lucene.codecs.lucene90.Lucene90HnswVectorsReader; +import org.apache.lucene.codecs.lucene91.Lucene91Codec; +import org.apache.lucene.codecs.lucene91.Lucene91HnswVectorsFormat; +import org.apache.lucene.codecs.lucene91.Lucene91HnswVectorsReader; import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat; import org.apache.lucene.document.Document; import org.apache.lucene.document.KnnVectorField; @@ -39,6 +38,7 @@ import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriterConfig; import org.apache.lucene.index.KnnGraphValues; +import org.apache.lucene.index.KnnGraphValues.NodesIterator; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.RandomAccessVectorValues; import org.apache.lucene.index.RandomAccessVectorValuesProducer; @@ -81,10 +81,10 @@ public class TestHnswGraph extends LuceneTestCase { IndexWriterConfig iwc = new IndexWriterConfig() .setCodec( - new Lucene90Codec() { + new Lucene91Codec() { @Override public KnnVectorsFormat getKnnVectorsFormatForField(String field) { - return new Lucene90HnswVectorsFormat(maxConn, beamWidth); + return new Lucene91HnswVectorsFormat(maxConn, beamWidth); } }); try (IndexWriter iw = new IndexWriter(dir, iwc)) { @@ -111,38 +111,64 @@ public class TestHnswGraph extends LuceneTestCase { assertEquals(indexedDoc, ctx.reader().numDocs()); assertVectorsEqual(v3, values); KnnGraphValues graphValues = - ((Lucene90HnswVectorsReader) + ((Lucene91HnswVectorsReader) ((PerFieldKnnVectorsFormat.FieldsReader) ((CodecReader) ctx.reader()).getVectorReader()) .getFieldReader("field")) .getGraphValues("field"); - assertGraphEqual(hnsw, graphValues, nVec); + assertGraphEqual(hnsw, graphValues); } } } } + private void assertGraphEqual(KnnGraphValues g, KnnGraphValues h) throws IOException { + assertEquals("the number of levels in the graphs are different!", g.numLevels(), h.numLevels()); + assertEquals("the number of nodes in the graphs are different!", g.size(), h.size()); + + // assert equal nodes on each level + for (int level = 0; level < g.numLevels(); level++) { + NodesIterator nodesOnLevel = g.getNodesOnLevel(level); + NodesIterator nodesOnLevel2 = h.getNodesOnLevel(level); + while (nodesOnLevel.hasNext() && nodesOnLevel2.hasNext()) { + int node = nodesOnLevel.nextInt(); + int node2 = nodesOnLevel2.nextInt(); + assertEquals("nodes in the graphs are different", node, node2); + } + } + + // assert equal nodes' neighbours on each level + for (int level = 0; level < g.numLevels(); level++) { + NodesIterator nodesOnLevel = g.getNodesOnLevel(level); + while (nodesOnLevel.hasNext()) { + int node = nodesOnLevel.nextInt(); + g.seek(level, node); + h.seek(level, node); + assertEquals("arcs differ for node " + node, getNeighborNodes(g), getNeighborNodes(h)); + } + } + } + // Make sure we actually approximately find the closest k elements. Mostly this is about // ensuring that we have all the distance functions, comparators, priority queues and so on // oriented in the right directions public void testAknnDiverse() throws IOException { + int maxConn = 10; int nDoc = 100; CircularVectorValues vectors = new CircularVectorValues(nDoc); HnswGraphBuilder builder = new HnswGraphBuilder( - vectors, VectorSimilarityFunction.DOT_PRODUCT, 16, 100, random().nextInt()); + vectors, VectorSimilarityFunction.DOT_PRODUCT, maxConn, 100, random().nextInt()); HnswGraph hnsw = builder.build(vectors); // run some searches NeighborQueue nn = HnswGraph.search( new float[] {1, 0}, 10, - 10, vectors.randomAccess(), VectorSimilarityFunction.DOT_PRODUCT, hnsw, - null, - new SplittableRandom(random().nextLong())); + null); int[] nodes = nn.nodes(); assertTrue("Number of found results is not equal to [10].", nodes.length == 10); @@ -155,7 +181,7 @@ public class TestHnswGraph extends LuceneTestCase { assertTrue("sum(result docs)=" + sum, sum < 75); for (int i = 0; i < nDoc; i++) { - NeighborArray neighbors = hnsw.getNeighbors(i); + NeighborArray neighbors = hnsw.getNeighbors(0, i); int[] nnodes = neighbors.node; for (int j = 0; j < neighbors.size(); j++) { // all neighbors should be valid node ids. @@ -166,24 +192,22 @@ public class TestHnswGraph extends LuceneTestCase { public void testSearchWithAcceptOrds() throws IOException { int nDoc = 100; + int maxConn = 16; CircularVectorValues vectors = new CircularVectorValues(nDoc); HnswGraphBuilder builder = new HnswGraphBuilder( - vectors, VectorSimilarityFunction.DOT_PRODUCT, 16, 100, random().nextInt()); + vectors, VectorSimilarityFunction.DOT_PRODUCT, maxConn, 100, random().nextInt()); HnswGraph hnsw = builder.build(vectors); - // the first 10 docs must not be deleted to ensure the expected recall Bits acceptOrds = createRandomAcceptOrds(10, vectors.size); NeighborQueue nn = HnswGraph.search( new float[] {1, 0}, 10, - 10, vectors.randomAccess(), VectorSimilarityFunction.DOT_PRODUCT, hnsw, - acceptOrds, - new SplittableRandom(random().nextLong())); + acceptOrds); int[] nodes = nn.nodes(); assertTrue("Number of found results is not equal to [10].", nodes.length == 10); int sum = 0; @@ -213,12 +237,10 @@ public class TestHnswGraph extends LuceneTestCase { HnswGraph.search( new float[] {1, 0}, 10, - 10, vectors.randomAccess(), VectorSimilarityFunction.EUCLIDEAN, hnsw, - acceptOrds, - new SplittableRandom(random().nextLong())); + acceptOrds); int[] nodes = nn.nodes(); assertTrue("Number of found results is not equal to [10].", nodes.length == 10); int sum = 0; @@ -295,46 +317,46 @@ public class TestHnswGraph extends LuceneTestCase { vectors, VectorSimilarityFunction.DOT_PRODUCT, 2, 10, random().nextInt()); // node 0 is added by the builder constructor // builder.addGraphNode(vectors.vectorValue(0)); - builder.addGraphNode(vectors.vectorValue(1)); - builder.addGraphNode(vectors.vectorValue(2)); + builder.addGraphNode(1, vectors.vectorValue(1)); + builder.addGraphNode(2, vectors.vectorValue(2)); // now every node has tried to attach every other node as a neighbor, but // some were excluded based on diversity check. - assertNeighbors(builder.hnsw, 0, 1, 2); - assertNeighbors(builder.hnsw, 1, 0); - assertNeighbors(builder.hnsw, 2, 0); + assertLevel0Neighbors(builder.hnsw, 0, 1, 2); + assertLevel0Neighbors(builder.hnsw, 1, 0); + assertLevel0Neighbors(builder.hnsw, 2, 0); - builder.addGraphNode(vectors.vectorValue(3)); - assertNeighbors(builder.hnsw, 0, 1, 2); + builder.addGraphNode(3, vectors.vectorValue(3)); + assertLevel0Neighbors(builder.hnsw, 0, 1, 2); // we added 3 here - assertNeighbors(builder.hnsw, 1, 0, 3); - assertNeighbors(builder.hnsw, 2, 0); - assertNeighbors(builder.hnsw, 3, 1); + assertLevel0Neighbors(builder.hnsw, 1, 0, 3); + assertLevel0Neighbors(builder.hnsw, 2, 0); + assertLevel0Neighbors(builder.hnsw, 3, 1); // supplant an existing neighbor - builder.addGraphNode(vectors.vectorValue(4)); + builder.addGraphNode(4, vectors.vectorValue(4)); // 4 is the same distance from 0 that 2 is; we leave the existing node in place - assertNeighbors(builder.hnsw, 0, 1, 2); + assertLevel0Neighbors(builder.hnsw, 0, 1, 2); // 4 is closer to 1 than either existing neighbor (0, 3). 3 fails diversity check with 4, so // replace it - assertNeighbors(builder.hnsw, 1, 0, 4); - assertNeighbors(builder.hnsw, 2, 0); + assertLevel0Neighbors(builder.hnsw, 1, 0, 4); + assertLevel0Neighbors(builder.hnsw, 2, 0); // 1 survives the diversity check - assertNeighbors(builder.hnsw, 3, 1, 4); - assertNeighbors(builder.hnsw, 4, 1, 3); + assertLevel0Neighbors(builder.hnsw, 3, 1, 4); + assertLevel0Neighbors(builder.hnsw, 4, 1, 3); - builder.addGraphNode(vectors.vectorValue(5)); - assertNeighbors(builder.hnsw, 0, 1, 2); - assertNeighbors(builder.hnsw, 1, 0, 5); - assertNeighbors(builder.hnsw, 2, 0); + builder.addGraphNode(5, vectors.vectorValue(5)); + assertLevel0Neighbors(builder.hnsw, 0, 1, 2); + assertLevel0Neighbors(builder.hnsw, 1, 0, 5); + assertLevel0Neighbors(builder.hnsw, 2, 0); // even though 5 is closer, 3 is not a neighbor of 5, so no update to *its* neighbors occurs - assertNeighbors(builder.hnsw, 3, 1, 4); - assertNeighbors(builder.hnsw, 4, 3, 5); - assertNeighbors(builder.hnsw, 5, 1, 4); + assertLevel0Neighbors(builder.hnsw, 3, 1, 4); + assertLevel0Neighbors(builder.hnsw, 4, 3, 5); + assertLevel0Neighbors(builder.hnsw, 5, 1, 4); } - private void assertNeighbors(HnswGraph graph, int node, int... expected) { + private void assertLevel0Neighbors(HnswGraph graph, int node, int... expected) { Arrays.sort(expected); - NeighborArray nn = graph.getNeighbors(node); + NeighborArray nn = graph.getNeighbors(0, node); int[] actual = ArrayUtil.copyOfSubArray(nn.node, 0, nn.size()); Arrays.sort(actual); assertArrayEquals( @@ -346,13 +368,14 @@ public class TestHnswGraph extends LuceneTestCase { public void testRandom() throws IOException { int size = atLeast(100); int dim = atLeast(10); - int topK = 5; + int maxConn = 10; RandomVectorValues vectors = new RandomVectorValues(size, dim, random()); VectorSimilarityFunction similarityFunction = VectorSimilarityFunction.values()[ random().nextInt(VectorSimilarityFunction.values().length - 1) + 1]; + int topK = 5; HnswGraphBuilder builder = - new HnswGraphBuilder(vectors, similarityFunction, 10, 30, random().nextLong()); + new HnswGraphBuilder(vectors, similarityFunction, maxConn, 30, random().nextLong()); HnswGraph hnsw = builder.build(vectors); Bits acceptOrds = random().nextBoolean() ? null : createRandomAcceptOrds(0, size); @@ -360,15 +383,7 @@ public class TestHnswGraph extends LuceneTestCase { for (int i = 0; i < 100; i++) { float[] query = randomVector(random(), dim); NeighborQueue actual = - HnswGraph.search( - query, - topK, - 100, - vectors, - similarityFunction, - hnsw, - acceptOrds, - new SplittableRandom(random().nextLong())); + HnswGraph.search(query, topK, vectors, similarityFunction, hnsw, acceptOrds); NeighborQueue expected = new NeighborQueue(topK, similarityFunction.reversed); for (int j = 0; j < size; j++) { if (vectors.vectorValue(j) != null && (acceptOrds == null || acceptOrds.get(j))) { @@ -383,7 +398,7 @@ public class TestHnswGraph extends LuceneTestCase { } double overlap = totalMatches / (double) (100 * topK); System.out.println("overlap=" + overlap + " totalMatches=" + totalMatches); - assertTrue("overlap=" + overlap, overlap > 0.9); + assertTrue("overlap=" + overlap, overlap > 0.8); } private int computeOverlap(int[] a, int[] b) { @@ -487,14 +502,6 @@ public class TestHnswGraph extends LuceneTestCase { return value; } - private void assertGraphEqual(KnnGraphValues g, KnnGraphValues h, int size) throws IOException { - for (int node = 0; node < size; node++) { - g.seek(node); - h.seek(node); - assertEquals("arcs differ for node " + node, getNeighborNodes(g), getNeighborNodes(h)); - } - } - private Set getNeighborNodes(KnnGraphValues g) throws IOException { Set neighbors = new HashSet<>(); for (int n = g.nextNeighbor(); n != NO_MORE_DOCS; n = g.nextNeighbor()) { diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java index 3220647289c..0fe44266361 100644 --- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java +++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java @@ -40,7 +40,7 @@ import org.apache.lucene.analysis.tokenattributes.PayloadAttribute; import org.apache.lucene.analysis.tokenattributes.TypeAttribute; import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.PostingsFormat; -import org.apache.lucene.codecs.lucene90.Lucene90Codec; +import org.apache.lucene.codecs.lucene91.Lucene91Codec; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; import org.apache.lucene.document.IntPoint; @@ -961,7 +961,7 @@ public class TestSuggestField extends LuceneTestCase { IndexWriterConfig iwc = newIndexWriterConfig(random(), analyzer); iwc.setMergePolicy(newLogMergePolicy()); Codec filterCodec = - new Lucene90Codec() { + new Lucene91Codec() { CompletionPostingsFormat.FSTLoadMode fstLoadMode = RandomPicks.randomFrom(random(), CompletionPostingsFormat.FSTLoadMode.values()); PostingsFormat postingsFormat = new Completion90PostingsFormat(fstLoadMode); diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestRuleSetupAndRestoreClassEnv.java b/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestRuleSetupAndRestoreClassEnv.java index 8d9767586f6..a7f9a7afdc6 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestRuleSetupAndRestoreClassEnv.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestRuleSetupAndRestoreClassEnv.java @@ -38,7 +38,7 @@ import java.util.TimeZone; import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.DocValuesFormat; import org.apache.lucene.codecs.PostingsFormat; -import org.apache.lucene.codecs.lucene90.Lucene90Codec; +import org.apache.lucene.codecs.lucene91.Lucene91Codec; import org.apache.lucene.codecs.simpletext.SimpleTextCodec; import org.apache.lucene.search.similarities.Similarity; import org.apache.lucene.tests.codecs.asserting.AssertingCodec; @@ -193,9 +193,9 @@ final class TestRuleSetupAndRestoreClassEnv extends AbstractBeforeAfterRule { } else if ("Compressing".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 6 && !shouldAvoidCodec("Compressing"))) { codec = CompressingCodec.randomInstance(random); - } else if ("Lucene90".equals(TEST_CODEC) - || ("random".equals(TEST_CODEC) && randomVal == 5 && !shouldAvoidCodec("Lucene90"))) { - codec = new Lucene90Codec(RandomPicks.randomFrom(random, Lucene90Codec.Mode.values())); + } else if ("Lucene91".equals(TEST_CODEC) + || ("random".equals(TEST_CODEC) && randomVal == 5 && !shouldAvoidCodec("Lucene91"))) { + codec = new Lucene91Codec(RandomPicks.randomFrom(random, Lucene91Codec.Mode.values())); } else if (!"random".equals(TEST_CODEC)) { codec = Codec.forName(TEST_CODEC); } else if ("random".equals(TEST_POSTINGSFORMAT)) { diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestUtil.java b/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestUtil.java index 68741d0b5ee..0d1c982e3cd 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestUtil.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestUtil.java @@ -53,10 +53,10 @@ import org.apache.lucene.codecs.DocValuesFormat; import org.apache.lucene.codecs.KnnVectorsFormat; import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.blocktreeords.BlockTreeOrdsPostingsFormat; -import org.apache.lucene.codecs.lucene90.Lucene90Codec; import org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat; -import org.apache.lucene.codecs.lucene90.Lucene90HnswVectorsFormat; import org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat; +import org.apache.lucene.codecs.lucene91.Lucene91Codec; +import org.apache.lucene.codecs.lucene91.Lucene91HnswVectorsFormat; import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat; import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat; import org.apache.lucene.document.BinaryDocValuesField; @@ -1236,7 +1236,7 @@ public final class TestUtil { * different than {@link Codec#getDefault()} because that is randomized. */ public static Codec getDefaultCodec() { - return new Lucene90Codec(); + return new Lucene91Codec(); } /** @@ -1322,7 +1322,7 @@ public final class TestUtil { * Lucene. */ public static KnnVectorsFormat getDefaultKnnVectorsFormat() { - return new Lucene90HnswVectorsFormat(); + return new Lucene91HnswVectorsFormat(); } public static boolean anyFilesExceptWriteLock(Directory dir) throws IOException {