forked from apache/lucene
-
Notifications
You must be signed in to change notification settings - Fork 0
Add TwoPhaseKnnVectorQuery #29
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Open
dungba88
wants to merge
26
commits into
main
Choose a base branch
from
two-phase-vector
base: main
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Open
Changes from 9 commits
Commits
Show all changes
26 commits
Select commit
Hold shift + click to select a range
edef1b2
Initial commit: Add TwoPhaseKnnVectorQuery
dungba88 bbc7081
Add tests
dungba88 96d2987
Remove forbidden API
dungba88 e2ab4bc
Remove forbidden API
dungba88 4e32971
Add javadoc
dungba88 ccd3e25
Make the Query experimental
dungba88 f9da336
Use Math.ceil instead of rounding
dungba88 8d88cab
Store target separately in child class
dungba88 b67637a
Change abstraction to wrap around KNN query
dungba88 8cd3ccf
Fix doc ord bug & flush writer multiple times
dungba88 30e377a
Add null check
dungba88 5d1910c
Refactor test case
dungba88 22288e5
Merge branch 'main' into two-phase-vector
dungba88 feda6af
Simplify Codec
dungba88 3178bbc
short-circuit for case there is no oversample
dungba88 4eb019d
Merge branch 'main' into two-phase-vector
dungba88 b3b58fd
Make inner Query generic
dungba88 15415dc
Use DoubleValueSource for generic re-scoring
dungba88 71f3a94
Add license
dungba88 d3d744c
Add change log
dungba88 307f468
Merge branch 'main' into two-phase-vector
dungba88 f65f80a
Address comments
dungba88 b9554ed
Add license
dungba88 6a0572a
fix comment
dungba88 8812731
Update Changes to target 10.3
vigyasharma da96929
lint fix
vigyasharma File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
110 changes: 110 additions & 0 deletions
110
lucene/core/src/java/org/apache/lucene/search/RerankKnnFloatVectorQuery.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,110 @@ | ||
| /* | ||
| * 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.search; | ||
|
|
||
| import static org.apache.lucene.search.AbstractKnnVectorQuery.createRewrittenQuery; | ||
|
|
||
| import java.io.IOException; | ||
| import java.util.Arrays; | ||
| import java.util.Objects; | ||
| import org.apache.lucene.index.FieldInfo; | ||
| import org.apache.lucene.index.FloatVectorValues; | ||
| import org.apache.lucene.index.IndexReader; | ||
| import org.apache.lucene.index.VectorSimilarityFunction; | ||
|
|
||
| /** | ||
| * A wrapper of KnnFloatVectorQuery which does full-precision reranking. | ||
| * | ||
| * @lucene.experimental | ||
| */ | ||
| public class RerankKnnFloatVectorQuery extends Query { | ||
|
|
||
| private final int k; | ||
| private final float[] target; | ||
| private final KnnFloatVectorQuery query; | ||
|
|
||
| /** | ||
| * Execute the KnnFloatVectorQuery and re-rank using full-precision vectors | ||
| * | ||
| * @param query the KNN query to execute as initial phase | ||
| * @param target the target of the search | ||
| * @param k the number of documents to find | ||
| * @throws IllegalArgumentException if <code>k</code> is less than 1 | ||
| */ | ||
| public RerankKnnFloatVectorQuery(KnnFloatVectorQuery query, float[] target, int k) { | ||
| this.query = query; | ||
| this.target = target; | ||
| this.k = k; | ||
| } | ||
|
|
||
| @Override | ||
| public Query rewrite(IndexSearcher indexSearcher) throws IOException { | ||
| IndexReader reader = indexSearcher.getIndexReader(); | ||
| Query rewritten = indexSearcher.rewrite(query); | ||
| Weight weight = indexSearcher.createWeight(rewritten, ScoreMode.COMPLETE_NO_SCORES, 1.0f); | ||
| HitQueue queue = new HitQueue(k, false); | ||
| for (var leaf : reader.leaves()) { | ||
| Scorer scorer = weight.scorer(leaf); | ||
| if (scorer == null) { | ||
| continue; | ||
| } | ||
| FloatVectorValues floatVectorValues = leaf.reader().getFloatVectorValues(query.getField()); | ||
| if (floatVectorValues == null) { | ||
| continue; | ||
| } | ||
| FieldInfo fi = leaf.reader().getFieldInfos().fieldInfo(query.getField()); | ||
| VectorSimilarityFunction comparer = fi.getVectorSimilarityFunction(); | ||
| DocIdSetIterator iterator = scorer.iterator(); | ||
| while (iterator.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) { | ||
| int docId = iterator.docID(); | ||
| float[] vectorValue = floatVectorValues.vectorValue(docId); | ||
| float score = comparer.compare(vectorValue, target); | ||
| queue.insertWithOverflow(new ScoreDoc(docId, score)); | ||
| } | ||
| } | ||
| int i = 0; | ||
| ScoreDoc[] scoreDocs = new ScoreDoc[queue.size()]; | ||
| for (ScoreDoc topDoc : queue) { | ||
| scoreDocs[i++] = topDoc; | ||
| } | ||
| return createRewrittenQuery(reader, scoreDocs); | ||
| } | ||
|
|
||
| @Override | ||
| public int hashCode() { | ||
| int result = Arrays.hashCode(target); | ||
| result = 31 * result + Objects.hash(query, k); | ||
| return result; | ||
| } | ||
|
|
||
| @Override | ||
| public boolean equals(Object o) { | ||
| if (this == o) return true; | ||
| RerankKnnFloatVectorQuery that = (RerankKnnFloatVectorQuery) o; | ||
| return Objects.equals(query, that.query) && k == that.k; | ||
| } | ||
|
|
||
| @Override | ||
| public void visit(QueryVisitor visitor) { | ||
| query.visit(visitor); | ||
| } | ||
|
|
||
| @Override | ||
| public String toString(String field) { | ||
| return getClass().getSimpleName() + ":" + query.toString(field) + "[" + k + "]"; | ||
| } | ||
| } |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
126 changes: 126 additions & 0 deletions
126
lucene/core/src/test/org/apache/lucene/search/TestRerankKnnFloatVectorQuery.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,126 @@ | ||
| /* | ||
| * 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.search; | ||
|
|
||
| import java.util.HashMap; | ||
| import java.util.Map; | ||
| import java.util.Random; | ||
| import org.apache.lucene.codecs.FilterCodec; | ||
| import org.apache.lucene.codecs.KnnVectorsFormat; | ||
| import org.apache.lucene.codecs.lucene100.Lucene100Codec; | ||
| import org.apache.lucene.codecs.lucene99.Lucene99HnswScalarQuantizedVectorsFormat; | ||
| import org.apache.lucene.document.Document; | ||
| import org.apache.lucene.document.Field; | ||
| import org.apache.lucene.document.IntField; | ||
| import org.apache.lucene.document.KnnFloatVectorField; | ||
| import org.apache.lucene.index.DirectoryReader; | ||
| import org.apache.lucene.index.IndexReader; | ||
| import org.apache.lucene.index.IndexWriter; | ||
| import org.apache.lucene.index.IndexWriterConfig; | ||
| import org.apache.lucene.index.VectorSimilarityFunction; | ||
| import org.apache.lucene.store.ByteBuffersDirectory; | ||
| import org.apache.lucene.store.Directory; | ||
| import org.apache.lucene.tests.util.LuceneTestCase; | ||
| import org.junit.Assert; | ||
| import org.junit.Before; | ||
| import org.junit.Test; | ||
|
|
||
| public class TestRerankKnnFloatVectorQuery extends LuceneTestCase { | ||
|
|
||
| private static final String FIELD = "vector"; | ||
| public static final VectorSimilarityFunction VECTOR_SIMILARITY_FUNCTION = | ||
| VectorSimilarityFunction.COSINE; | ||
| private Directory directory; | ||
| private IndexWriterConfig config; | ||
| private static final int NUM_VECTORS = 1000; | ||
| private static final int VECTOR_DIMENSION = 128; | ||
|
|
||
| @Before | ||
| @Override | ||
| public void setUp() throws Exception { | ||
| super.setUp(); | ||
| directory = new ByteBuffersDirectory(); | ||
|
|
||
| // Set up the IndexWriterConfig to use quantized vector storage | ||
| config = new IndexWriterConfig(); | ||
| config.setCodec(new QuantizedCodec()); | ||
| } | ||
|
|
||
| @Test | ||
| public void testTwoPhaseKnnVectorQuery() throws Exception { | ||
| Map<Integer, float[]> vectors = new HashMap<>(); | ||
|
|
||
| Random random = random(); | ||
|
|
||
| // Step 1: Index random vectors in quantized format | ||
| try (IndexWriter writer = new IndexWriter(directory, config)) { | ||
| for (int i = 0; i < NUM_VECTORS; i++) { | ||
| float[] vector = randomFloatVector(VECTOR_DIMENSION, random); | ||
| Document doc = new Document(); | ||
| doc.add(new IntField("id", i, Field.Store.YES)); | ||
| doc.add(new KnnFloatVectorField(FIELD, vector, VECTOR_SIMILARITY_FUNCTION)); | ||
| writer.addDocument(doc); | ||
| vectors.put(i, vector); | ||
| } | ||
| } | ||
|
|
||
| // Step 2: Run TwoPhaseKnnVectorQuery with a random target vector | ||
| try (IndexReader reader = DirectoryReader.open(directory)) { | ||
| IndexSearcher searcher = new IndexSearcher(reader); | ||
| float[] targetVector = randomFloatVector(VECTOR_DIMENSION, random); | ||
| int k = 10; | ||
| double oversample = 1.0; | ||
|
|
||
| KnnFloatVectorQuery knnQuery = | ||
| new KnnFloatVectorQuery(FIELD, targetVector, k + (int) (k * oversample)); | ||
| RerankKnnFloatVectorQuery query = new RerankKnnFloatVectorQuery(knnQuery, targetVector, k); | ||
| TopDocs topDocs = searcher.search(query, k); | ||
|
|
||
| // Step 3: Verify that TopDocs scores match similarity with unquantized vectors | ||
| for (ScoreDoc scoreDoc : topDocs.scoreDocs) { | ||
| Document retrievedDoc = searcher.storedFields().document(scoreDoc.doc); | ||
| float[] docVector = vectors.get(retrievedDoc.getField("id").numericValue().intValue()); | ||
| float expectedScore = VECTOR_SIMILARITY_FUNCTION.compare(targetVector, docVector); | ||
| Assert.assertEquals( | ||
| "Score does not match expected similarity for docId: " + scoreDoc.doc, | ||
| expectedScore, | ||
| scoreDoc.score, | ||
| 1e-5); | ||
| } | ||
| } | ||
| } | ||
|
|
||
| private float[] randomFloatVector(int dimension, Random random) { | ||
| float[] vector = new float[dimension]; | ||
| for (int i = 0; i < dimension; i++) { | ||
| vector[i] = random.nextFloat(); | ||
| } | ||
| return vector; | ||
| } | ||
|
|
||
| public static class QuantizedCodec extends FilterCodec { | ||
|
|
||
| public QuantizedCodec() { | ||
| super("QuantizedCodec", new Lucene100Codec()); | ||
| } | ||
|
|
||
| @Override | ||
| public KnnVectorsFormat knnVectorsFormat() { | ||
| return new Lucene99HnswScalarQuantizedVectorsFormat(); | ||
| } | ||
| } | ||
| } | ||
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Maybe we should flush the writers a few time to create more than one segments, this make sure the code path that results are merged from multiple segments are executed.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Good points, I added the flush() and uncovered/fixed a bug with doc ord.