mirror of https://github.com/apache/lucene.git
Vectors Format Refactor first draft
This commit is contained in:
parent
92420d345a
commit
77cec44423
|
@ -23,12 +23,12 @@ import org.apache.lucene.codecs.hnsw.DefaultFlatVectorScorer;
|
|||
import org.apache.lucene.codecs.hnsw.FlatVectorsFormat;
|
||||
import org.apache.lucene.codecs.hnsw.FlatVectorsWriter;
|
||||
import org.apache.lucene.codecs.hnsw.ScalarQuantizedVectorScorer;
|
||||
import org.apache.lucene.codecs.lucene99.Lucene99FlatVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene99.Lucene99HnswScalarQuantizedVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsWriter;
|
||||
import org.apache.lucene.codecs.lucene99.Lucene99ScalarQuantizedVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene99.Lucene99ScalarQuantizedVectorsWriter;
|
||||
import org.apache.lucene.codecs.lucene99.vectors.storage.Lucene99FlatVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene99.vectors.graph.Lucene99HnswScalarQuantizedVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene99.vectors.graph.Lucene99HnswVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene99.vectors.graph.Lucene99HnswVectorsWriter;
|
||||
import org.apache.lucene.codecs.lucene99.vectors.storage.Lucene99ScalarQuantizedVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene99.vectors.storage.Lucene99ScalarQuantizedVectorsWriter;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
|
||||
class Lucene99RWHnswScalarQuantizationVectorsFormat
|
||||
|
|
|
@ -23,8 +23,8 @@ import java.io.IOException;
|
|||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.KnnVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene99.Lucene99Codec;
|
||||
import org.apache.lucene.codecs.lucene99.Lucene99HnswScalarQuantizedVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene99.vectors.graph.Lucene99HnswScalarQuantizedVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene99.vectors.graph.Lucene99HnswVectorsFormat;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.FieldType;
|
||||
|
|
|
@ -17,11 +17,11 @@
|
|||
|
||||
package org.apache.lucene.codecs.bitvectors;
|
||||
|
||||
import static org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat.DEFAULT_BEAM_WIDTH;
|
||||
import static org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat.DEFAULT_MAX_CONN;
|
||||
import static org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat.DEFAULT_NUM_MERGE_WORKER;
|
||||
import static org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat.MAXIMUM_BEAM_WIDTH;
|
||||
import static org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat.MAXIMUM_MAX_CONN;
|
||||
import static org.apache.lucene.codecs.lucene99.vectors.graph.Lucene99HnswVectorsFormat.DEFAULT_BEAM_WIDTH;
|
||||
import static org.apache.lucene.codecs.lucene99.vectors.graph.Lucene99HnswVectorsFormat.DEFAULT_MAX_CONN;
|
||||
import static org.apache.lucene.codecs.lucene99.vectors.graph.Lucene99HnswVectorsFormat.DEFAULT_NUM_MERGE_WORKER;
|
||||
import static org.apache.lucene.codecs.lucene99.vectors.graph.Lucene99HnswVectorsFormat.MAXIMUM_BEAM_WIDTH;
|
||||
import static org.apache.lucene.codecs.lucene99.vectors.graph.Lucene99HnswVectorsFormat.MAXIMUM_MAX_CONN;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
|
@ -30,10 +30,10 @@ import org.apache.lucene.codecs.KnnVectorsFormat;
|
|||
import org.apache.lucene.codecs.KnnVectorsReader;
|
||||
import org.apache.lucene.codecs.KnnVectorsWriter;
|
||||
import org.apache.lucene.codecs.hnsw.FlatVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene99.Lucene99FlatVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsReader;
|
||||
import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsWriter;
|
||||
import org.apache.lucene.codecs.lucene99.vectors.storage.Lucene99FlatVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene99.vectors.graph.Lucene99HnswVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene99.vectors.graph.Lucene99HnswVectorsReader;
|
||||
import org.apache.lucene.codecs.lucene99.vectors.graph.Lucene99HnswVectorsWriter;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.MergeState;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
|
|
|
@ -64,6 +64,8 @@ module org.apache.lucene.core {
|
|||
|
||||
exports org.apache.lucene.util.quantization;
|
||||
exports org.apache.lucene.codecs.hnsw;
|
||||
exports org.apache.lucene.codecs.lucene99.vectors.storage;
|
||||
exports org.apache.lucene.codecs.lucene99.vectors.graph;
|
||||
|
||||
provides org.apache.lucene.analysis.TokenizerFactory with
|
||||
org.apache.lucene.analysis.standard.StandardTokenizerFactory;
|
||||
|
@ -72,8 +74,8 @@ module org.apache.lucene.core {
|
|||
provides org.apache.lucene.codecs.DocValuesFormat with
|
||||
org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat;
|
||||
provides org.apache.lucene.codecs.KnnVectorsFormat with
|
||||
org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat,
|
||||
org.apache.lucene.codecs.lucene99.Lucene99HnswScalarQuantizedVectorsFormat;
|
||||
org.apache.lucene.codecs.lucene99.vectors.graph.Lucene99HnswVectorsFormat,
|
||||
org.apache.lucene.codecs.lucene99.vectors.graph.Lucene99HnswScalarQuantizedVectorsFormat;
|
||||
provides org.apache.lucene.codecs.PostingsFormat with
|
||||
org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat;
|
||||
provides org.apache.lucene.index.SortFieldProvider with
|
||||
|
|
|
@ -55,7 +55,7 @@ public abstract class KnnVectorsFormat implements NamedSPILoader.NamedSPI {
|
|||
}
|
||||
}
|
||||
|
||||
private final String name;
|
||||
protected final String name;
|
||||
|
||||
/** Sole constructor */
|
||||
protected KnnVectorsFormat(String name) {
|
||||
|
|
|
@ -21,7 +21,7 @@
|
|||
* storage formats or scoring without significant changes to the HNSW code. Some examples for
|
||||
* scoring include {@link org.apache.lucene.codecs.hnsw.ScalarQuantizedVectorScorer} and {@link
|
||||
* org.apache.lucene.codecs.hnsw.DefaultFlatVectorScorer}. Some examples for storing include {@link
|
||||
* org.apache.lucene.codecs.lucene99.Lucene99FlatVectorsFormat} and {@link
|
||||
* org.apache.lucene.codecs.lucene99.Lucene99ScalarQuantizedVectorsFormat}.
|
||||
* org.apache.lucene.codecs.lucene99.vectors.storage.Lucene99FlatVectorsFormat} and {@link
|
||||
* org.apache.lucene.codecs.lucene99.vectors.storage.Lucene99ScalarQuantizedVectorsFormat}.
|
||||
*/
|
||||
package org.apache.lucene.codecs.hnsw;
|
||||
|
|
|
@ -20,6 +20,7 @@ import java.util.Objects;
|
|||
import org.apache.lucene.codecs.*;
|
||||
import org.apache.lucene.codecs.lucene90.*;
|
||||
import org.apache.lucene.codecs.lucene94.Lucene94FieldInfosFormat;
|
||||
import org.apache.lucene.codecs.lucene99.vectors.graph.Lucene99HnswVectorsFormat;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
|
||||
|
|
|
@ -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.lucene99.Lucene99HnswVectorsFormat Vector values}. The
|
||||
* <li>{@link org.apache.lucene.codecs.lucene99.vectors.graph.Lucene99HnswVectorsFormat 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.lucene99.Lucene99HnswVectorsFormat Vector values}</td>
|
||||
* <td>{@link org.apache.lucene.codecs.lucene99.vectors.graph.Lucene99HnswVectorsFormat Vector values}</td>
|
||||
* <td>.vec, .vem, .veq, vex</td>
|
||||
* <td>Holds indexed vectors; <code>.vec</code> files contain the raw vector data,
|
||||
* <code>.vem</code> the vector metadata, <code>.veq</code> the quantized vector data, and <code>.vex</code> the
|
||||
|
|
|
@ -15,24 +15,11 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.lucene.codecs.lucene99;
|
||||
package org.apache.lucene.codecs.lucene99.vectors.graph;
|
||||
|
||||
import static org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat.DEFAULT_BEAM_WIDTH;
|
||||
import static org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat.DEFAULT_MAX_CONN;
|
||||
import static org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat.DEFAULT_NUM_MERGE_WORKER;
|
||||
import static org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat.MAXIMUM_BEAM_WIDTH;
|
||||
import static org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat.MAXIMUM_MAX_CONN;
|
||||
import org.apache.lucene.codecs.lucene99.vectors.storage.Lucene99ScalarQuantizedVectorsFormat;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import org.apache.lucene.codecs.KnnVectorsFormat;
|
||||
import org.apache.lucene.codecs.KnnVectorsReader;
|
||||
import org.apache.lucene.codecs.KnnVectorsWriter;
|
||||
import org.apache.lucene.codecs.hnsw.FlatVectorsFormat;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.search.TaskExecutor;
|
||||
import org.apache.lucene.util.hnsw.HnswGraph;
|
||||
|
||||
/**
|
||||
* Lucene 9.9 vector format, which encodes numeric vector values into an associated graph connecting
|
||||
|
@ -42,32 +29,20 @@ import org.apache.lucene.util.hnsw.HnswGraph;
|
|||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class Lucene99HnswScalarQuantizedVectorsFormat extends KnnVectorsFormat {
|
||||
public class Lucene99HnswScalarQuantizedVectorsFormat extends Lucene99HnswVectorsFormat {
|
||||
|
||||
public static final String NAME = "Lucene99HnswScalarQuantizedVectorsFormat";
|
||||
|
||||
|
||||
/**
|
||||
* Controls how many of the nearest neighbor candidates are connected to the new node. Defaults to
|
||||
* {@link Lucene99HnswVectorsFormat#DEFAULT_MAX_CONN}. See {@link HnswGraph} for more details.
|
||||
* to motivate
|
||||
*/
|
||||
private final int maxConn;
|
||||
public static final int DEFAULT_QUANTIZATION_BITS = 7;
|
||||
|
||||
/**
|
||||
* The number of candidate neighbors to track while searching the graph for each newly inserted
|
||||
* node. Defaults to {@link Lucene99HnswVectorsFormat#DEFAULT_BEAM_WIDTH}. See {@link HnswGraph}
|
||||
* for details.
|
||||
*/
|
||||
private final int beamWidth;
|
||||
|
||||
/** The format for storing, reading, merging vectors on disk */
|
||||
private final FlatVectorsFormat flatVectorsFormat;
|
||||
|
||||
private final int numMergeWorkers;
|
||||
private final TaskExecutor mergeExec;
|
||||
|
||||
/** Constructs a format using default graph construction parameters */
|
||||
public Lucene99HnswScalarQuantizedVectorsFormat() {
|
||||
this(DEFAULT_MAX_CONN, DEFAULT_BEAM_WIDTH, DEFAULT_NUM_MERGE_WORKER, 7, true, null, null);
|
||||
this(DEFAULT_MAX_CONN, DEFAULT_BEAM_WIDTH, DEFAULT_NUM_MERGE_WORKER, DEFAULT_QUANTIZATION_BITS, true, null, null);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -77,7 +52,7 @@ public class Lucene99HnswScalarQuantizedVectorsFormat extends KnnVectorsFormat {
|
|||
* @param beamWidth the size of the queue maintained during graph construction.
|
||||
*/
|
||||
public Lucene99HnswScalarQuantizedVectorsFormat(int maxConn, int beamWidth) {
|
||||
this(maxConn, beamWidth, DEFAULT_NUM_MERGE_WORKER, 7, true, null, null);
|
||||
this(maxConn, beamWidth, DEFAULT_NUM_MERGE_WORKER, DEFAULT_QUANTIZATION_BITS, true, null, null);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -105,58 +80,11 @@ public class Lucene99HnswScalarQuantizedVectorsFormat extends KnnVectorsFormat {
|
|||
boolean compress,
|
||||
Float confidenceInterval,
|
||||
ExecutorService mergeExec) {
|
||||
super(NAME);
|
||||
if (maxConn <= 0 || maxConn > MAXIMUM_MAX_CONN) {
|
||||
throw new IllegalArgumentException(
|
||||
"maxConn must be positive and less than or equal to "
|
||||
+ MAXIMUM_MAX_CONN
|
||||
+ "; maxConn="
|
||||
+ maxConn);
|
||||
}
|
||||
if (beamWidth <= 0 || beamWidth > MAXIMUM_BEAM_WIDTH) {
|
||||
throw new IllegalArgumentException(
|
||||
"beamWidth must be positive and less than or equal to "
|
||||
+ MAXIMUM_BEAM_WIDTH
|
||||
+ "; beamWidth="
|
||||
+ beamWidth);
|
||||
}
|
||||
this.maxConn = maxConn;
|
||||
this.beamWidth = beamWidth;
|
||||
if (numMergeWorkers == 1 && mergeExec != null) {
|
||||
throw new IllegalArgumentException(
|
||||
"No executor service is needed as we'll use single thread to merge");
|
||||
}
|
||||
this.numMergeWorkers = numMergeWorkers;
|
||||
if (mergeExec != null) {
|
||||
this.mergeExec = new TaskExecutor(mergeExec);
|
||||
} else {
|
||||
this.mergeExec = null;
|
||||
}
|
||||
this.flatVectorsFormat =
|
||||
super(NAME, maxConn, beamWidth, numMergeWorkers, mergeExec);
|
||||
super.flatVectorsFormat =
|
||||
new Lucene99ScalarQuantizedVectorsFormat(confidenceInterval, bits, compress);
|
||||
}
|
||||
|
||||
@Override
|
||||
public KnnVectorsWriter fieldsWriter(SegmentWriteState state) throws IOException {
|
||||
return new Lucene99HnswVectorsWriter(
|
||||
state,
|
||||
maxConn,
|
||||
beamWidth,
|
||||
flatVectorsFormat.fieldsWriter(state),
|
||||
numMergeWorkers,
|
||||
mergeExec);
|
||||
}
|
||||
|
||||
@Override
|
||||
public KnnVectorsReader fieldsReader(SegmentReadState state) throws IOException {
|
||||
return new Lucene99HnswVectorsReader(state, flatVectorsFormat.fieldsReader(state));
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMaxDimensions(String fieldName) {
|
||||
return 1024;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "Lucene99HnswScalarQuantizedVectorsFormat(name=Lucene99HnswScalarQuantizedVectorsFormat, maxConn="
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.lucene.codecs.lucene99;
|
||||
package org.apache.lucene.codecs.lucene99.vectors.graph;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
|
@ -25,6 +25,7 @@ import org.apache.lucene.codecs.KnnVectorsWriter;
|
|||
import org.apache.lucene.codecs.hnsw.DefaultFlatVectorScorer;
|
||||
import org.apache.lucene.codecs.hnsw.FlatVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene90.IndexedDISI;
|
||||
import org.apache.lucene.codecs.lucene99.vectors.storage.Lucene99FlatVectorsFormat;
|
||||
import org.apache.lucene.index.MergePolicy;
|
||||
import org.apache.lucene.index.MergeScheduler;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
|
@ -86,8 +87,8 @@ import org.apache.lucene.util.hnsw.HnswGraph;
|
|||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public final class Lucene99HnswVectorsFormat extends KnnVectorsFormat {
|
||||
|
||||
public class Lucene99HnswVectorsFormat extends KnnVectorsFormat {
|
||||
static final String NAME = "Lucene99HnswVectorsFormat";
|
||||
static final String META_CODEC_NAME = "Lucene99HnswVectorsFormatMeta";
|
||||
static final String VECTOR_INDEX_CODEC_NAME = "Lucene99HnswVectorsFormatIndex";
|
||||
static final String META_EXTENSION = "vem";
|
||||
|
@ -128,25 +129,25 @@ public final class Lucene99HnswVectorsFormat extends KnnVectorsFormat {
|
|||
* Controls how many of the nearest neighbor candidates are connected to the new node. Defaults to
|
||||
* {@link Lucene99HnswVectorsFormat#DEFAULT_MAX_CONN}. See {@link HnswGraph} for more details.
|
||||
*/
|
||||
private final int maxConn;
|
||||
protected final int maxConn;
|
||||
|
||||
/**
|
||||
* The number of candidate neighbors to track while searching the graph for each newly inserted
|
||||
* node. Defaults to {@link Lucene99HnswVectorsFormat#DEFAULT_BEAM_WIDTH}. See {@link HnswGraph}
|
||||
* for details.
|
||||
*/
|
||||
private final int beamWidth;
|
||||
protected final int beamWidth;
|
||||
|
||||
/** The format for storing, reading, merging vectors on disk */
|
||||
private static final FlatVectorsFormat flatVectorsFormat =
|
||||
protected static FlatVectorsFormat flatVectorsFormat =
|
||||
new Lucene99FlatVectorsFormat(new DefaultFlatVectorScorer());
|
||||
|
||||
private final int numMergeWorkers;
|
||||
private final TaskExecutor mergeExec;
|
||||
protected final int numMergeWorkers;
|
||||
protected final TaskExecutor mergeExec;
|
||||
|
||||
/** Constructs a format using default graph construction parameters */
|
||||
public Lucene99HnswVectorsFormat() {
|
||||
this(DEFAULT_MAX_CONN, DEFAULT_BEAM_WIDTH, DEFAULT_NUM_MERGE_WORKER, null);
|
||||
this(NAME, DEFAULT_MAX_CONN, DEFAULT_BEAM_WIDTH, DEFAULT_NUM_MERGE_WORKER, null);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -156,23 +157,24 @@ public final class Lucene99HnswVectorsFormat extends KnnVectorsFormat {
|
|||
* @param beamWidth the size of the queue maintained during graph construction.
|
||||
*/
|
||||
public Lucene99HnswVectorsFormat(int maxConn, int beamWidth) {
|
||||
this(maxConn, beamWidth, DEFAULT_NUM_MERGE_WORKER, null);
|
||||
this(NAME, maxConn, beamWidth, DEFAULT_NUM_MERGE_WORKER, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructs a format using the given graph construction parameters and scalar quantization.
|
||||
*
|
||||
* @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.
|
||||
* @param name name of the codec
|
||||
* @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.
|
||||
* @param numMergeWorkers number of workers (threads) that will be used when doing merge. If
|
||||
* larger than 1, a non-null {@link ExecutorService} must be passed as mergeExec
|
||||
* @param mergeExec the {@link ExecutorService} that will be used by ALL vector writers that are
|
||||
* generated by this format to do the merge. If null, the configured {@link
|
||||
* MergeScheduler#getIntraMergeExecutor(MergePolicy.OneMerge)} is used.
|
||||
* larger than 1, a non-null {@link ExecutorService} must be passed as mergeExec
|
||||
* @param mergeExec the {@link ExecutorService} that will be used by ALL vector writers that are
|
||||
* generated by this format to do the merge. If null, the configured {@link
|
||||
* MergeScheduler#getIntraMergeExecutor(MergePolicy.OneMerge)} is used.
|
||||
*/
|
||||
public Lucene99HnswVectorsFormat(
|
||||
int maxConn, int beamWidth, int numMergeWorkers, ExecutorService mergeExec) {
|
||||
super("Lucene99HnswVectorsFormat");
|
||||
String name, int maxConn, int beamWidth, int numMergeWorkers, ExecutorService mergeExec) {
|
||||
super(name);
|
||||
if (maxConn <= 0 || maxConn > MAXIMUM_MAX_CONN) {
|
||||
throw new IllegalArgumentException(
|
||||
"maxConn must be positive and less than or equal to "
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.lucene.codecs.lucene99;
|
||||
package org.apache.lucene.codecs.lucene99.vectors.graph;
|
||||
|
||||
import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
|
||||
|
|
@ -15,10 +15,10 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.lucene.codecs.lucene99;
|
||||
package org.apache.lucene.codecs.lucene99.vectors.graph;
|
||||
|
||||
import static org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat.DIRECT_MONOTONIC_BLOCK_SHIFT;
|
||||
import static org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsReader.SIMILARITY_FUNCTIONS;
|
||||
import static org.apache.lucene.codecs.lucene99.vectors.graph.Lucene99HnswVectorsFormat.DIRECT_MONOTONIC_BLOCK_SHIFT;
|
||||
import static org.apache.lucene.codecs.lucene99.vectors.graph.Lucene99HnswVectorsReader.SIMILARITY_FUNCTIONS;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.lucene.codecs.lucene99;
|
||||
package org.apache.lucene.codecs.lucene99.vectors.storage;
|
||||
|
||||
import java.io.IOException;
|
||||
import org.apache.lucene.codecs.hnsw.FlatVectorsFormat;
|
|
@ -15,10 +15,10 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.lucene.codecs.lucene99;
|
||||
package org.apache.lucene.codecs.lucene99.vectors.storage;
|
||||
|
||||
import static org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsReader.readSimilarityFunction;
|
||||
import static org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsReader.readVectorEncoding;
|
||||
import static org.apache.lucene.codecs.lucene99.vectors.graph.Lucene99HnswVectorsReader.readSimilarityFunction;
|
||||
import static org.apache.lucene.codecs.lucene99.vectors.graph.Lucene99HnswVectorsReader.readVectorEncoding;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
|
@ -15,9 +15,9 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.lucene.codecs.lucene99;
|
||||
package org.apache.lucene.codecs.lucene99.vectors.storage;
|
||||
|
||||
import static org.apache.lucene.codecs.lucene99.Lucene99FlatVectorsFormat.DIRECT_MONOTONIC_BLOCK_SHIFT;
|
||||
import static org.apache.lucene.codecs.lucene99.vectors.storage.Lucene99FlatVectorsFormat.DIRECT_MONOTONIC_BLOCK_SHIFT;
|
||||
import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
|
||||
|
||||
import java.io.Closeable;
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.lucene.codecs.lucene99;
|
||||
package org.apache.lucene.codecs.lucene99.vectors.storage;
|
||||
|
||||
import static org.apache.lucene.codecs.hnsw.ScalarQuantizedVectorScorer.quantizeQuery;
|
||||
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.lucene.codecs.lucene99;
|
||||
package org.apache.lucene.codecs.lucene99.vectors.storage;
|
||||
|
||||
import java.io.IOException;
|
||||
import org.apache.lucene.codecs.hnsw.DefaultFlatVectorScorer;
|
|
@ -15,10 +15,10 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.lucene.codecs.lucene99;
|
||||
package org.apache.lucene.codecs.lucene99.vectors.storage;
|
||||
|
||||
import static org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsReader.readSimilarityFunction;
|
||||
import static org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsReader.readVectorEncoding;
|
||||
import static org.apache.lucene.codecs.lucene99.vectors.graph.Lucene99HnswVectorsReader.readSimilarityFunction;
|
||||
import static org.apache.lucene.codecs.lucene99.vectors.graph.Lucene99HnswVectorsReader.readVectorEncoding;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
|
@ -15,11 +15,11 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.lucene.codecs.lucene99;
|
||||
package org.apache.lucene.codecs.lucene99.vectors.storage;
|
||||
|
||||
import static org.apache.lucene.codecs.lucene99.Lucene99FlatVectorsFormat.DIRECT_MONOTONIC_BLOCK_SHIFT;
|
||||
import static org.apache.lucene.codecs.lucene99.Lucene99ScalarQuantizedVectorsFormat.QUANTIZED_VECTOR_COMPONENT;
|
||||
import static org.apache.lucene.codecs.lucene99.Lucene99ScalarQuantizedVectorsFormat.calculateDefaultConfidenceInterval;
|
||||
import static org.apache.lucene.codecs.lucene99.vectors.storage.Lucene99FlatVectorsFormat.DIRECT_MONOTONIC_BLOCK_SHIFT;
|
||||
import static org.apache.lucene.codecs.lucene99.vectors.storage.Lucene99ScalarQuantizedVectorsFormat.QUANTIZED_VECTOR_COMPONENT;
|
||||
import static org.apache.lucene.codecs.lucene99.vectors.storage.Lucene99ScalarQuantizedVectorsFormat.calculateDefaultConfidenceInterval;
|
||||
import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
|
||||
import static org.apache.lucene.util.RamUsageEstimator.shallowSizeOfInstance;
|
||||
|
||||
|
@ -839,13 +839,13 @@ public final class Lucene99ScalarQuantizedVectorsWriter extends FlatVectorsWrite
|
|||
}
|
||||
}
|
||||
|
||||
static class FloatVectorWrapper extends FloatVectorValues {
|
||||
public static class FloatVectorWrapper extends FloatVectorValues {
|
||||
private final List<float[]> vectorList;
|
||||
private final float[] copy;
|
||||
private final boolean normalize;
|
||||
protected int curDoc = -1;
|
||||
|
||||
FloatVectorWrapper(List<float[]> vectorList, boolean normalize) {
|
||||
public FloatVectorWrapper(List<float[]> vectorList, boolean normalize) {
|
||||
this.vectorList = vectorList;
|
||||
this.copy = new float[vectorList.get(0).length];
|
||||
this.normalize = normalize;
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.lucene.codecs.lucene99;
|
||||
package org.apache.lucene.codecs.lucene99.vectors.storage;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
|
@ -13,5 +13,5 @@
|
|||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
|
||||
org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat
|
||||
org.apache.lucene.codecs.lucene99.Lucene99HnswScalarQuantizedVectorsFormat
|
||||
org.apache.lucene.codecs.lucene99.vectors.graph.Lucene99HnswVectorsFormat
|
||||
org.apache.lucene.codecs.lucene99.vectors.graph.Lucene99HnswScalarQuantizedVectorsFormat
|
||||
|
|
|
@ -32,7 +32,7 @@ import java.util.List;
|
|||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import org.apache.lucene.codecs.lucene95.OffHeapByteVectorValues;
|
||||
import org.apache.lucene.codecs.lucene95.OffHeapFloatVectorValues;
|
||||
import org.apache.lucene.codecs.lucene99.Lucene99ScalarQuantizedVectorScorer;
|
||||
import org.apache.lucene.codecs.lucene99.vectors.storage.Lucene99ScalarQuantizedVectorScorer;
|
||||
import org.apache.lucene.index.VectorSimilarityFunction;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
|
|
|
@ -14,7 +14,7 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.lucene.codecs.lucene99;
|
||||
package org.apache.lucene.codecs.lucene99.vectors.graph;
|
||||
|
||||
import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
|
||||
|
||||
|
@ -25,6 +25,8 @@ import org.apache.lucene.codecs.Codec;
|
|||
import org.apache.lucene.codecs.FilterCodec;
|
||||
import org.apache.lucene.codecs.KnnVectorsFormat;
|
||||
import org.apache.lucene.codecs.KnnVectorsReader;
|
||||
import org.apache.lucene.codecs.lucene99.Lucene99Codec;
|
||||
import org.apache.lucene.codecs.lucene99.vectors.storage.Lucene99ScalarQuantizedVectorsWriter;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.KnnFloatVectorField;
|
|
@ -14,7 +14,7 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.lucene.codecs.lucene99;
|
||||
package org.apache.lucene.codecs.lucene99.vectors.graph;
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.FilterCodec;
|
||||
|
@ -51,6 +51,6 @@ public class TestLucene99HnswVectorsFormat extends BaseKnnVectorsFormatTestCase
|
|||
expectThrows(IllegalArgumentException.class, () -> new Lucene99HnswVectorsFormat(20, 3201));
|
||||
expectThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> new Lucene99HnswVectorsFormat(20, 100, 1, new SameThreadExecutorService()));
|
||||
() -> new Lucene99HnswVectorsFormat("Lucene99HnswVectorsFormat", 20, 100, 1, new SameThreadExecutorService()));
|
||||
}
|
||||
}
|
|
@ -15,9 +15,9 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.lucene.codecs.lucene99;
|
||||
package org.apache.lucene.codecs.lucene99.vectors.storage;
|
||||
|
||||
import static org.apache.lucene.codecs.lucene99.OffHeapQuantizedByteVectorValues.compressBytes;
|
||||
import static org.apache.lucene.codecs.lucene99.vectors.storage.OffHeapQuantizedByteVectorValues.compressBytes;
|
||||
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.IOException;
|
||||
|
@ -27,6 +27,10 @@ import org.apache.lucene.codecs.Codec;
|
|||
import org.apache.lucene.codecs.KnnVectorsFormat;
|
||||
import org.apache.lucene.codecs.KnnVectorsReader;
|
||||
import org.apache.lucene.codecs.hnsw.DefaultFlatVectorScorer;
|
||||
import org.apache.lucene.codecs.lucene99.Lucene99Codec;
|
||||
import org.apache.lucene.codecs.lucene99.vectors.graph.Lucene99HnswScalarQuantizedVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene99.vectors.graph.Lucene99HnswVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene99.vectors.graph.Lucene99HnswVectorsReader;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
|
@ -29,7 +29,7 @@ import org.apache.lucene.codecs.KnnFieldVectorsWriter;
|
|||
import org.apache.lucene.codecs.KnnVectorsFormat;
|
||||
import org.apache.lucene.codecs.KnnVectorsReader;
|
||||
import org.apache.lucene.codecs.KnnVectorsWriter;
|
||||
import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene99.vectors.graph.Lucene99HnswVectorsFormat;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.KnnFloatVectorField;
|
||||
|
|
|
@ -32,9 +32,9 @@ import java.util.concurrent.CountDownLatch;
|
|||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.FilterCodec;
|
||||
import org.apache.lucene.codecs.KnnVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene99.Lucene99HnswScalarQuantizedVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsReader;
|
||||
import org.apache.lucene.codecs.lucene99.vectors.graph.Lucene99HnswScalarQuantizedVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene99.vectors.graph.Lucene99HnswVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene99.vectors.graph.Lucene99HnswVectorsReader;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
|
|
|
@ -41,8 +41,8 @@ import java.util.stream.Collectors;
|
|||
import org.apache.lucene.codecs.FilterCodec;
|
||||
import org.apache.lucene.codecs.KnnVectorsFormat;
|
||||
import org.apache.lucene.codecs.hnsw.DefaultFlatVectorScorer;
|
||||
import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsReader;
|
||||
import org.apache.lucene.codecs.lucene99.vectors.graph.Lucene99HnswVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene99.vectors.graph.Lucene99HnswVectorsReader;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
|
|
|
@ -18,7 +18,7 @@ package org.apache.lucene.tests.codecs.vector;
|
|||
|
||||
import org.apache.lucene.codecs.FilterCodec;
|
||||
import org.apache.lucene.codecs.KnnVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene99.vectors.graph.Lucene99HnswVectorsFormat;
|
||||
import org.apache.lucene.tests.util.TestUtil;
|
||||
|
||||
/**
|
||||
|
|
|
@ -57,7 +57,7 @@ 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.lucene99.Lucene99Codec;
|
||||
import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene99.vectors.graph.Lucene99HnswVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
|
||||
|
|
Loading…
Reference in New Issue