Create Lucene94 Codec and move Lucene92 to backwards_codecs (#1041)

This commit is contained in:
Mayya Sharipova 2022-07-22 10:04:10 -04:00 committed by GitHub
parent 6bdeb141b7
commit bd360f9b3e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
32 changed files with 2188 additions and 81 deletions

View File

@ -31,6 +31,7 @@ module org.apache.lucene.backward_codecs {
exports org.apache.lucene.backward_codecs.lucene87;
exports org.apache.lucene.backward_codecs.lucene90;
exports org.apache.lucene.backward_codecs.lucene91;
exports org.apache.lucene.backward_codecs.lucene92;
exports org.apache.lucene.backward_codecs.packed;
exports org.apache.lucene.backward_codecs.store;
@ -41,12 +42,14 @@ module org.apache.lucene.backward_codecs {
org.apache.lucene.backward_codecs.lucene84.Lucene84PostingsFormat;
provides org.apache.lucene.codecs.KnnVectorsFormat with
org.apache.lucene.backward_codecs.lucene90.Lucene90HnswVectorsFormat,
org.apache.lucene.backward_codecs.lucene91.Lucene91HnswVectorsFormat;
org.apache.lucene.backward_codecs.lucene91.Lucene91HnswVectorsFormat,
org.apache.lucene.backward_codecs.lucene92.Lucene92HnswVectorsFormat;
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.lucene90.Lucene90Codec,
org.apache.lucene.backward_codecs.lucene91.Lucene91Codec;
org.apache.lucene.backward_codecs.lucene91.Lucene91Codec,
org.apache.lucene.backward_codecs.lucene92.Lucene92Codec;
}

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene92;
package org.apache.lucene.backward_codecs.lucene92;
import java.util.Objects;
import org.apache.lucene.codecs.Codec;
@ -49,7 +49,7 @@ import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
*
* <p>If you want to reuse functionality of this codec in another codec, extend {@link FilterCodec}.
*
* @see org.apache.lucene.codecs.lucene92 package documentation for file format details.
* @see org.apache.lucene.backward_codecs.lucene92 package documentation for file format details.
* @lucene.experimental
*/
public class Lucene92Codec extends Codec {
@ -164,7 +164,7 @@ public class Lucene92Codec extends Codec {
}
@Override
public final KnnVectorsFormat knnVectorsFormat() {
public KnnVectorsFormat knnVectorsFormat() {
return knnVectorsFormat;
}

View File

@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene92;
package org.apache.lucene.backward_codecs.lucene92;
import java.io.IOException;
import org.apache.lucene.codecs.KnnVectorsFormat;
@ -95,7 +95,15 @@ import org.apache.lucene.util.hnsw.HnswGraph;
*
* @lucene.experimental
*/
public final class Lucene92HnswVectorsFormat extends KnnVectorsFormat {
public class Lucene92HnswVectorsFormat extends KnnVectorsFormat {
/** 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;
static final String META_CODEC_NAME = "lucene92HnswVectorsFormatMeta";
static final String VECTOR_DATA_CODEC_NAME = "lucene92HnswVectorsFormatData";
@ -107,32 +115,24 @@ public final class Lucene92HnswVectorsFormat 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 during a graph construction.
*/
public static final int DEFAULT_BEAM_WIDTH = 100;
static final int DIRECT_MONOTONIC_BLOCK_SHIFT = 16;
/**
* Controls how many of the nearest neighbor candidates are connected to the new node. Defaults to
* {@link Lucene92HnswVectorsFormat#DEFAULT_MAX_CONN}. See {@link HnswGraph} for more details.
* {@link #maxConn}. 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 Lucene92HnswVectorsFormat#DEFAULT_BEAM_WIDTH}. See {@link
* HnswGraph} for details.
* node. Defaults to to {@link #DEFAULT_BEAM_WIDTH}. See {@link HnswGraph} for details.
*/
private final int beamWidth;
final int beamWidth;
/** A constructor for vectors format with default parameters */
public Lucene92HnswVectorsFormat() {
this(DEFAULT_MAX_CONN, DEFAULT_BEAM_WIDTH);
}
/** Constructs a format for reading old indexes */
public Lucene92HnswVectorsFormat(int maxConn, int beamWidth) {
super("lucene92HnswVectorsFormat");
this.maxConn = maxConn;
@ -141,7 +141,7 @@ public final class Lucene92HnswVectorsFormat extends KnnVectorsFormat {
@Override
public KnnVectorsWriter fieldsWriter(SegmentWriteState state) throws IOException {
return new Lucene92HnswVectorsWriter(state, maxConn, beamWidth);
throw new UnsupportedOperationException("Old codecs may only be used for reading");
}
@Override
@ -151,7 +151,7 @@ public final class Lucene92HnswVectorsFormat extends KnnVectorsFormat {
@Override
public String toString() {
return "lucene92HnswVectorsFormat(name = lucene92HnswVectorsFormat, maxConn = "
return "Lucene92HnswVectorsFormat(name = Lucene92HnswVectorsFormat, maxConn = "
+ maxConn
+ ", beamWidth="
+ beamWidth

View File

@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene92;
package org.apache.lucene.backward_codecs.lucene92;
import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;

View File

@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene92;
package org.apache.lucene.backward_codecs.lucene92;
import java.io.IOException;
import java.nio.ByteBuffer;

View File

@ -0,0 +1,422 @@
/*
* 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.2 file format.
*
* <h2>Apache Lucene - Index File Formats</h2>
*
* <div>
*
* <ul>
* <li><a href="#Introduction">Introduction</a>
* <li><a href="#Definitions">Definitions</a>
* <ul>
* <li><a href="#Inverted_Indexing">Inverted Indexing</a>
* <li><a href="#Types_of_Fields">Types of Fields</a>
* <li><a href="#Segments">Segments</a>
* <li><a href="#Document_Numbers">Document Numbers</a>
* </ul>
* <li><a href="#Overview">Index Structure Overview</a>
* <li><a href="#File_Naming">File Naming</a>
* <li><a href="#file-names">Summary of File Extensions</a>
* <ul>
* <li><a href="#Lock_File">Lock File</a>
* <li><a href="#History">History</a>
* <li><a href="#Limitations">Limitations</a>
* </ul>
* </ul>
*
* </div> <a id="Introduction"></a>
*
* <h3>Introduction</h3>
*
* <div>
*
* <p>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 <code>docs/</code> that was distributed
* with the version you are using.
*
* <p>This document attempts to provide a high-level definition of the Apache Lucene file formats.
* </div> <a id="Definitions"></a>
*
* <h3>Definitions</h3>
*
* <div>
*
* <p>The fundamental concepts in Lucene are index, document, field and term.
*
* <p>An index contains a sequence of documents.
*
* <ul>
* <li>A document is a sequence of fields.
* <li>A field is a named sequence of terms.
* <li>A term is a sequence of bytes.
* </ul>
*
* <p>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. <a
* id="Inverted_Indexing"></a>
*
* <h4>Inverted Indexing</h4>
*
* <p>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
* <i>inverted index.</i> 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. <a
* id="Types_of_Fields"></a>
*
* <h4>Types of Fields</h4>
*
* <p>In Lucene, fields may be <i>stored</i>, in which case their text is stored in the index
* literally, in a non-inverted manner. Fields that are inverted are called <i>indexed</i>. A field
* may be both stored and indexed.
*
* <p>The text of a field may be <i>tokenized</i> 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.
*
* <p>See the {@link org.apache.lucene.document.Field Field} java docs for more information on
* Fields. <a id="Segments"></a>
*
* <h4>Segments</h4>
*
* <p>Lucene indexes may be composed of multiple sub-indexes, or <i>segments</i>. Each segment is a
* fully independent index, which could be searched separately. Indexes evolve by:
*
* <ol>
* <li>Creating new segments for newly added documents.
* <li>Merging existing segments.
* </ol>
*
* <p>Searches may involve multiple segments and/or multiple indexes, each index potentially
* composed of a set of segments. <a id="Document_Numbers"></a>
*
* <h4>Document Numbers</h4>
*
* <p>Internally, Lucene refers to documents by an integer <i>document number</i>. The first
* document added to an index is numbered zero, and each subsequent document added gets a number one
* greater than the previous.
*
* <p>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:
*
* <ul>
* <li>
* <p>The numbers stored in each segment are unique only within the segment, and must be
* converted before they can be used in a larger context. The standard technique is to
* allocate each segment a range of values, based on the range of numbers used in that
* segment. To convert a document number from a segment to an external value, the segment's
* <i>base</i> document number is added. To convert an external value back to a
* segment-specific value, the segment is identified by the range that the external value is
* in, and the segment's base value is subtracted. For example two five document segments
* might be combined, so that the first segment has a base value of zero, and the second of
* five. Document three from the second segment would have an external value of eight.
* <li>
* <p>When documents are deleted, gaps are created in the numbering. These are eventually
* removed as the index evolves through merging. Deleted documents are dropped when segments
* are merged. A freshly-merged segment thus has no gaps in its numbering.
* </ul>
*
* </div> <a id="Overview"></a>
*
* <h3>Index Structure Overview</h3>
*
* <div>
*
* <p>Each segment index maintains the following:
*
* <ul>
* <li>{@link org.apache.lucene.codecs.lucene90.Lucene90SegmentInfoFormat Segment info}. This
* contains metadata about a segment, such as the number of documents, what files it uses, and
* information about how the segment is sorted
* <li>{@link org.apache.lucene.codecs.lucene90.Lucene90FieldInfosFormat Field names}. This
* contains metadata about the set of named fields used in the index.
* <li>{@link org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat Stored Field values}.
* This contains, for each document, a list of attribute-value pairs, where the attributes are
* field names. These are used to store auxiliary information about the document, such as its
* title, url, or an identifier to access a database. The set of stored fields are what is
* returned for each hit when searching. This is keyed by document number.
* <li>{@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat Term dictionary}. A
* dictionary containing all of the terms used in all of the indexed fields of all of the
* documents. The dictionary also contains the number of documents which contain the term, and
* pointers to the term's frequency and proximity data.
* <li>{@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat Term Frequency data}. For
* each term in the dictionary, the numbers of all the documents that contain that term, and
* the frequency of the term in that document, unless frequencies are omitted ({@link
* org.apache.lucene.index.IndexOptions#DOCS IndexOptions.DOCS})
* <li>{@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat Term Proximity data}. For
* each term in the dictionary, the positions that the term occurs in each document. Note that
* this will not exist if all fields in all documents omit position data.
* <li>{@link org.apache.lucene.codecs.lucene90.Lucene90NormsFormat Normalization factors}. For
* each field in each document, a value is stored that is multiplied into the score for hits
* on that field.
* <li>{@link org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat Term Vectors}. For each
* field in each document, the term vector (sometimes called document vector) may be stored. A
* term vector consists of term text and term frequency. To add Term Vectors to your index see
* the {@link org.apache.lucene.document.Field Field} constructors
* <li>{@link org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat Per-document values}. Like
* stored values, these are also keyed by document number, but are generally intended to be
* loaded into main memory for fast access. Whereas stored values are generally intended for
* summary results from searches, per-document values are useful for things like scoring
* factors.
* <li>{@link org.apache.lucene.codecs.lucene90.Lucene90LiveDocsFormat Live documents}. An
* optional file indicating which documents are live.
* <li>{@link org.apache.lucene.codecs.lucene90.Lucene90PointsFormat Point values}. Optional pair
* of files, recording dimensionally indexed fields, to enable fast numeric range filtering
* and large numeric values like BigInteger and BigDecimal (1D) and geographic shape
* intersection (2D, 3D).
* <li>{@link org.apache.lucene.backward_codecs.lucene92.Lucene92HnswVectorsFormat Vector values}.
* The vector format stores numeric vectors in a format optimized for random access and
* computation, supporting high-dimensional nearest-neighbor search.
* </ul>
*
* <p>Details on each of these are provided in their linked pages. </div> <a id="File_Naming"></a>
*
* <h3>File Naming</h3>
*
* <div>
*
* <p>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)
*
* <p>Typically, all segments in an index are stored in a single directory, although this is not
* required.
*
* <p>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. </div> <a id="file-names"></a>
*
* <h3>Summary of File Extensions</h3>
*
* <div>
*
* <p>The following table summarizes the names and extensions of the files in Lucene:
*
* <table class="padding4" style="border-spacing: 1px; border-collapse: separate">
* <caption>lucene filenames by extension</caption>
* <tr>
* <th>Name</th>
* <th>Extension</th>
* <th>Brief Description</th>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.index.SegmentInfos Segments File}</td>
* <td>segments_N</td>
* <td>Stores information about a commit point</td>
* </tr>
* <tr>
* <td><a href="#Lock_File">Lock File</a></td>
* <td>write.lock</td>
* <td>The Write lock prevents multiple IndexWriters from writing to the same
* file.</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90SegmentInfoFormat Segment Info}</td>
* <td>.si</td>
* <td>Stores metadata about a segment</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90CompoundFormat Compound File}</td>
* <td>.cfs, .cfe</td>
* <td>An optional "virtual" file consisting of all the other index files for
* systems that frequently run out of file handles.</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90FieldInfosFormat Fields}</td>
* <td>.fnm</td>
* <td>Stores information about the fields</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat Field Index}</td>
* <td>.fdx</td>
* <td>Contains pointers to field data</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat Field Data}</td>
* <td>.fdt</td>
* <td>The stored fields for documents</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat Term Dictionary}</td>
* <td>.tim</td>
* <td>The term dictionary, stores term info</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat Term Index}</td>
* <td>.tip</td>
* <td>The index into the Term Dictionary</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat Frequencies}</td>
* <td>.doc</td>
* <td>Contains the list of docs which contain each term along with frequency</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat Positions}</td>
* <td>.pos</td>
* <td>Stores position information about where a term occurs in the index</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat Payloads}</td>
* <td>.pay</td>
* <td>Stores additional per-position metadata information such as character offsets and user payloads</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90NormsFormat Norms}</td>
* <td>.nvd, .nvm</td>
* <td>Encodes length and boost factors for docs and fields</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat Per-Document Values}</td>
* <td>.dvd, .dvm</td>
* <td>Encodes additional scoring factors or other per-document information.</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat Term Vector Index}</td>
* <td>.tvx</td>
* <td>Stores offset into the document data file</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat Term Vector Data}</td>
* <td>.tvd</td>
* <td>Contains term vector data.</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90LiveDocsFormat Live Documents}</td>
* <td>.liv</td>
* <td>Info about what documents are live</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90PointsFormat Point values}</td>
* <td>.dii, .dim</td>
* <td>Holds indexed points</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.backward_codecs.lucene92.Lucene92HnswVectorsFormat Vector values}</td>
* <td>.vec, .vem</td>
* <td>Holds indexed vectors; <code>.vec</code> files contain the raw vector data, and
* <code>.vem</code> the vector metadata</td>
* </tr>
* </table>
*
* </div> <a id="Lock_File"></a>
*
* <h3>Lock File</h3>
*
* 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. <a id="History"></a>
*
* <h3>History</h3>
*
* <p>Compatibility notes are provided in this document, describing how file formats have changed
* from prior versions:
*
* <ul>
* <li>In version 2.1, the file format was changed to allow lock-less commits (ie, no more commit
* lock). The change is fully backwards compatible: you can open a pre-2.1 index for searching
* or adding/deleting of docs. When the new segments file is saved (committed), it will be
* written in the new file format (meaning no specific "upgrade" process is needed). But note
* that once a commit has occurred, pre-2.1 Lucene will not be able to read the index.
* <li>In version 2.3, the file format was changed to allow segments to share a single set of doc
* store (vectors &amp; stored fields) files. This allows for faster indexing in certain
* cases. The change is fully backwards compatible (in the same way as the lock-less commits
* change in 2.1).
* <li>In version 2.4, Strings are now written as true UTF-8 byte sequence, not Java's modified
* UTF-8. See <a href="http://issues.apache.org/jira/browse/LUCENE-510">LUCENE-510</a> for
* details.
* <li>In version 2.9, an optional opaque Map&lt;String,String&gt; CommitUserData may be passed to
* IndexWriter's commit methods (and later retrieved), which is recorded in the segments_N
* file. See <a href="http://issues.apache.org/jira/browse/LUCENE-1382">LUCENE-1382</a> for
* details. Also, diagnostics were added to each segment written recording details about why
* it was written (due to flush, merge; which OS/JRE was used; etc.). See issue <a
* href="http://issues.apache.org/jira/browse/LUCENE-1654">LUCENE-1654</a> for details.
* <li>In version 3.0, compressed fields are no longer written to the index (they can still be
* read, but on merge the new segment will write them, uncompressed). See issue <a
* href="http://issues.apache.org/jira/browse/LUCENE-1960">LUCENE-1960</a> for details.
* <li>In version 3.1, segments records the code version that created them. See <a
* href="http://issues.apache.org/jira/browse/LUCENE-2720">LUCENE-2720</a> for details.
* Additionally segments track explicitly whether or not they have term vectors. See <a
* href="http://issues.apache.org/jira/browse/LUCENE-2811">LUCENE-2811</a> for details.
* <li>In version 3.2, numeric fields are written as natively to stored fields file, previously
* they were stored in text format only.
* <li>In version 3.4, fields can omit position data while still indexing term frequencies.
* <li>In version 4.0, the format of the inverted index became extensible via the {@link
* org.apache.lucene.codecs.Codec Codec} api. Fast per-document storage ({@code DocValues})
* was introduced. Normalization factors need no longer be a single byte, they can be any
* {@link org.apache.lucene.index.NumericDocValues NumericDocValues}. Terms need not be
* unicode strings, they can be any byte sequence. Term offsets can optionally be indexed into
* the postings lists. Payloads can be stored in the term vectors.
* <li>In version 4.1, the format of the postings list changed to use either of FOR compression or
* variable-byte encoding, depending upon the frequency of the term. Terms appearing only once
* were changed to inline directly into the term dictionary. Stored fields are compressed by
* default.
* <li>In version 4.2, term vectors are compressed by default. DocValues has a new multi-valued
* type (SortedSet), that can be used for faceting/grouping/joining on multi-valued fields.
* <li>In version 4.5, DocValues were extended to explicitly represent missing values.
* <li>In version 4.6, FieldInfos were extended to support per-field DocValues generation, to
* allow updating NumericDocValues fields.
* <li>In version 4.8, checksum footers were added to the end of each index file for improved data
* integrity. Specifically, the last 8 bytes of every index file contain the zlib-crc32
* checksum of the file.
* <li>In version 4.9, DocValues has a new multi-valued numeric type (SortedNumeric) that is
* suitable for faceting/sorting/analytics.
* <li>In version 5.4, DocValues have been improved to store more information on disk: addresses
* for binary fields and ord indexes for multi-valued fields.
* <li>In version 6.0, Points were added, for multi-dimensional range/distance search.
* <li>In version 6.2, new Segment info format that reads/writes the index sort, to support index
* sorting.
* <li>In version 7.0, DocValues have been improved to better support sparse doc values thanks to
* an iterator API.
* <li>In version 8.0, postings have been enhanced to record, for each block of doc ids, the (term
* freq, normalization factor) pairs that may trigger the maximum score of the block. This
* information is recorded alongside skip data in order to be able to skip blocks of doc ids
* if they may not produce high enough scores. Additionally doc values and norms has been
* extended with jump-tables to make access O(1) instead of O(n), where n is the number of
* elements to skip when advancing in the data.
* <li>In version 8.4, postings, positions, offsets and payload lengths have move to a more
* performant encoding that is vectorized.
* <li>In version 8.6, index sort serialization is delegated to the sorts themselves, to allow
* user-defined sorts to be used
* <li>In version 8.7, stored fields compression became adaptive to better handle documents with
* smaller stored fields.
* <li>In version 9.0, vector-valued fields were added.
* <li>In version 9.1, vector-valued fields were modified to add a graph hierarchy.
* <li>In version 9.2, docs of vector-valued fields were moved from .vem to .vec and encoded by
* IndexDISI. ordToDoc mappings was added to .vem.
* </ul>
*
* <a id="Limitations"></a>
*
* <h3>Limitations</h3>
*
* <div>
*
* <p>Lucene uses a Java <code>int</code> to refer to document numbers, and the index file format
* uses an <code>Int32</code> 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
* <code>UInt64</code> values, or better yet, {@link org.apache.lucene.store.DataOutput#writeVInt
* VInt} values which have no limit. </div>
*/
package org.apache.lucene.backward_codecs.lucene92;

View File

@ -19,3 +19,4 @@ org.apache.lucene.backward_codecs.lucene86.Lucene86Codec
org.apache.lucene.backward_codecs.lucene87.Lucene87Codec
org.apache.lucene.backward_codecs.lucene90.Lucene90Codec
org.apache.lucene.backward_codecs.lucene91.Lucene91Codec
org.apache.lucene.backward_codecs.lucene92.Lucene92Codec

View File

@ -15,3 +15,4 @@
org.apache.lucene.backward_codecs.lucene90.Lucene90HnswVectorsFormat
org.apache.lucene.backward_codecs.lucene91.Lucene91HnswVectorsFormat
org.apache.lucene.backward_codecs.lucene92.Lucene92HnswVectorsFormat

View File

@ -15,9 +15,9 @@
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene92;
package org.apache.lucene.backward_codecs.lucene92;
import static org.apache.lucene.codecs.lucene92.Lucene92HnswVectorsFormat.DIRECT_MONOTONIC_BLOCK_SHIFT;
import static org.apache.lucene.backward_codecs.lucene92.Lucene92RWHnswVectorsFormat.DIRECT_MONOTONIC_BLOCK_SHIFT;
import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
import java.io.IOException;

View File

@ -0,0 +1,46 @@
/*
* 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.lucene92;
import org.apache.lucene.codecs.KnnVectorsFormat;
import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
/** Implements the Lucene 9.2 index format for backwards compat testing */
public class Lucene92RWCodec extends Lucene92Codec {
private final KnnVectorsFormat defaultKnnVectorsFormat;
private final KnnVectorsFormat knnVectorsFormat =
new PerFieldKnnVectorsFormat() {
@Override
public KnnVectorsFormat getKnnVectorsFormatForField(String field) {
return defaultKnnVectorsFormat;
}
};
/** Instantiates a new codec. */
public Lucene92RWCodec() {
defaultKnnVectorsFormat =
new Lucene92RWHnswVectorsFormat(
Lucene92HnswVectorsFormat.DEFAULT_MAX_CONN,
Lucene92HnswVectorsFormat.DEFAULT_BEAM_WIDTH);
}
@Override
public final KnnVectorsFormat knnVectorsFormat() {
return knnVectorsFormat;
}
}

View File

@ -0,0 +1,52 @@
/*
* 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.lucene92;
import java.io.IOException;
import org.apache.lucene.codecs.KnnVectorsReader;
import org.apache.lucene.codecs.KnnVectorsWriter;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
public final class Lucene92RWHnswVectorsFormat extends Lucene92HnswVectorsFormat {
static final int DIRECT_MONOTONIC_BLOCK_SHIFT = 16;
public Lucene92RWHnswVectorsFormat(int maxConn, int beamWidth) {
super(maxConn, beamWidth);
}
@Override
public KnnVectorsWriter fieldsWriter(SegmentWriteState state) throws IOException {
return new Lucene92HnswVectorsWriter(state, DEFAULT_MAX_CONN, DEFAULT_BEAM_WIDTH);
}
@Override
public KnnVectorsReader fieldsReader(SegmentReadState state) throws IOException {
return new Lucene92HnswVectorsReader(state);
}
@Override
public String toString() {
return "Lucene92RWHnswVectorsFormat(name = Lucene92RWHnswVectorsFormat, maxConn = "
+ maxConn
+ ", beamWidth="
+ beamWidth
+ ")";
}
}

View File

@ -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.lucene92;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.KnnVectorsFormat;
import org.apache.lucene.tests.index.BaseKnnVectorsFormatTestCase;
public class TestLucene92HnswVectorsFormat extends BaseKnnVectorsFormatTestCase {
@Override
protected Codec getCodec() {
return new Lucene92RWCodec();
}
public void testToString() {
Codec customCodec =
new Lucene92RWCodec() {
@Override
public KnnVectorsFormat getKnnVectorsFormatForField(String field) {
return new Lucene92RWHnswVectorsFormat(10, 20);
}
};
String expectedString =
"Lucene92RWHnswVectorsFormat(name = Lucene92RWHnswVectorsFormat, maxConn = 10, beamWidth=20)";
assertEquals(
expectedString,
((Lucene92Codec) customCodec).getKnnVectorsFormatForField("bogus_field").toString());
}
}

View File

@ -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.lucene92.Lucene92Codec;
import org.apache.lucene.codecs.lucene94.Lucene94Codec;
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 Lucene92Codec() {
new Lucene94Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return postingsFormatChosen;

View File

@ -15,8 +15,8 @@
* limitations under the License.
*/
import org.apache.lucene.codecs.lucene92.Lucene92Codec;
import org.apache.lucene.codecs.lucene92.Lucene92HnswVectorsFormat;
import org.apache.lucene.codecs.lucene94.Lucene94Codec;
import org.apache.lucene.codecs.lucene94.Lucene94HnswVectorsFormat;
/** Lucene Core. */
@SuppressWarnings("module") // the test framework is compiled after the core...
@ -31,7 +31,7 @@ module org.apache.lucene.core {
exports org.apache.lucene.codecs;
exports org.apache.lucene.codecs.compressing;
exports org.apache.lucene.codecs.lucene90;
exports org.apache.lucene.codecs.lucene92;
exports org.apache.lucene.codecs.lucene94;
exports org.apache.lucene.codecs.lucene90.blocktree;
exports org.apache.lucene.codecs.lucene90.compressing;
exports org.apache.lucene.codecs.perfield;
@ -63,11 +63,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
Lucene92Codec;
Lucene94Codec;
provides org.apache.lucene.codecs.DocValuesFormat with
org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat;
provides org.apache.lucene.codecs.KnnVectorsFormat with
Lucene92HnswVectorsFormat;
Lucene94HnswVectorsFormat;
provides org.apache.lucene.codecs.PostingsFormat with
org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat;
provides org.apache.lucene.index.SortFieldProvider with

View File

@ -55,7 +55,7 @@ public abstract class Codec implements NamedSPILoader.NamedSPI {
return LOADER;
}
static Codec defaultCodec = LOADER.lookup("Lucene92");
static Codec defaultCodec = LOADER.lookup("Lucene94");
}
private final String name;

View File

@ -84,8 +84,7 @@ public abstract class KnnVectorsFormat implements NamedSPILoader.NamedSPI {
new KnnVectorsFormat("EMPTY") {
@Override
public KnnVectorsWriter fieldsWriter(SegmentWriteState state) {
throw new UnsupportedOperationException(
"Attempt to write EMPTY VectorValues: maybe you forgot to use codec=Lucene92");
throw new UnsupportedOperationException("Attempt to write EMPTY VectorValues");
}
@Override

View File

@ -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.lucene94;
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.4 index format
*
* <p>If you want to reuse functionality of this codec in another codec, extend {@link FilterCodec}.
*
* @see org.apache.lucene.codecs.lucene94 package documentation for file format details.
* @lucene.experimental
*/
public class Lucene94Codec 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 Lucene94Codec.this.getPostingsFormatForField(field);
}
};
private final DocValuesFormat defaultDVFormat;
private final DocValuesFormat docValuesFormat =
new PerFieldDocValuesFormat() {
@Override
public DocValuesFormat getDocValuesFormatForField(String field) {
return Lucene94Codec.this.getDocValuesFormatForField(field);
}
};
private final KnnVectorsFormat defaultKnnVectorsFormat;
private final KnnVectorsFormat knnVectorsFormat =
new PerFieldKnnVectorsFormat() {
@Override
public KnnVectorsFormat getKnnVectorsFormatForField(String field) {
return Lucene94Codec.this.getKnnVectorsFormatForField(field);
}
};
private final StoredFieldsFormat storedFieldsFormat;
/** Instantiates a new codec. */
public Lucene94Codec() {
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 Lucene94Codec(Mode mode) {
super("Lucene94");
this.storedFieldsFormat =
new Lucene90StoredFieldsFormat(Objects.requireNonNull(mode).storedMode);
this.defaultPostingsFormat = new Lucene90PostingsFormat();
this.defaultDVFormat = new Lucene90DocValuesFormat();
this.defaultKnnVectorsFormat = new Lucene94HnswVectorsFormat();
}
@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 <code>field</code>.
*
* <p>The default implementation always returns "Lucene90".
*
* <p><b>WARNING:</b> 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 <code>field</code>
* .
*
* <p>The default implementation always returns "Lucene90".
*
* <p><b>WARNING:</b> 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 <code>field</code>
*
* <p>The default implementation always returns "lucene94".
*
* <p><b>WARNING:</b> 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;
}
}

View File

@ -0,0 +1,167 @@
/*
* 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.lucene94;
import java.io.IOException;
import org.apache.lucene.codecs.KnnVectorsFormat;
import org.apache.lucene.codecs.KnnVectorsReader;
import org.apache.lucene.codecs.KnnVectorsWriter;
import org.apache.lucene.codecs.lucene90.IndexedDISI;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.hnsw.HnswGraph;
/**
* Lucene 9.4 vector format, which encodes numeric vector values and an optional associated graph
* connecting the documents having values. The graph is used to power HNSW search. The format
* consists of three files:
*
* <h2>.vec (vector data) file</h2>
*
* <p>For each field:
*
* <ul>
* <li>Floating-point vector data ordered by field, document ordinal, and vector dimension. The
* floats are stored in little-endian byte order
* <li>DocIds encoded by {@link IndexedDISI#writeBitSet(DocIdSetIterator, IndexOutput, byte)},
* note that only in sparse case
* <li>OrdToDoc was encoded by {@link org.apache.lucene.util.packed.DirectMonotonicWriter}, note
* that only in sparse case
* </ul>
*
* <h2>.vex (vector index)</h2>
*
* <p>Stores graphs connecting the documents for each field organized as a list of nodes' neighbours
* as following:
*
* <ul>
* <li>For each level:
* <ul>
* <li>For each node:
* <ul>
* <li><b>[int32]</b> the number of neighbor nodes
* <li><b>array[int32]</b> the neighbor ordinals
* <li><b>array[int32]</b> padding if the number of the node's neighbors is less than
* the maximum number of connections allowed on this level. Padding is equal to
* ((maxConnOnLevel the number of neighbours) * 4) bytes.
* </ul>
* </ul>
* </ul>
*
* <h2>.vem (vector metadata) file</h2>
*
* <p>For each field:
*
* <ul>
* <li><b>[int32]</b> field number
* <li><b>[int32]</b> vector similarity function ordinal
* <li><b>[vlong]</b> offset to this field's vectors in the .vec file
* <li><b>[vlong]</b> length of this field's vectors, in bytes
* <li><b>[vlong]</b> offset to this field's index in the .vex file
* <li><b>[vlong]</b> length of this field's index data, in bytes
* <li><b>[int]</b> dimension of this field's vectors
* <li><b>[int]</b> the number of documents having values for this field
* <li><b>[int8]</b> if equals to -1, dense all documents have values for a field. If equals to
* 0, sparse some documents missing values.
* <li>DocIds were encoded by {@link IndexedDISI#writeBitSet(DocIdSetIterator, IndexOutput, byte)}
* <li>OrdToDoc was encoded by {@link org.apache.lucene.util.packed.DirectMonotonicWriter}, note
* that only in sparse case
* <li><b>[int]</b> the maximum number of connections (neigbours) that each node can have
* <li><b>[int]</b> number of levels in the graph
* <li>Graph nodes by level. For each level
* <ul>
* <li><b>[int]</b> the number of nodes on this level
* <li><b>array[int]</b> for levels greater than 0 list of nodes on this level, stored as
* the the level 0th nodes ordinals.
* </ul>
* </ul>
*
* @lucene.experimental
*/
public final class Lucene94HnswVectorsFormat extends KnnVectorsFormat {
static final String META_CODEC_NAME = "lucene94HnswVectorsFormatMeta";
static final String VECTOR_DATA_CODEC_NAME = "lucene94HnswVectorsFormatData";
static final String VECTOR_INDEX_CODEC_NAME = "lucene94HnswVectorsFormatIndex";
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;
static final int DIRECT_MONOTONIC_BLOCK_SHIFT = 16;
/**
* Controls how many of the nearest neighbor candidates are connected to the new node. Defaults to
* {@link Lucene94HnswVectorsFormat#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 Lucene94HnswVectorsFormat#DEFAULT_BEAM_WIDTH}. See {@link
* HnswGraph} for details.
*/
private final int beamWidth;
/** Constructs a format using default graph construction parameters */
public Lucene94HnswVectorsFormat() {
this(DEFAULT_MAX_CONN, DEFAULT_BEAM_WIDTH);
}
/**
* Constructs a format using the given graph construction parameters.
*
* @param maxConn the maximum number of connections to a node in the HNSW graph
* @param beamWidth the size of the queue maintained during graph construction.
*/
public Lucene94HnswVectorsFormat(int maxConn, int beamWidth) {
super("Lucene94HnswVectorsFormat");
this.maxConn = maxConn;
this.beamWidth = beamWidth;
}
@Override
public KnnVectorsWriter fieldsWriter(SegmentWriteState state) throws IOException {
return new Lucene94HnswVectorsWriter(state, maxConn, beamWidth);
}
@Override
public KnnVectorsReader fieldsReader(SegmentReadState state) throws IOException {
return new Lucene94HnswVectorsReader(state);
}
@Override
public String toString() {
return "Lucene94HnswVectorsFormat(name=Lucene94HnswVectorsFormat, maxConn="
+ maxConn
+ ", beamWidth="
+ beamWidth
+ ")";
}
}

View File

@ -0,0 +1,463 @@
/*
* 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.lucene94;
import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
import java.io.IOException;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Map;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.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.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.IOUtils;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.hnsw.HnswGraph;
import org.apache.lucene.util.hnsw.HnswGraphSearcher;
import org.apache.lucene.util.hnsw.NeighborQueue;
import org.apache.lucene.util.packed.DirectMonotonicReader;
/**
* Reads vectors from the index segments along with index data structures supporting KNN search.
*
* @lucene.experimental
*/
public final class Lucene94HnswVectorsReader extends KnnVectorsReader {
private final FieldInfos fieldInfos;
private final Map<String, FieldEntry> fields = new HashMap<>();
private final IndexInput vectorData;
private final IndexInput vectorIndex;
Lucene94HnswVectorsReader(SegmentReadState state) throws IOException {
this.fieldInfos = state.fieldInfos;
int versionMeta = readMetadata(state);
boolean success = false;
try {
vectorData =
openDataInput(
state,
versionMeta,
Lucene94HnswVectorsFormat.VECTOR_DATA_EXTENSION,
Lucene94HnswVectorsFormat.VECTOR_DATA_CODEC_NAME);
vectorIndex =
openDataInput(
state,
versionMeta,
Lucene94HnswVectorsFormat.VECTOR_INDEX_EXTENSION,
Lucene94HnswVectorsFormat.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, Lucene94HnswVectorsFormat.META_EXTENSION);
int versionMeta = -1;
try (ChecksumIndexInput meta = state.directory.openChecksumInput(metaFileName, state.context)) {
Throwable priorE = null;
try {
versionMeta =
CodecUtil.checkIndexHeader(
meta,
Lucene94HnswVectorsFormat.META_CODEC_NAME,
Lucene94HnswVectorsFormat.VERSION_START,
Lucene94HnswVectorsFormat.VERSION_CURRENT,
state.segmentInfo.getId(),
state.segmentSuffix);
readFields(meta, state.fieldInfos);
} catch (Throwable exception) {
priorE = exception;
} finally {
CodecUtil.checkFooter(meta, priorE);
}
}
return versionMeta;
}
private static IndexInput openDataInput(
SegmentReadState state, int versionMeta, String fileExtension, String codecName)
throws IOException {
String fileName =
IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, fileExtension);
IndexInput in = state.directory.openInput(fileName, state.context);
boolean success = false;
try {
int versionVectorData =
CodecUtil.checkIndexHeader(
in,
codecName,
Lucene94HnswVectorsFormat.VERSION_START,
Lucene94HnswVectorsFormat.VERSION_CURRENT,
state.segmentInfo.getId(),
state.segmentSuffix);
if (versionMeta != versionVectorData) {
throw new CorruptIndexException(
"Format versions mismatch: meta="
+ versionMeta
+ ", "
+ codecName
+ "="
+ versionVectorData,
in);
}
CodecUtil.retrieveChecksum(in);
success = true;
return in;
} finally {
if (success == false) {
IOUtils.closeWhileHandlingException(in);
}
}
}
private void readFields(ChecksumIndexInput meta, FieldInfos infos) throws IOException {
for (int fieldNumber = meta.readInt(); fieldNumber != -1; fieldNumber = meta.readInt()) {
FieldInfo info = infos.fieldInfo(fieldNumber);
if (info == null) {
throw new CorruptIndexException("Invalid field number: " + fieldNumber, meta);
}
FieldEntry fieldEntry = readField(meta);
validateFieldEntry(info, fieldEntry);
fields.put(info.name, fieldEntry);
}
}
private void validateFieldEntry(FieldInfo info, FieldEntry fieldEntry) {
int dimension = info.getVectorDimension();
if (dimension != fieldEntry.dimension) {
throw new IllegalStateException(
"Inconsistent vector dimension for field=\""
+ info.name
+ "\"; "
+ dimension
+ " != "
+ fieldEntry.dimension);
}
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(IndexInput input) throws IOException {
VectorSimilarityFunction similarityFunction = readSimilarityFunction(input);
return new FieldEntry(input, similarityFunction);
}
@Override
public long ramBytesUsed() {
long totalBytes = RamUsageEstimator.shallowSizeOfInstance(Lucene94HnswVectorsFormat.class);
totalBytes +=
RamUsageEstimator.sizeOfMap(
fields, RamUsageEstimator.shallowSizeOfInstance(FieldEntry.class));
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 OffHeapVectorValues.load(fieldEntry, vectorData);
}
@Override
public TopDocs search(String field, float[] target, int k, Bits acceptDocs, int visitedLimit)
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 = OffHeapVectorValues.load(fieldEntry, vectorData);
NeighborQueue results =
HnswGraphSearcher.search(
target,
k,
vectorValues,
fieldEntry.similarityFunction,
getGraph(fieldEntry),
vectorValues.getAcceptOrds(acceptDocs),
visitedLimit);
int i = 0;
ScoreDoc[] scoreDocs = new ScoreDoc[Math.min(results.size(), k)];
while (results.size() > 0) {
int node = results.topNode();
float score = results.topScore();
results.pop();
scoreDocs[scoreDocs.length - ++i] = new ScoreDoc(vectorValues.ordToDoc(node), score);
}
TotalHits.Relation relation =
results.incomplete()
? TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO
: TotalHits.Relation.EQUAL_TO;
return new TopDocs(new TotalHits(results.visitedCount(), relation), scoreDocs);
}
/** Get knn graph values; used for testing */
public HnswGraph getGraph(String field) throws IOException {
FieldInfo info = fieldInfos.fieldInfo(field);
if (info == null) {
throw new IllegalArgumentException("No such field '" + field + "'");
}
FieldEntry entry = fields.get(field);
if (entry != null && entry.vectorIndexLength > 0) {
return getGraph(entry);
} else {
return HnswGraph.EMPTY;
}
}
private HnswGraph getGraph(FieldEntry entry) throws IOException {
IndexInput bytesSlice =
vectorIndex.slice("graph-data", entry.vectorIndexOffset, entry.vectorIndexLength);
return new OffHeapHnswGraph(entry, bytesSlice);
}
@Override
public void close() throws IOException {
IOUtils.close(vectorData, vectorIndex);
}
static class FieldEntry {
final VectorSimilarityFunction similarityFunction;
final long vectorDataOffset;
final long vectorDataLength;
final long vectorIndexOffset;
final long vectorIndexLength;
final int M;
final int numLevels;
final int dimension;
final int size;
final int[][] nodesByLevel;
// for each level the start offsets in vectorIndex file from where to read neighbours
final long[] graphOffsetsByLevel;
// the following four variables used to read docIds encoded by IndexDISI
// special values of docsWithFieldOffset are -1 and -2
// -1 : dense
// -2 : empty
// other: sparse
final long docsWithFieldOffset;
final long docsWithFieldLength;
final short jumpTableEntryCount;
final byte denseRankPower;
// the following four variables used to read ordToDoc encoded by DirectMonotonicWriter
// note that only spare case needs to store ordToDoc
final long addressesOffset;
final int blockShift;
final DirectMonotonicReader.Meta meta;
final long addressesLength;
FieldEntry(IndexInput input, VectorSimilarityFunction similarityFunction) throws IOException {
this.similarityFunction = similarityFunction;
vectorDataOffset = input.readVLong();
vectorDataLength = input.readVLong();
vectorIndexOffset = input.readVLong();
vectorIndexLength = input.readVLong();
dimension = input.readInt();
size = input.readInt();
docsWithFieldOffset = input.readLong();
docsWithFieldLength = input.readLong();
jumpTableEntryCount = input.readShort();
denseRankPower = input.readByte();
// dense or empty
if (docsWithFieldOffset == -1 || docsWithFieldOffset == -2) {
addressesOffset = 0;
blockShift = 0;
meta = null;
addressesLength = 0;
} else {
// sparse
addressesOffset = input.readLong();
blockShift = input.readVInt();
meta = DirectMonotonicReader.loadMeta(input, size, blockShift);
addressesLength = input.readLong();
}
// read nodes by level
M = 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.readInt();
}
}
}
// 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 if (level == 1) {
int numNodesOnLevel0 = size;
graphOffsetsByLevel[level] = (1 + (M * 2)) * Integer.BYTES * numNodesOnLevel0;
} else {
int numNodesOnPrevLevel = nodesByLevel[level - 1].length;
graphOffsetsByLevel[level] =
graphOffsetsByLevel[level - 1] + (1 + M) * Integer.BYTES * numNodesOnPrevLevel;
}
}
}
int size() {
return size;
}
}
/** Read the nearest-neighbors graph from the index input */
private static final class OffHeapHnswGraph extends HnswGraph {
final IndexInput dataIn;
final int[][] nodesByLevel;
final long[] graphOffsetsByLevel;
final int numLevels;
final int entryNode;
final int size;
final long bytesForConns;
final long bytesForConns0;
int arcCount;
int arcUpTo;
int arc;
OffHeapHnswGraph(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.M + 1) * Integer.BYTES;
this.bytesForConns0 = ((long) (entry.M * 2) + 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 * (level == 0 ? bytesForConns0 : 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);
}
}
}
}

View File

@ -0,0 +1,330 @@
/*
* 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.lucene94;
import static org.apache.lucene.codecs.lucene94.Lucene94HnswVectorsFormat.DIRECT_MONOTONIC_BLOCK_SHIFT;
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.codecs.lucene90.IndexedDISI;
import org.apache.lucene.index.DocsWithFieldSet;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexFileNames;
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.search.DocIdSetIterator;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.hnsw.HnswGraph.NodesIterator;
import org.apache.lucene.util.hnsw.HnswGraphBuilder;
import org.apache.lucene.util.hnsw.NeighborArray;
import org.apache.lucene.util.hnsw.OnHeapHnswGraph;
import org.apache.lucene.util.packed.DirectMonotonicWriter;
/**
* Writes vector values and knn graphs to index segments.
*
* @lucene.experimental
*/
public final class Lucene94HnswVectorsWriter extends KnnVectorsWriter {
private final SegmentWriteState segmentWriteState;
private final IndexOutput meta, vectorData, vectorIndex;
private final int maxDoc;
private final int M;
private final int beamWidth;
private boolean finished;
Lucene94HnswVectorsWriter(SegmentWriteState state, int M, int beamWidth) throws IOException {
this.M = M;
this.beamWidth = beamWidth;
assert state.fieldInfos.hasVectorValues();
segmentWriteState = state;
String metaFileName =
IndexFileNames.segmentFileName(
state.segmentInfo.name, state.segmentSuffix, Lucene94HnswVectorsFormat.META_EXTENSION);
String vectorDataFileName =
IndexFileNames.segmentFileName(
state.segmentInfo.name,
state.segmentSuffix,
Lucene94HnswVectorsFormat.VECTOR_DATA_EXTENSION);
String indexDataFileName =
IndexFileNames.segmentFileName(
state.segmentInfo.name,
state.segmentSuffix,
Lucene94HnswVectorsFormat.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,
Lucene94HnswVectorsFormat.META_CODEC_NAME,
Lucene94HnswVectorsFormat.VERSION_CURRENT,
state.segmentInfo.getId(),
state.segmentSuffix);
CodecUtil.writeIndexHeader(
vectorData,
Lucene94HnswVectorsFormat.VECTOR_DATA_CODEC_NAME,
Lucene94HnswVectorsFormat.VERSION_CURRENT,
state.segmentInfo.getId(),
state.segmentSuffix);
CodecUtil.writeIndexHeader(
vectorIndex,
Lucene94HnswVectorsFormat.VECTOR_INDEX_CODEC_NAME,
Lucene94HnswVectorsFormat.VERSION_CURRENT,
state.segmentInfo.getId(),
state.segmentSuffix);
maxDoc = state.segmentInfo.maxDoc();
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
DocsWithFieldSet docsWithField = 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
// we use Lucene94HnswVectorsReader.DenseOffHeapVectorValues for the graph construction
// doesn't need to know docIds
// TODO: separate random access vector values from DocIdSetIterator?
OffHeapVectorValues offHeapVectors =
new OffHeapVectorValues.DenseOffHeapVectorValues(
vectors.dimension(), docsWithField.cardinality(), vectorDataInput);
OnHeapHnswGraph graph =
offHeapVectors.size() == 0
? null
: writeGraph(offHeapVectors, fieldInfo.getVectorSimilarityFunction());
long vectorIndexLength = vectorIndex.getFilePointer() - vectorIndexOffset;
writeMeta(
fieldInfo,
vectorDataOffset,
vectorDataLength,
vectorIndexOffset,
vectorIndexLength,
docsWithField,
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 set of documents that contains vectors.
*/
private static DocsWithFieldSet writeVectorData(IndexOutput output, VectorValues vectors)
throws IOException {
DocsWithFieldSet docsWithField = new DocsWithFieldSet();
for (int docV = vectors.nextDoc(); docV != NO_MORE_DOCS; docV = vectors.nextDoc()) {
// write vector
BytesRef binaryValue = vectors.binaryValue();
assert binaryValue.length == vectors.dimension() * Float.BYTES;
output.writeBytes(binaryValue.bytes, binaryValue.offset, binaryValue.length);
docsWithField.add(docV);
}
return docsWithField;
}
private void writeMeta(
FieldInfo field,
long vectorDataOffset,
long vectorDataLength,
long vectorIndexOffset,
long vectorIndexLength,
DocsWithFieldSet docsWithField,
OnHeapHnswGraph 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());
// write docIDs
int count = docsWithField.cardinality();
meta.writeInt(count);
if (count == 0) {
meta.writeLong(-2); // docsWithFieldOffset
meta.writeLong(0L); // docsWithFieldLength
meta.writeShort((short) -1); // jumpTableEntryCount
meta.writeByte((byte) -1); // denseRankPower
} else if (count == maxDoc) {
meta.writeLong(-1); // docsWithFieldOffset
meta.writeLong(0L); // docsWithFieldLength
meta.writeShort((short) -1); // jumpTableEntryCount
meta.writeByte((byte) -1); // denseRankPower
} else {
long offset = vectorData.getFilePointer();
meta.writeLong(offset); // docsWithFieldOffset
final short jumpTableEntryCount =
IndexedDISI.writeBitSet(
docsWithField.iterator(), vectorData, IndexedDISI.DEFAULT_DENSE_RANK_POWER);
meta.writeLong(vectorData.getFilePointer() - offset); // docsWithFieldLength
meta.writeShort(jumpTableEntryCount);
meta.writeByte(IndexedDISI.DEFAULT_DENSE_RANK_POWER);
// write ordToDoc mapping
long start = vectorData.getFilePointer();
meta.writeLong(start);
meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
// dense case and empty case do not need to store ordToMap mapping
final DirectMonotonicWriter ordToDocWriter =
DirectMonotonicWriter.getInstance(meta, vectorData, count, DIRECT_MONOTONIC_BLOCK_SHIFT);
DocIdSetIterator iterator = docsWithField.iterator();
for (int doc = iterator.nextDoc();
doc != DocIdSetIterator.NO_MORE_DOCS;
doc = iterator.nextDoc()) {
ordToDocWriter.add(doc);
}
ordToDocWriter.finish();
meta.writeLong(vectorData.getFilePointer() - start);
}
meta.writeInt(M);
// 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.writeInt(node); // list of nodes on a level
}
}
}
}
}
private OnHeapHnswGraph writeGraph(
RandomAccessVectorValuesProducer vectorValues, VectorSimilarityFunction similarityFunction)
throws IOException {
// build graph
HnswGraphBuilder hnswGraphBuilder =
new HnswGraphBuilder(
vectorValues, similarityFunction, M, beamWidth, HnswGraphBuilder.randSeed);
hnswGraphBuilder.setInfoStream(segmentWriteState.infoStream);
OnHeapHnswGraph 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++) {
int maxConnOnLevel = level == 0 ? (M * 2) : M;
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 < maxConnOnLevel; 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);
}
}

View File

@ -0,0 +1,321 @@
/*
* 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.lucene94;
import java.io.IOException;
import java.nio.ByteBuffer;
import org.apache.lucene.codecs.lucene90.IndexedDISI;
import org.apache.lucene.index.RandomAccessVectorValues;
import org.apache.lucene.index.RandomAccessVectorValuesProducer;
import org.apache.lucene.index.VectorValues;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.RandomAccessInput;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.packed.DirectMonotonicReader;
/** Read the vector values from the index input. This supports both iterated and random access. */
abstract class OffHeapVectorValues extends VectorValues
implements RandomAccessVectorValues, RandomAccessVectorValuesProducer {
protected final int dimension;
protected final int size;
protected final IndexInput slice;
protected final BytesRef binaryValue;
protected final ByteBuffer byteBuffer;
protected final int byteSize;
protected final float[] value;
OffHeapVectorValues(int dimension, int size, IndexInput slice) {
this.dimension = dimension;
this.size = size;
this.slice = slice;
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 size;
}
@Override
public long cost() {
return size;
}
@Override
public float[] vectorValue(int targetOrd) throws IOException {
slice.seek((long) targetOrd * byteSize);
slice.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 {
slice.seek((long) targetOrd * byteSize);
slice.readBytes(byteBuffer.array(), byteBuffer.arrayOffset(), byteSize);
}
public abstract int ordToDoc(int ord);
static OffHeapVectorValues load(
Lucene94HnswVectorsReader.FieldEntry fieldEntry, IndexInput vectorData) throws IOException {
if (fieldEntry.docsWithFieldOffset == -2) {
return new EmptyOffHeapVectorValues(fieldEntry.dimension);
}
IndexInput bytesSlice =
vectorData.slice("vector-data", fieldEntry.vectorDataOffset, fieldEntry.vectorDataLength);
if (fieldEntry.docsWithFieldOffset == -1) {
return new DenseOffHeapVectorValues(fieldEntry.dimension, fieldEntry.size, bytesSlice);
} else {
return new SparseOffHeapVectorValues(fieldEntry, vectorData, bytesSlice);
}
}
abstract Bits getAcceptOrds(Bits acceptDocs);
static class DenseOffHeapVectorValues extends OffHeapVectorValues {
private int doc = -1;
public DenseOffHeapVectorValues(int dimension, int size, IndexInput slice) {
super(dimension, size, slice);
}
@Override
public float[] vectorValue() throws IOException {
slice.seek((long) doc * byteSize);
slice.readFloats(value, 0, value.length);
return value;
}
@Override
public BytesRef binaryValue() throws IOException {
slice.seek((long) doc * byteSize);
slice.readBytes(byteBuffer.array(), byteBuffer.arrayOffset(), byteSize, false);
return binaryValue;
}
@Override
public int docID() {
return doc;
}
@Override
public int nextDoc() throws IOException {
return advance(doc + 1);
}
@Override
public int advance(int target) throws IOException {
assert docID() < target;
if (target >= size) {
return doc = NO_MORE_DOCS;
}
return doc = target;
}
@Override
public RandomAccessVectorValues randomAccess() throws IOException {
return new DenseOffHeapVectorValues(dimension, size, slice.clone());
}
@Override
public int ordToDoc(int ord) {
return ord;
}
@Override
Bits getAcceptOrds(Bits acceptDocs) {
return acceptDocs;
}
}
private static class SparseOffHeapVectorValues extends OffHeapVectorValues {
private final DirectMonotonicReader ordToDoc;
private final IndexedDISI disi;
// dataIn was used to init a new IndexedDIS for #randomAccess()
private final IndexInput dataIn;
private final Lucene94HnswVectorsReader.FieldEntry fieldEntry;
public SparseOffHeapVectorValues(
Lucene94HnswVectorsReader.FieldEntry fieldEntry, IndexInput dataIn, IndexInput slice)
throws IOException {
super(fieldEntry.dimension, fieldEntry.size, slice);
this.fieldEntry = fieldEntry;
final RandomAccessInput addressesData =
dataIn.randomAccessSlice(fieldEntry.addressesOffset, fieldEntry.addressesLength);
this.dataIn = dataIn;
this.ordToDoc = DirectMonotonicReader.getInstance(fieldEntry.meta, addressesData);
this.disi =
new IndexedDISI(
dataIn,
fieldEntry.docsWithFieldOffset,
fieldEntry.docsWithFieldLength,
fieldEntry.jumpTableEntryCount,
fieldEntry.denseRankPower,
fieldEntry.size);
}
@Override
public float[] vectorValue() throws IOException {
slice.seek((long) (disi.index()) * byteSize);
slice.readFloats(value, 0, value.length);
return value;
}
@Override
public BytesRef binaryValue() throws IOException {
slice.seek((long) (disi.index()) * byteSize);
slice.readBytes(byteBuffer.array(), byteBuffer.arrayOffset(), byteSize, false);
return binaryValue;
}
@Override
public int docID() {
return disi.docID();
}
@Override
public int nextDoc() throws IOException {
return disi.nextDoc();
}
@Override
public int advance(int target) throws IOException {
assert docID() < target;
return disi.advance(target);
}
@Override
public RandomAccessVectorValues randomAccess() throws IOException {
return new SparseOffHeapVectorValues(fieldEntry, dataIn, slice.clone());
}
@Override
public int ordToDoc(int ord) {
return (int) ordToDoc.get(ord);
}
@Override
Bits getAcceptOrds(Bits acceptDocs) {
if (acceptDocs == null) {
return null;
}
return new Bits() {
@Override
public boolean get(int index) {
return acceptDocs.get(ordToDoc(index));
}
@Override
public int length() {
return size;
}
};
}
}
private static class EmptyOffHeapVectorValues extends OffHeapVectorValues {
public EmptyOffHeapVectorValues(int dimension) {
super(dimension, 0, null);
}
private int doc = -1;
@Override
public int dimension() {
return super.dimension();
}
@Override
public int size() {
return 0;
}
@Override
public float[] vectorValue() throws IOException {
throw new UnsupportedOperationException();
}
@Override
public BytesRef binaryValue() throws IOException {
throw new UnsupportedOperationException();
}
@Override
public int docID() {
return doc;
}
@Override
public int nextDoc() throws IOException {
return advance(doc + 1);
}
@Override
public int advance(int target) throws IOException {
return doc = NO_MORE_DOCS;
}
@Override
public long cost() {
return 0;
}
@Override
public RandomAccessVectorValues randomAccess() throws IOException {
throw new UnsupportedOperationException();
}
@Override
public float[] vectorValue(int targetOrd) throws IOException {
throw new UnsupportedOperationException();
}
@Override
public BytesRef binaryValue(int targetOrd) throws IOException {
throw new UnsupportedOperationException();
}
@Override
public int ordToDoc(int ord) {
throw new UnsupportedOperationException();
}
@Override
Bits getAcceptOrds(Bits acceptDocs) {
return null;
}
}
}

View File

@ -16,7 +16,7 @@
*/
/**
* Lucene 9.2 file format.
* Lucene 9.3 file format.
*
* <h2>Apache Lucene - Index File Formats</h2>
*
@ -180,7 +180,7 @@
* of files, recording dimensionally indexed fields, to enable fast numeric range filtering
* and large numeric values like BigInteger and BigDecimal (1D) and geographic shape
* intersection (2D, 3D).
* <li>{@link org.apache.lucene.codecs.lucene92.Lucene92HnswVectorsFormat Vector values}. The
* <li>{@link org.apache.lucene.codecs.lucene94.Lucene94HnswVectorsFormat Vector values}. The
* vector format stores numeric vectors in a format optimized for random access and
* computation, supporting high-dimensional nearest-neighbor search.
* </ul>
@ -310,7 +310,7 @@
* <td>Holds indexed points</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene92.Lucene92HnswVectorsFormat Vector values}</td>
* <td>{@link org.apache.lucene.codecs.lucene94.Lucene94HnswVectorsFormat Vector values}</td>
* <td>.vec, .vem</td>
* <td>Holds indexed vectors; <code>.vec</code> files contain the raw vector data, and
* <code>.vem</code> the vector metadata</td>
@ -419,4 +419,4 @@
* <code>UInt64</code> values, or better yet, {@link org.apache.lucene.store.DataOutput#writeVInt
* VInt} values which have no limit. </div>
*/
package org.apache.lucene.codecs.lucene92;
package org.apache.lucene.codecs.lucene94;

View File

@ -13,4 +13,4 @@
# See the License for the specific language governing permissions and
# limitations under the License.
org.apache.lucene.codecs.lucene92.Lucene92Codec
org.apache.lucene.codecs.lucene94.Lucene94Codec

View File

@ -13,4 +13,4 @@
# See the License for the specific language governing permissions and
# limitations under the License.
org.apache.lucene.codecs.lucene92.Lucene92HnswVectorsFormat
org.apache.lucene.codecs.lucene94.Lucene94HnswVectorsFormat

View File

@ -18,8 +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.lucene92.Lucene92Codec;
import org.apache.lucene.codecs.lucene92.Lucene92Codec.Mode;
import org.apache.lucene.codecs.lucene94.Lucene94Codec;
import org.apache.lucene.codecs.lucene94.Lucene94Codec.Mode;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.StoredField;
import org.apache.lucene.index.DirectoryReader;
@ -31,7 +31,7 @@ import org.apache.lucene.tests.index.BaseStoredFieldsFormatTestCase;
public class TestLucene90StoredFieldsFormatHighCompression extends BaseStoredFieldsFormatTestCase {
@Override
protected Codec getCodec() {
return new Lucene92Codec(Mode.BEST_COMPRESSION);
return new Lucene94Codec(Mode.BEST_COMPRESSION);
}
/**
@ -41,7 +41,7 @@ public class TestLucene90StoredFieldsFormatHighCompression extends BaseStoredFie
Directory dir = newDirectory();
for (int i = 0; i < 10; i++) {
IndexWriterConfig iwc = newIndexWriterConfig();
iwc.setCodec(new Lucene92Codec(RandomPicks.randomFrom(random(), Mode.values())));
iwc.setCodec(new Lucene94Codec(RandomPicks.randomFrom(random(), Mode.values())));
IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig());
Document doc = new Document();
doc.add(new StoredField("field1", "value1"));
@ -70,7 +70,7 @@ public class TestLucene90StoredFieldsFormatHighCompression extends BaseStoredFie
expectThrows(
NullPointerException.class,
() -> {
new Lucene92Codec(null);
new Lucene94Codec(null);
});
expectThrows(

View File

@ -0,0 +1,42 @@
/*
* 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.lucene94;
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 TestLucene94HnswVectorsFormat extends BaseKnnVectorsFormatTestCase {
@Override
protected Codec getCodec() {
return TestUtil.getDefaultCodec();
}
public void testToString() {
Lucene94Codec customCodec =
new Lucene94Codec() {
@Override
public KnnVectorsFormat getKnnVectorsFormatForField(String field) {
return new Lucene94HnswVectorsFormat(10, 20);
}
};
String expectedString =
"Lucene94HnswVectorsFormat(name=Lucene94HnswVectorsFormat, maxConn=10, beamWidth=20)";
assertEquals(expectedString, customCodec.getKnnVectorsFormatForField("bogus_field").toString());
}
}

View File

@ -30,9 +30,9 @@ 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.lucene92.Lucene92Codec;
import org.apache.lucene.codecs.lucene92.Lucene92HnswVectorsFormat;
import org.apache.lucene.codecs.lucene92.Lucene92HnswVectorsReader;
import org.apache.lucene.codecs.lucene94.Lucene94Codec;
import org.apache.lucene.codecs.lucene94.Lucene94HnswVectorsFormat;
import org.apache.lucene.codecs.lucene94.Lucene94HnswVectorsReader;
import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
@ -64,7 +64,7 @@ public class TestKnnGraph extends LuceneTestCase {
private static final String KNN_GRAPH_FIELD = "vector";
private static int M = Lucene92HnswVectorsFormat.DEFAULT_MAX_CONN;
private static int M = Lucene94HnswVectorsFormat.DEFAULT_MAX_CONN;
private Codec codec;
private VectorSimilarityFunction similarityFunction;
@ -77,10 +77,10 @@ public class TestKnnGraph extends LuceneTestCase {
}
codec =
new Lucene92Codec() {
new Lucene94Codec() {
@Override
public KnnVectorsFormat getKnnVectorsFormatForField(String field) {
return new Lucene92HnswVectorsFormat(M, Lucene92HnswVectorsFormat.DEFAULT_BEAM_WIDTH);
return new Lucene94HnswVectorsFormat(M, Lucene94HnswVectorsFormat.DEFAULT_BEAM_WIDTH);
}
};
@ -90,7 +90,7 @@ public class TestKnnGraph extends LuceneTestCase {
@After
public void cleanup() {
M = Lucene92HnswVectorsFormat.DEFAULT_MAX_CONN;
M = Lucene94HnswVectorsFormat.DEFAULT_MAX_CONN;
}
/** Basic test of creating documents in a graph */
@ -237,8 +237,8 @@ public class TestKnnGraph extends LuceneTestCase {
PerFieldKnnVectorsFormat.FieldsReader perFieldReader =
(PerFieldKnnVectorsFormat.FieldsReader)
((CodecReader) getOnlyLeafReader(reader)).getVectorReader();
Lucene92HnswVectorsReader vectorReader =
(Lucene92HnswVectorsReader) perFieldReader.getFieldReader(KNN_GRAPH_FIELD);
Lucene94HnswVectorsReader vectorReader =
(Lucene94HnswVectorsReader) perFieldReader.getFieldReader(KNN_GRAPH_FIELD);
graph = copyGraph(vectorReader.getGraph(KNN_GRAPH_FIELD));
}
}
@ -436,8 +436,8 @@ public class TestKnnGraph extends LuceneTestCase {
if (perFieldReader == null) {
continue;
}
Lucene92HnswVectorsReader vectorReader =
(Lucene92HnswVectorsReader) perFieldReader.getFieldReader(vectorField);
Lucene94HnswVectorsReader vectorReader =
(Lucene94HnswVectorsReader) perFieldReader.getFieldReader(vectorField);
if (vectorReader == null) {
continue;
}

View File

@ -39,9 +39,9 @@ import java.util.Objects;
import java.util.Set;
import org.apache.lucene.codecs.KnnVectorsFormat;
import org.apache.lucene.codecs.KnnVectorsReader;
import org.apache.lucene.codecs.lucene92.Lucene92Codec;
import org.apache.lucene.codecs.lucene92.Lucene92HnswVectorsFormat;
import org.apache.lucene.codecs.lucene92.Lucene92HnswVectorsReader;
import org.apache.lucene.codecs.lucene94.Lucene94Codec;
import org.apache.lucene.codecs.lucene94.Lucene94HnswVectorsFormat;
import org.apache.lucene.codecs.lucene94.Lucene94HnswVectorsReader;
import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.FieldType;
@ -248,7 +248,7 @@ public class KnnGraphTester {
KnnVectorsReader vectorsReader =
((PerFieldKnnVectorsFormat.FieldsReader) ((CodecReader) leafReader).getVectorReader())
.getFieldReader(KNN_FIELD);
HnswGraph knnValues = ((Lucene92HnswVectorsReader) vectorsReader).getGraph(KNN_FIELD);
HnswGraph knnValues = ((Lucene94HnswVectorsReader) vectorsReader).getGraph(KNN_FIELD);
System.out.printf("Leaf %d has %d documents\n", context.ord, leafReader.maxDoc());
printGraphFanout(knnValues, leafReader.maxDoc());
}
@ -579,10 +579,10 @@ public class KnnGraphTester {
private int createIndex(Path docsPath, Path indexPath) throws IOException {
IndexWriterConfig iwc = new IndexWriterConfig().setOpenMode(IndexWriterConfig.OpenMode.CREATE);
iwc.setCodec(
new Lucene92Codec() {
new Lucene94Codec() {
@Override
public KnnVectorsFormat getKnnVectorsFormatForField(String field) {
return new Lucene92HnswVectorsFormat(maxConn, beamWidth);
return new Lucene94HnswVectorsFormat(maxConn, beamWidth);
}
});
// iwc.setMergePolicy(NoMergePolicy.INSTANCE);

View File

@ -25,9 +25,9 @@ import java.util.HashSet;
import java.util.Random;
import java.util.Set;
import org.apache.lucene.codecs.KnnVectorsFormat;
import org.apache.lucene.codecs.lucene92.Lucene92Codec;
import org.apache.lucene.codecs.lucene92.Lucene92HnswVectorsFormat;
import org.apache.lucene.codecs.lucene92.Lucene92HnswVectorsReader;
import org.apache.lucene.codecs.lucene94.Lucene94Codec;
import org.apache.lucene.codecs.lucene94.Lucene94HnswVectorsFormat;
import org.apache.lucene.codecs.lucene94.Lucene94HnswVectorsReader;
import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.KnnVectorField;
@ -81,10 +81,10 @@ public class TestHnswGraph extends LuceneTestCase {
IndexWriterConfig iwc =
new IndexWriterConfig()
.setCodec(
new Lucene92Codec() {
new Lucene94Codec() {
@Override
public KnnVectorsFormat getKnnVectorsFormatForField(String field) {
return new Lucene92HnswVectorsFormat(M, beamWidth);
return new Lucene94HnswVectorsFormat(M, beamWidth);
}
});
try (IndexWriter iw = new IndexWriter(dir, iwc)) {
@ -111,7 +111,7 @@ public class TestHnswGraph extends LuceneTestCase {
assertEquals(indexedDoc, ctx.reader().numDocs());
assertVectorsEqual(v3, values);
HnswGraph graphValues =
((Lucene92HnswVectorsReader)
((Lucene94HnswVectorsReader)
((PerFieldKnnVectorsFormat.FieldsReader)
((CodecReader) ctx.reader()).getVectorReader())
.getFieldReader("field"))

View File

@ -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.lucene92.Lucene92Codec;
import org.apache.lucene.codecs.lucene94.Lucene94Codec;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.IntPoint;
@ -959,7 +959,7 @@ public class TestSuggestField extends LuceneTestCase {
IndexWriterConfig iwc = newIndexWriterConfig(random(), analyzer);
iwc.setMergePolicy(newLogMergePolicy());
Codec filterCodec =
new Lucene92Codec() {
new Lucene94Codec() {
CompletionPostingsFormat.FSTLoadMode fstLoadMode =
RandomPicks.randomFrom(random(), CompletionPostingsFormat.FSTLoadMode.values());
PostingsFormat postingsFormat = new Completion90PostingsFormat(fstLoadMode);

View File

@ -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.lucene92.Lucene92Codec;
import org.apache.lucene.codecs.lucene94.Lucene94Codec;
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 ("Lucene92".equals(TEST_CODEC)
|| ("random".equals(TEST_CODEC) && randomVal == 5 && !shouldAvoidCodec("Lucene92"))) {
codec = new Lucene92Codec(RandomPicks.randomFrom(random, Lucene92Codec.Mode.values()));
} else if ("Lucene94".equals(TEST_CODEC)
|| ("random".equals(TEST_CODEC) && randomVal == 5 && !shouldAvoidCodec("Lucene94"))) {
codec = new Lucene94Codec(RandomPicks.randomFrom(random, Lucene94Codec.Mode.values()));
} else if (!"random".equals(TEST_CODEC)) {
codec = Codec.forName(TEST_CODEC);
} else if ("random".equals(TEST_POSTINGSFORMAT)) {

View File

@ -55,8 +55,8 @@ import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.blocktreeords.BlockTreeOrdsPostingsFormat;
import org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat;
import org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat;
import org.apache.lucene.codecs.lucene92.Lucene92Codec;
import org.apache.lucene.codecs.lucene92.Lucene92HnswVectorsFormat;
import org.apache.lucene.codecs.lucene94.Lucene94Codec;
import org.apache.lucene.codecs.lucene94.Lucene94HnswVectorsFormat;
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 Lucene92Codec();
return new Lucene94Codec();
}
/**
@ -1322,7 +1322,7 @@ public final class TestUtil {
* Lucene.
*/
public static KnnVectorsFormat getDefaultKnnVectorsFormat() {
return new Lucene92HnswVectorsFormat();
return new Lucene94HnswVectorsFormat();
}
public static boolean anyFilesExceptWriteLock(Directory dir) throws IOException {