You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by "alessandrobenedetti (via GitHub)" <gi...@apache.org> on 2023/05/19 15:46:48 UTC

[GitHub] [lucene] alessandrobenedetti opened a new pull request, #12314: Multi-value support for KnnVectorField

alessandrobenedetti opened a new pull request, #12314:
URL: https://github.com/apache/lucene/pull/12314

   ### Description
   This pull request aims to introduce support for multiple values in a single Knn vector field.
   The adopted solution relies on:
   **Index time**
   Sparse vector values approach where an Ordinal(vectorId) to DocId map is used to keep the relation between a DocId and all its vectors.
   In the current sparse vector approach, we have just one vectorId per docID
   In this proposed contribution, multiple vectorIds are mapped to the same docID
   **Query time**
   A multi-valued strategy choice is offered to the user:
   MAX/SUM
   In exact nearest neighbor, for each document accepted by the query/filter :
   MAX = the similarity score between the query and each vector is computed, the max score is chosen for the search result
   SUM = the similarity score between the query and each vector is computed, all scores are summed to get the final score 
   
   In aproximate nearest neighbor, for each document accepted by the query/filter :
   MAX = every time we find a nearest neighbor vector to be added to the topK, if the document is already there, its score is updated keeping the maximum between what it was there and the new score
   SUM = every time we find a nearest neighbor vector to be added to the topK, if the document is already there, its score is updated summing the old and new score
   
   N.B. This Pull Request is not meant to be ready to be merged at this stage.
   I can identify at least this set of activities before this draft can move to a 'production ready' version:
   
   1) validate the overall idea and approach
   2) validate index time usage of sparse vector values for the multi-valued use case
   3) validate merge policy for the multi-valued use case
   4) validate query time MAX/SUM approach
   5) validate query time modifiable heap and neighborQueue usage
   6) validate regressions
   7) introduce more tests
   
   It's a big contribution and It will take time and effort to be completed.
   Any help is welcome.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] alessandrobenedetti commented on pull request #12314: Multi-value support for KnnVectorField

Posted by "alessandrobenedetti (via GitHub)" <gi...@apache.org>.
alessandrobenedetti commented on PR #12314:
URL: https://github.com/apache/lucene/pull/12314#issuecomment-1561429217

   I have been working a bit on the removal of the need for "vectorMultiValued" as a special fieldInfo and index time information.
   I think the results are encouraging, the current tests are relatively stable and diff went down to 'only' 27 files (we started with 85!).
   I'll keep cleaning it up and refine it, the necessity of additional pair of eyes for the review is still there as I am not convinced some variables are in the best place (multiValued in the vector values and graph) but it's progressing.
   
   I may have also brought regressions with this simplifications, but everything is in the git history so it should be doable to fix them.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on pull request #12314: Multi-value support for KnnVectorField

Posted by "mayya-sharipova (via GitHub)" <gi...@apache.org>.
mayya-sharipova commented on PR #12314:
URL: https://github.com/apache/lucene/pull/12314#issuecomment-1563043435

   @alessandrobenedetti Thanks for starting this effort. We also found a need for multi-value vector fields, and here is the list of questions we thought about for its usage:
   
   1.  Is the main usage for breaking a long text into several paragraphs? Or is it also to search across several different fields (e.g. [embedding_of_title, embedding_of_content]) where we create a single graph for several fields?
   2. Would it be possible to retrieve which vector was the closest match? For example, if we break a long text into paragraphs and want to highlight which paragraph was the closest match. This could be crucial for some use case.
   3. An extension to question 2, can we support some metadata attached to vector values?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] alessandrobenedetti commented on pull request #12314: Multi-value support for KnnVectorField

Posted by "alessandrobenedetti (via GitHub)" <gi...@apache.org>.
alessandrobenedetti commented on PR #12314:
URL: https://github.com/apache/lucene/pull/12314#issuecomment-1558921019

   Thanks @benwtrent and no rush, I am not planning to commit to this work anytime soon.
   I'll need various validations from other committers on the various areas of the code touched and also spend some additional time on my own on this.
   Any feedback will be super welcome as this is a critical new functionality.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] uschindler commented on a diff in pull request #12314: Multi-value support for KnnVectorField

Posted by "uschindler (via GitHub)" <gi...@apache.org>.
uschindler commented on code in PR #12314:
URL: https://github.com/apache/lucene/pull/12314#discussion_r1246177620


##########
lucene/core/src/java/org/apache/lucene/index/DocsWithVectorsSet.java:
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.index;
+
+import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.RamUsageEstimator;
+
+import java.util.Stack;

Review Comment:
   Please don't use the outdated `Stack` class, which is also synchronized. Modern replacement is `ArrayDeque` implementing `Deque` interface.



##########
lucene/core/src/java/org/apache/lucene/index/FloatVectorValues.java:
##########
@@ -57,4 +63,13 @@ public final long cost() {
    * @return the vector value
    */
   public abstract float[] vectorValue() throws IOException;
+
+  /**
+   * Return the document ID corresponding to the input vector id(ordinal)
+   * @param ord vector ID(ordinal)
+   * @return the document ID
+   */
+  public int ordToDoc(int ord){

Review Comment:
   The problem with the new implementation is that ordinal numbers need to get longs (like in SortedSetDocValues), because as each document of a 2B docs index sement may have more than one value, so the total number of ordinals may go beyond 2B.
   
   Of course this needs more work.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] alessandrobenedetti commented on pull request #12314: Multi-value support for KnnVectorField

Posted by "alessandrobenedetti (via GitHub)" <gi...@apache.org>.
alessandrobenedetti commented on PR #12314:
URL: https://github.com/apache/lucene/pull/12314#issuecomment-1554825253

   I'll follow up with many clean up and tidy up on my own in the next few weeks.
   I should have a bit of bandwidth from now till Berlin Buzzword (mid June).
   Any feedback is welcome, I am pretty sure there are a *lot* of wrong/imperfect stuff around.
   I worked on this for many months, in my spare time, I probably spent most of my time merging from the main as this area changed a lot :)
   Didn't bother with pre-commit and stuff, I'll do it in the next few days.
   @tang-hi the idea is to allow multiple vectors in a field, sometimes you have multiple paragraphs and long texts you may want to split before encoding them to vectors, rather than concatenate everything to a single vector


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] alessandrobenedetti commented on a diff in pull request #12314: Multi-value support for KnnVectorField

Posted by "alessandrobenedetti (via GitHub)" <gi...@apache.org>.
alessandrobenedetti commented on code in PR #12314:
URL: https://github.com/apache/lucene/pull/12314#discussion_r1238063391


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphSearcher.java:
##########
@@ -248,35 +247,41 @@ public NeighborQueue searchLevel(
       RandomAccessVectorValues<T> vectors,
       HnswGraph graph)
       throws IOException {
-    return searchLevel(query, topK, level, eps, vectors, graph, null, Integer.MAX_VALUE);
+    return searchLevel(query, topK, level, eps, vectors, graph, null, Integer.MAX_VALUE, false);
   }
 
   private NeighborQueue searchLevel(
-      T query,
-      int topK,
-      int level,
-      final int[] eps,
-      RandomAccessVectorValues<T> vectors,
-      HnswGraph graph,
-      Bits acceptOrds,
-      int visitedLimit)
-      throws IOException {
+          T query,
+          int topK,
+          int level,
+          final int[] entryPoints,
+          RandomAccessVectorValues<T> vectors,
+          HnswGraph graph,
+          Bits acceptOrds,
+          int visitedLimit,
+          boolean multiValued)
+          throws IOException {
     int size = graph.size();
     NeighborQueue results = new NeighborQueue(topK, false);
     prepareScratchState(vectors.size());
 
-    int numVisited = 0;
-    for (int ep : eps) {
-      if (visited.getAndSet(ep) == false) {
-        if (numVisited >= visitedLimit) {
+    int vectorIdsVisited = 0;
+    for (int vectorId : entryPoints) {
+      if (visited.getAndSet(vectorId) == false) {
+        if (vectorIdsVisited >= visitedLimit) {
           results.markIncomplete();
           break;
         }
-        float score = compare(query, vectors, ep);
-        numVisited++;
-        candidates.add(ep, score);
-        if (acceptOrds == null || acceptOrds.get(ep)) {
-          results.add(ep, score);
+        float score = compare(query, vectors, vectorId);
+        vectorIdsVisited++;
+        candidates.add(vectorId, score);
+        int docId = vectors.ordToDoc(vectorId);
+        if (acceptOrds == null || acceptOrds.get(vectorId)) {

Review Comment:
   nice catch @benwtrent, I reviewed the commit history and the change got lost in one of the thousands merges.
   I've restored it to the correct form acceptOrds.get(docId) now!
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] alessandrobenedetti commented on a diff in pull request #12314: Multi-value support for KnnVectorField

Posted by "alessandrobenedetti (via GitHub)" <gi...@apache.org>.
alessandrobenedetti commented on code in PR #12314:
URL: https://github.com/apache/lucene/pull/12314#discussion_r1206533493


##########
lucene/core/src/java/org/apache/lucene/index/DocsWithFieldSet.java:
##########
@@ -32,8 +34,14 @@ public final class DocsWithFieldSet extends DocIdSet {
       RamUsageEstimator.shallowSizeOfInstance(DocsWithFieldSet.class);
 
   private FixedBitSet set;
-  private int cardinality = 0;
-  private int lastDocId = -1;
+  private int docsCount = 0;
+  private int lastDocId = 0; // at a certain point in time this was changed to 0? why?
+  
+  private Stack<Integer> valuesPerDocuments;
+  private int currentDocVectorsCount;
+  private int vectorsCount;
+  
+  private boolean multiValued = false;

Review Comment:
   I don't have a strong opinion about this, it's a simple change though if necessary



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] alessandrobenedetti commented on a diff in pull request #12314: Multi-value support for KnnVectorField

Posted by "alessandrobenedetti (via GitHub)" <gi...@apache.org>.
alessandrobenedetti commented on code in PR #12314:
URL: https://github.com/apache/lucene/pull/12314#discussion_r1244453767


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/Lucene95HnswVectorsWriter.java:
##########
@@ -822,40 +836,37 @@ private void writeMeta(
    * Writes the byte vector values to the output and returns a set of documents that contains
    * vectors.
    */
-  private static DocsWithFieldSet writeByteVectorData(
-      IndexOutput output, ByteVectorValues byteVectorValues) throws IOException {
-    DocsWithFieldSet docsWithField = new DocsWithFieldSet();
-    for (int docV = byteVectorValues.nextDoc();
-        docV != NO_MORE_DOCS;
-        docV = byteVectorValues.nextDoc()) {
-      // write vector
-      byte[] binaryValue = byteVectorValues.vectorValue();
-      assert binaryValue.length == byteVectorValues.dimension() * VectorEncoding.BYTE.byteSize;
+  private static DocsWithVectorsSet writeByteVectorData(
+      IndexOutput output, ByteVectorValues mergedVectorValues) throws IOException {
+    DocsWithVectorsSet docsWithVectors = new DocsWithVectorsSet();
+    for (int vectorId = mergedVectorValues.nextDoc(); vectorId != NO_MORE_DOCS; vectorId = mergedVectorValues.nextDoc()) {
+      int docID = mergedVectorValues.ordToDoc(vectorId);
+      byte[] binaryValue = mergedVectorValues.vectorValue();
+      assert binaryValue.length == mergedVectorValues.dimension() * VectorEncoding.BYTE.byteSize;
       output.writeBytes(binaryValue, binaryValue.length);
-      docsWithField.add(docV);
+      docsWithVectors.add(docID);
     }
-    return docsWithField;
+    return docsWithVectors;

Review Comment:
   As far as I wanted and hopefully implemented we get the document order for free since all vectors have to be supplied at the same time.
   As everything in this Pull Request, it's open to discussion as my focus was to bring a starting milestone working end to end and then refine each step.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] alessandrobenedetti commented on pull request #12314: Multi-value support for KnnVectorField

Posted by "alessandrobenedetti (via GitHub)" <gi...@apache.org>.
alessandrobenedetti commented on PR #12314:
URL: https://github.com/apache/lucene/pull/12314#issuecomment-1602090466

   > Thinking more on this implementation. It seems like we will need at a minimum a new `NeighborQueue`
   > 
   > I am not sure the existing one needs to be updated, but we instead should have a `MultiValueNeighborQueue`.
   > 
   > The reason for this is that not only does this queue contain information about result sets, it keeps track of how many nodes are visited and the TopHits returned utilize that number. Consequently, the visited count should keep track of documents visited, not vectors visited. All these changes indicates a new queueing mechanism for multi-valued vector fields.
   > 
   > Another thought is that Lucene already has the concept of index `join` values. Effectively creating child document IDs under a single parent. This allows for even greater flexibility by indexing the passage the vector represents, and potentially even greater scoring flexibility.
   > 
   > The issue I could see happening here is ensuring the topdocs searching has the ability to deduplicate (if desired) based on parent document ID.
   > 
   > Did you consider utilizing this when digging into this implementation?
   
   I think it's a good idea to create a new dedicated MultiValued NeighborQueue, I'll do it when I have time but feel free to do it if you like!
   
   In regards to index time join, I am not sure it's relevant here (are you talking about block join?):
   isn't it a different concept from multivalued?
   i.e. we have the mechanism in Lucene along multi-valued vectors for pretty much all the field types, haven't we?
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] benwtrent commented on a diff in pull request #12314: Multi-value support for KnnVectorField

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12314:
URL: https://github.com/apache/lucene/pull/12314#discussion_r1243611229


##########
lucene/core/src/java/org/apache/lucene/search/VectorScorer.java:
##########
@@ -85,6 +96,26 @@ public boolean advanceExact(int doc) throws IOException {
       return vectorDoc == doc;
     }
 
+    @Override
+    public Map<Integer, Float> scoreMultiValued(BitSet acceptedDocs) throws IOException {
+      Map<Integer, Float> docToScore = new HashMap<>();
+      for (int vectorId = values.nextDoc(); vectorId != NO_MORE_DOCS; vectorId = values.nextDoc()) {
+        int docID = values.ordToDoc(vectorId);
+        if (acceptedDocs.get(docID)) {
+          float currentScore = similarity.compare(query, values.vectorValue());
+          docToScore.putIfAbsent(docID, currentScore);
+          docToScore.computeIfPresent(docID,
+                  (key, previousScore) -> Math.max(previousScore, currentScore));
+        }
+      }
+      return docToScore;
+    }

Review Comment:
   I don't see how we can do this here. Instead of randomly iterating the BitSet and going to the exact document, we are iterating every vector in the scorer and checking its membership in `acceptedDocs`. 
   
   Even multi-valued `*VectorValues` classes need to be able to iterate to the exact document. Otherwise searching with a filter like this is untenable.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] alessandrobenedetti commented on pull request #12314: Multi-value support for KnnVectorField

Posted by "alessandrobenedetti (via GitHub)" <gi...@apache.org>.
alessandrobenedetti commented on PR #12314:
URL: https://github.com/apache/lucene/pull/12314#issuecomment-1563074046

   > Would be nice to get yours and other's ideas on them:
   > 
   > 1. Is the main usage for breaking a long text into several paragraphs? Or is it also to search across several different fields (e.g. [embedding_of_title, embedding_of_content]) where we create a single graph for several fields?
   I would say the first, I have seen examples of that, especially when the large language model chosen by the customer has a limit in input tokens smaller than the document length for the customer.
   A single graph over several fields would be the equivalent of a catch-all field for the lexical search, it was not my primary focus but should be doable once we have multi-valued fields.
   > 2. Would it be possible to retrieve which vector was the closest match? For example, if we break a long text into paragraphs and want to highlight which paragraph was the closest match. This could be crucial for some use cases.
   I agree, that can be a nice addition to the explain!
   
   I guess attaching metadata to vectors is a different story, but I agree it could be a good idea!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] alessandrobenedetti commented on pull request #12314: Multi-value support for KnnVectorField

Posted by "alessandrobenedetti (via GitHub)" <gi...@apache.org>.
alessandrobenedetti commented on PR #12314:
URL: https://github.com/apache/lucene/pull/12314#issuecomment-1564579302

   > My main worry is the change to `FloatVectorValue`, moving to a multivalued iterator changes the access pattern so I don't find it right to change the interface and the meaning of the ordinals that are returned based on multivalued or not.
   > If only `search` was exposed in the format that would be ok I think but we're exposing direct access to the document's vector so the parallel with doc values is important
   
   Hi @jimczi, nothing in this PR is final nor I have any strong opinion about it.
   My main intention is to keep the PR as small and as valuable as possible, to build a common ground (and tests) to build the functionality (if nice to have, if not, it was a cool exercise and that's equally fine).
   
   In regards to your main worry, can you point me to the areas of code you don't like specifically and I can have a thought in how to modify them!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] alessandrobenedetti commented on pull request #12314: Multi-value support for KnnVectorField

Posted by "alessandrobenedetti (via GitHub)" <gi...@apache.org>.
alessandrobenedetti commented on PR #12314:
URL: https://github.com/apache/lucene/pull/12314#issuecomment-1557354463

   I pushed a commit with the query time simplification (only MAX strategy is supported).
   The diff is simpler but I am not convinced it's better.
   I also remembered a bit more why I chose to explicitly differentiate Index/query time behavior for multi-valued search:
   The main reason is that at the moment the multi-valued approach requires some extra work both at indexing and query time, from data structures and processing perspective.
   With a clear distinction, I tried to impact the single-valued approach as less as possible (literally using different code when multi-valued occurs).
   
   For example, at query time this is visible in the org.apache.lucene.util.hnsw.NeighborQueue.
   With the latest commit, I am pretty sure single valued search gets a performance regression (that shouldn't be the case with the 'NONE' strategy as it was before).
   I anyway think it was a good exercise to do the simplification and we can iterate/revert from there.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on a diff in pull request #12314: Multi-value support for KnnVectorField

Posted by "mayya-sharipova (via GitHub)" <gi...@apache.org>.
mayya-sharipova commented on code in PR #12314:
URL: https://github.com/apache/lucene/pull/12314#discussion_r1205995223


##########
lucene/core/src/java/org/apache/lucene/index/DocsWithFieldSet.java:
##########
@@ -22,6 +22,8 @@
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.RamUsageEstimator;
 
+import java.util.Stack;

Review Comment:
   I think we should keep this class unchanged; it is used in other places besides vectors and is only about documents.
   
   For keeping track the number of values in docs, may be we can use another data-structure (`PackedLongValues.Builder`, similar how `SortedSetDocValuesWriter` does it)?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on a diff in pull request #12314: Multi-value support for KnnVectorField

Posted by "mayya-sharipova (via GitHub)" <gi...@apache.org>.
mayya-sharipova commented on code in PR #12314:
URL: https://github.com/apache/lucene/pull/12314#discussion_r1206004550


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/Lucene95HnswVectorsWriter.java:
##########
@@ -762,13 +776,18 @@ private void writeMeta(
       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);
+          DirectMonotonicWriter.getInstance(meta, vectorData, vectorsCount, DIRECT_MONOTONIC_BLOCK_SHIFT);
       DocIdSetIterator iterator = docsWithField.iterator();
+      int[] valuesPerDocument = docsWithField.getValuesPerDocument();

Review Comment:
   This seems to use a lot of memory, may be for inspiration, we can use how `SortedSetDocValuesWriter` class writes multiple values.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] alessandrobenedetti commented on pull request #12314: Multi-value support for KnnVectorField

Posted by "alessandrobenedetti (via GitHub)" <gi...@apache.org>.
alessandrobenedetti commented on PR #12314:
URL: https://github.com/apache/lucene/pull/12314#issuecomment-1557458202

   Adding the information that a graph was built multi-valued, in the segment allows a check at query time to differentiate the single values vs multi-valued approach.
   Not a big fan of how I quickly implemented it, but could be a viable way of keeping it simple at the beginning (no strategies, only MAX supported).
   It would be easy to add the different strategies later (MAX/SUM) and right now the diff is 65 classes (vs initial 85)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] alessandrobenedetti commented on pull request #12314: Multi-value support for KnnVectorField

Posted by "alessandrobenedetti (via GitHub)" <gi...@apache.org>.
alessandrobenedetti commented on PR #12314:
URL: https://github.com/apache/lucene/pull/12314#issuecomment-1568397130

   > > SUM = the similarity score between the query and each vector is computed, all scores are summed to get the final score
   > > SUM = every time we find a nearest neighbor vector to be added to the topK, if the document is already there, its score is updated summing the old and new score
   > 
   > Just a note on the aggregation functions `max` and `sum`. Most commonly it seems that `max` is used as it is length independent. When using `sum`, the longer the original text of a document field, and thus the more passages it will have, the higher the `sum` of all matching passages will be since all passages will "match", thus biasing scoring towards documents with longer text. I'm not sure if it will matter in the end, but my suggestion would be that if `sum` is used, one could optionally use a radius/similarity threshold to limit the advantage of longer texts, and/or allow using just a limited top-k passages of a document for `sum`.
   > 
   > @alessandrobenedetti Do you have any good references/papers on approaches to re-aggregating passages into documents for SERPs? It seems that the art was abandoned a couple years ago with most approaches settling on `max` passage (which I see is the only method implemented for now).
   
   Hi @joshdevins ,
   the dual strategy(MAX/SUM) is implemented in an old commit, I removed it to build an initial smaller and cleaner pull request.
   Some of the feedback was to introduce strategies later on, and that's fine, I agree with that.
   
   I didn't have the time to dive deeper into the aggregation strategies so I don't have references yet.
   My main focus was to reach a working prototype and then iterate on the various components to make them better/deal.
   
   Your observation regarding 'SUM' is correct.
   In my 'naive' first implementation I used an approach where only the closer vectors you encounter are considered in the SUM when running an approximate search.
   You can take a look at the commits before the simplification if you are curious, but I believe it would be better to address this discussion when we introduce 'Strategies' again in a separate future PR.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] uschindler commented on a diff in pull request #12314: Multi-value support for KnnVectorField

Posted by "uschindler (via GitHub)" <gi...@apache.org>.
uschindler commented on code in PR #12314:
URL: https://github.com/apache/lucene/pull/12314#discussion_r1246537955


##########
lucene/core/src/java/org/apache/lucene/index/DocsWithVectorsSet.java:
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.index;
+
+import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.RamUsageEstimator;
+
+import java.util.Stack;

Review Comment:
   I opened #12404 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] benwtrent commented on pull request #12314: Multi-value support for KnnVectorField

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on PR #12314:
URL: https://github.com/apache/lucene/pull/12314#issuecomment-1558005668

   Thank you for the simplification! I will take another look in about 2 weeks. I am on a cross country camping trip :).
   
   There is a ton of good work in this PR. Excited to get this in Lucene!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] alessandrobenedetti commented on a diff in pull request #12314: Multi-value support for KnnVectorField

Posted by "alessandrobenedetti (via GitHub)" <gi...@apache.org>.
alessandrobenedetti commented on code in PR #12314:
URL: https://github.com/apache/lucene/pull/12314#discussion_r1206532595


##########
lucene/core/src/java/org/apache/lucene/index/DocsWithFieldSet.java:
##########
@@ -22,6 +22,8 @@
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.RamUsageEstimator;
 
+import java.util.Stack;

Review Comment:
   I agree, for the time being I reverted the class and extended it. Feel free to have a look and apply/propose any change you see fit.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] jimczi commented on pull request #12314: Multi-value support for KnnVectorField

Posted by "jimczi (via GitHub)" <gi...@apache.org>.
jimczi commented on PR #12314:
URL: https://github.com/apache/lucene/pull/12314#issuecomment-1564208967

   > That was the initial approach, it was explicit at index and query time, and they were separate code paths from the single valued use case. So it was not affecting the single valued scenario at all.
   Following some of the feedback here I moved to transparent approach, that prooved extremely beneficial in reducing the complexity of the pull request (at the cost of impacting potentially some single valued case scenario).
   I iterated a bit, so I should have reduced the impact already on the single valued case, but not to 100% I guess.
   
   My main worry is the change to `FloatVectorValue`, moving to a multivalued iterator changes the access pattern so I don't find it right to change the interface and the meaning of the ordinals that are returned based on multivalued or not.
   If only `search` was exposed in the format that would be ok I think but we're exposing direct access to the document's vector so the parallel with doc values is important imo. 
   
   > Yes, with one vector per document, the maximum amount of supported vectors was in line with the docs, this is still the case but I agree that right now we potentially support less docs, happy to change that.
   
   Well I don't have a good idea to change that if we expect to have an ordinal per value. Switching the ordinal in the hnsw to int64 is not really practical. Adding a sub-id to each ordinal seems also quite invasive. Again not saying that it should block the change but we need to be clear on the downside of this approach and also ensures that we have the right guards for overflows.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] alessandrobenedetti commented on pull request #12314: Multi-value support for KnnVectorField

Posted by "alessandrobenedetti (via GitHub)" <gi...@apache.org>.
alessandrobenedetti commented on PR #12314:
URL: https://github.com/apache/lucene/pull/12314#issuecomment-1564148395

   I proceeded with some additional refactoring and refinements that find in the latest commits.
   The diff is down to 25 classes, query time has been simplified, and explicit multi-valued has been moved to transparent multi-valued.
   I am close to exhausting the funds (fully self-funded at the moment) my company has allocated to the project, but I'll be happy to continue with occasional discussions and reviews.
   If we get any external funds to continue the project, I'll let you know here.
   I'll follow up to a response to @jimczi in a next comment.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] benwtrent commented on pull request #12314: Multi-value support for KnnVectorField

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on PR #12314:
URL: https://github.com/apache/lucene/pull/12314#issuecomment-1554999300

   @alessandrobenedetti thank you for kick starting this!
   
   You are absolutely correct, this is a large, but pivotal and necessary change for vector search in Lucene. I have not yet reviewed or looked over the entire design, but two things stuck out to me immediately.
   
   1. To simplify the review, design, testing, etc. could you simplify and reduce scope? Meaning, for the first iteration of this, Lucene only supports `max` and is used by default for multi-vector fields. This should:
       - remove the need for the additional search parameter (can be added as a part 2 of this feature in the future).
       - reduce testing requirements
       - reduce user overhead as the closest document based on the closest vector is returned. This behavior parallels nicely with a single vector field.
   2. Declaring a vector field as "multi-vector" seems...weird and has a very blast radius on the entire code base. It seems to me that the vector codec should seamlessly handle multiple vectors per field, just like it seamlessly handles sparse vector encoding. Though, thinking about it more, it seems difficult to support both sparse & multi-vector values in this way. All that said, I am curious about to your reasoning here.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] benwtrent commented on a diff in pull request #12314: Multi-value support for KnnVectorField

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12314:
URL: https://github.com/apache/lucene/pull/12314#discussion_r1236994893


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphSearcher.java:
##########
@@ -248,35 +247,41 @@ public NeighborQueue searchLevel(
       RandomAccessVectorValues<T> vectors,
       HnswGraph graph)
       throws IOException {
-    return searchLevel(query, topK, level, eps, vectors, graph, null, Integer.MAX_VALUE);
+    return searchLevel(query, topK, level, eps, vectors, graph, null, Integer.MAX_VALUE, false);
   }
 
   private NeighborQueue searchLevel(
-      T query,
-      int topK,
-      int level,
-      final int[] eps,
-      RandomAccessVectorValues<T> vectors,
-      HnswGraph graph,
-      Bits acceptOrds,
-      int visitedLimit)
-      throws IOException {
+          T query,
+          int topK,
+          int level,
+          final int[] entryPoints,
+          RandomAccessVectorValues<T> vectors,
+          HnswGraph graph,
+          Bits acceptOrds,
+          int visitedLimit,
+          boolean multiValued)
+          throws IOException {
     int size = graph.size();
     NeighborQueue results = new NeighborQueue(topK, false);
     prepareScratchState(vectors.size());
 
-    int numVisited = 0;
-    for (int ep : eps) {
-      if (visited.getAndSet(ep) == false) {
-        if (numVisited >= visitedLimit) {
+    int vectorIdsVisited = 0;
+    for (int vectorId : entryPoints) {
+      if (visited.getAndSet(vectorId) == false) {
+        if (vectorIdsVisited >= visitedLimit) {
           results.markIncomplete();
           break;
         }
-        float score = compare(query, vectors, ep);
-        numVisited++;
-        candidates.add(ep, score);
-        if (acceptOrds == null || acceptOrds.get(ep)) {
-          results.add(ep, score);
+        float score = compare(query, vectors, vectorId);
+        vectorIdsVisited++;
+        candidates.add(vectorId, score);
+        int docId = vectors.ordToDoc(vectorId);
+        if (acceptOrds == null || acceptOrds.get(vectorId)) {

Review Comment:
   `acceptOrds` refers to the docIds as its the top level filter supplied which is a filter on doc ids.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] benwtrent commented on a diff in pull request #12314: Multi-value support for KnnVectorField

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12314:
URL: https://github.com/apache/lucene/pull/12314#discussion_r1243631127


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/Lucene95HnswVectorsWriter.java:
##########
@@ -822,40 +836,37 @@ private void writeMeta(
    * Writes the byte vector values to the output and returns a set of documents that contains
    * vectors.
    */
-  private static DocsWithFieldSet writeByteVectorData(
-      IndexOutput output, ByteVectorValues byteVectorValues) throws IOException {
-    DocsWithFieldSet docsWithField = new DocsWithFieldSet();
-    for (int docV = byteVectorValues.nextDoc();
-        docV != NO_MORE_DOCS;
-        docV = byteVectorValues.nextDoc()) {
-      // write vector
-      byte[] binaryValue = byteVectorValues.vectorValue();
-      assert binaryValue.length == byteVectorValues.dimension() * VectorEncoding.BYTE.byteSize;
+  private static DocsWithVectorsSet writeByteVectorData(
+      IndexOutput output, ByteVectorValues mergedVectorValues) throws IOException {
+    DocsWithVectorsSet docsWithVectors = new DocsWithVectorsSet();
+    for (int vectorId = mergedVectorValues.nextDoc(); vectorId != NO_MORE_DOCS; vectorId = mergedVectorValues.nextDoc()) {
+      int docID = mergedVectorValues.ordToDoc(vectorId);
+      byte[] binaryValue = mergedVectorValues.vectorValue();
+      assert binaryValue.length == mergedVectorValues.dimension() * VectorEncoding.BYTE.byteSize;
       output.writeBytes(binaryValue, binaryValue.length);
-      docsWithField.add(docV);
+      docsWithVectors.add(docID);
     }
-    return docsWithField;
+    return docsWithVectors;

Review Comment:
   From what I can tell, this is simply writing each vector value in order, regardless if they are part of the same document or not correct? 
   
   I suppose we get the document order for free since they all have to be supplied at the same time correct?
   
   The reason I say this, is that if we want to be able to iterate by document, being able to skip directly to the document and read its vectors is important, which wouldn't be easily possible unless all the vectors in a document were written right next to each other.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] tang-hi commented on pull request #12314: Multi-value support for KnnVectorField

Posted by "tang-hi (via GitHub)" <gi...@apache.org>.
tang-hi commented on PR #12314:
URL: https://github.com/apache/lucene/pull/12314#issuecomment-1554789228

   This concept is interesting, but I am curious about its practical uses.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] jimczi commented on pull request #12314: Multi-value support for KnnVectorField

Posted by "jimczi (via GitHub)" <gi...@apache.org>.
jimczi commented on PR #12314:
URL: https://github.com/apache/lucene/pull/12314#issuecomment-1564665498

   > nothing in this PR is final nor I have any strong opinion about it.
   
   Sure, we're just discussing the approach, no worries.
   
   > In regards to your main worry, can you point me to the areas of code you don't like specifically and I can have a thought in how to modify them!
   
   This change in [Float|Byte]VectorValues:
   
   ```
    /** The maximum length of a vector */
     public static final int MAX_DIMENSIONS = 1024;
   
     protected boolean multiValued = false;
   
     public boolean isMultiValued() {
       return multiValued;
     }
   
     /** Sole constructor */
     protected FloatVectorValues() {}
   
   @@ -57,4 +63,13 @@ public final long cost() {
      * @return the vector value
      */
     public abstract float[] vectorValue() throws IOException;
   
     /**
      * Return the document ID corresponding to the input vector id(ordinal)
      * @param ord vector ID(ordinal)
      * @return the document ID
      */
     public int ordToDoc(int ord){
       return ord;
     }
   }
   ```
   
   Today the expectation is that it iterates over doc ids. This change adds an indirection that needs to be checked (`isMultivalued`) and if it's the case then the doc id is an ordinal id that needs to be transformed with `ordToDoc` .
   That's trappy, I am not even sure how you can advance to a doc rather than an ordinal and the code would mean different things based on whether you're working on multivalued or not. Considering how the original APIs was made for single valued I don't think we should try to sneak multi-valued into the model. That's why I propose that we add it as separated like you originally did. It's a doc values + search APIs so it needs to be usable for these two purposes in a more predictive way.  
   
   
   
   
   
   
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] joshdevins commented on pull request #12314: Multi-value support for KnnVectorField

Posted by "joshdevins (via GitHub)" <gi...@apache.org>.
joshdevins commented on PR #12314:
URL: https://github.com/apache/lucene/pull/12314#issuecomment-1568187706

   > SUM = the similarity score between the query and each vector is computed, all scores are summed to get the final score
   > SUM = every time we find a nearest neighbor vector to be added to the topK, if the document is already there, its score is updated summing the old and new score
   
   Just a note on the aggregation functions `max` and `sum`. Most commonly it seems that `max` is used as it is length independent. When using `sum`, the longer the original text of a document field, and thus the more passages it will have, the higher the `sum` of all matching passages will be since all passages will "match". I'm not sure if it will matter in the end, but my suggestion would be that if `sum` is used, one could optionally use a radius/similarity threshold be used to limit the advantage of longer texts, and/or allow using just a limited top-k passages of a document for `sum`.
   
   @alessandrobenedetti Do you have any good references/papers on approaches to re-aggregating passages into documents for SERPs? It seems that the art was abandoned a couple years ago with most approaches settling on `max` passage.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] jimczi commented on pull request #12314: Multi-value support for KnnVectorField

Posted by "jimczi (via GitHub)" <gi...@apache.org>.
jimczi commented on PR #12314:
URL: https://github.com/apache/lucene/pull/12314#issuecomment-1563420102

   Thanks for sharing and working on a prototype @alessandrobenedetti !
   
   I have additional questions and comments ;)
   Starting with the devil advocate but I'd like to understand once more what is the use case? One possible approach today if the use case is passage retrieval is to index one document per passage/sentence. It works fine and you have access to the exact context of the match directly. What are the benefits of moving all passages to a single document? I am not saying there are none but they must be compelling if that requires all these changes.
   
   Regarding the change itself, I wonder if the approach could be more explicit. Instead of hiding the new feature I'd argue that we need to follow the doc values model where single and multi-valued are separated. In practice that would mean adding this to the KnnVectorsReader:
   ```
     public abstract MultiFloatVectorValues getMultiFloatVectorValues(String field) throws IOException;
     public abstract MultiByteVectorValues getMultiByteVectorValues(String field) throws IOException;
     public abstract TopDocs searchMulti(String field, float[] target, int k, Bits acceptDocs, int visitedLimit) throws IOException;
   ```
   The writer side is a bit more tricky but you see what I mean. It's a big change that impacts the implementation and expectations of many existing functions if the change is directly inserted in the existing knn field. 
   I know that it's easy to detect if a field is single or multi-valued under the hood so we could handle this transparently.
   We can do that with the explicit approach too. `searchMulti` can use `search` under the hood if we detect that the field contains exactly one vector per document. So my point is that we can try to share code as much as possible internally but we don't need to hide the difference externally.
   
   Another challenge for this change is the possible overflow in the graph representation. If we change the ordinal value to be the offset of the multivalued position, the maximum allowed is not per doc anymore but per vector. We use int32 in the graph to represents these ordinals, which matches with the max doc limit of Lucene. Maybe not a big deal for small scale but that's something to keep in mind when implementing things like merge or even multi-segments search.
   
   Anyway, quite exciting discussions, Ben and Mayya are reviewing too so just adding my 2cents and happy to help if/when we reach a consensus.
    


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on a diff in pull request #12314: Multi-value support for KnnVectorField

Posted by "mayya-sharipova (via GitHub)" <gi...@apache.org>.
mayya-sharipova commented on code in PR #12314:
URL: https://github.com/apache/lucene/pull/12314#discussion_r1205995223


##########
lucene/core/src/java/org/apache/lucene/index/DocsWithFieldSet.java:
##########
@@ -22,6 +22,8 @@
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.RamUsageEstimator;
 
+import java.util.Stack;

Review Comment:
   I think we should keep this class unchanged; it is used in other places besides vectors and is only about documents.
   
   For keeping track of the number of values in docs, may be we can use another data-structure (`PackedLongValues.Builder`, similar how `SortedSetDocValuesWriter` does it)?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on a diff in pull request #12314: Multi-value support for KnnVectorField

Posted by "mayya-sharipova (via GitHub)" <gi...@apache.org>.
mayya-sharipova commented on code in PR #12314:
URL: https://github.com/apache/lucene/pull/12314#discussion_r1206003463


##########
lucene/core/src/java/org/apache/lucene/index/DocsWithFieldSet.java:
##########
@@ -32,8 +34,14 @@ public final class DocsWithFieldSet extends DocIdSet {
       RamUsageEstimator.shallowSizeOfInstance(DocsWithFieldSet.class);
 
   private FixedBitSet set;
-  private int cardinality = 0;
-  private int lastDocId = -1;
+  private int docsCount = 0;
+  private int lastDocId = 0; // at a certain point in time this was changed to 0? why?
+  
+  private Stack<Integer> valuesPerDocuments;
+  private int currentDocVectorsCount;
+  private int vectorsCount;
+  
+  private boolean multiValued = false;

Review Comment:
   It is also strange to have this `multiValued` field, we probably should just multi values by presence/absence of another data structure that keeps number of values in docs.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] alessandrobenedetti commented on pull request #12314: Multi-value support for KnnVectorField

Posted by "alessandrobenedetti (via GitHub)" <gi...@apache.org>.
alessandrobenedetti commented on PR #12314:
URL: https://github.com/apache/lucene/pull/12314#issuecomment-1556882171

   Hi
   
   > @alessandrobenedetti thank you for kick starting this!
   > 
   > You are absolutely correct, this is a large, but pivotal and necessary change for vector search in Lucene. I have not yet reviewed or looked over the entire design, but two things stuck out to me immediately.
   > 
   > 1. To simplify the review, design, testing, etc. could you simplify and reduce scope? Meaning, for the first iteration of this, Lucene only supports `max` and is used by default for multi-vector fields. This should:
   >    
   >    * remove the need for the additional search parameter (can be added as a part 2 of this feature in the future).
   >    * reduce testing requirements
   >    * reduce user overhead as the closest document based on the closest vector is returned. This behavior parallels nicely with a single vector field.
   > 2. Declaring a vector field as "multi-vector" seems...weird and has a very blast radius on the entire code base. It seems to me that the vector codec should seamlessly handle multiple vectors per field, just like it seamlessly handles sparse vector encoding. Though, thinking about it more, it seems difficult to support both sparse & multi-vector values in this way. All that said, I am curious about to your reasoning here.
   
   Hi @benwtrent, thanks for the initial feedback!
   
   Let me take a look, I'll proceed with a first iterative pass of simplification now that "it works" .
   I'll re-assess if we really need the multi-valued-vector flag for the field (when I started the works, it seemed necessary to me, but maybe I can find a better way).
   
   I'll also check how simpler it becomes with just 'MAX'.
   
   Cheers


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] alessandrobenedetti commented on a diff in pull request #12314: Multi-value support for KnnVectorField

Posted by "alessandrobenedetti (via GitHub)" <gi...@apache.org>.
alessandrobenedetti commented on code in PR #12314:
URL: https://github.com/apache/lucene/pull/12314#discussion_r1206534495


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/Lucene95HnswVectorsWriter.java:
##########
@@ -762,13 +776,18 @@ private void writeMeta(
       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);
+          DirectMonotonicWriter.getInstance(meta, vectorData, vectorsCount, DIRECT_MONOTONIC_BLOCK_SHIFT);
       DocIdSetIterator iterator = docsWithField.iterator();
+      int[] valuesPerDocument = docsWithField.getValuesPerDocument();

Review Comment:
   I'll check later on, happy to use a different implementation



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] alessandrobenedetti commented on pull request #12314: Multi-value support for KnnVectorField

Posted by "alessandrobenedetti (via GitHub)" <gi...@apache.org>.
alessandrobenedetti commented on PR #12314:
URL: https://github.com/apache/lucene/pull/12314#issuecomment-1564165188

   > Thanks for sharing and working on a prototype @alessandrobenedetti !
   > 
   > I have additional questions and comments ;) Starting with the devil advocate but I'd like to understand once more what is the use case? One possible approach today if the use case is passage retrieval is to index one document per passage/sentence. It works fine and you have access to the exact context of the match directly. What are the benefits of moving all passages to a single document? I am not saying there are none but they must be compelling if that requires all these changes.
   
   Ale: I guess the use case for multi valued vector-fields is not much different from any other multi valued fields:
   you may want to avoid normalisation and the necessity to build additional complexity with multiple duplicated documents that only have the vector field as a difference.
   If you have simple documents with just the vector field, splitting them in passages and then aggregating them in the end of your search pipeline is not going to be that annoying, but imaging more complex situations where you have aggregations already, nested documents or just documents with many more metadata along.
   On top of that we got curiosity from some customers about the functionality, for this reason I felt it was a nice addition.
   
   > 
   > Regarding the change itself, I wonder if the approach could be more explicit. Instead of hiding the new feature I'd argue that we need to follow the doc values model where single and multi-valued are separated. In practice that would mean adding this to the KnnVectorsReader:
   > 
   > ```
   >   public abstract MultiFloatVectorValues getMultiFloatVectorValues(String field) throws IOException;
   >   public abstract MultiByteVectorValues getMultiByteVectorValues(String field) throws IOException;
   >   public abstract TopDocs searchMulti(String field, float[] target, int k, Bits acceptDocs, int visitedLimit) throws IOException;
   > ```
   > 
   > The writer side is a bit more tricky but you see what I mean. It's a big change that impacts the implementation and expectations of many existing functions if the change is directly inserted in the existing knn field. I know that it's easy to detect if a field is single or multi-valued under the hood so we could handle this transparently. We can do that with the explicit approach too. `searchMulti` can use `search` under the hood if we detect that the field contains exactly one vector per document. So my point is that we can try to share code as much as possible internally but we don't need to hide the difference externally.
   
   That was the initial approach, it was explicit at index and query time, and they were separate code paths from the single valued use case. So it was not affecting the single valued scenario at all.
   Following some of the feedback here I moved to transparent approach, that prooved extremely beneficial in reducing the complexity of the pull request (at the cost of impacting potentially some single valued case scenario).
   I iterated a bit, so I should have reduced the impact already on the single valued case, but not to 100% I guess.
   
   > 
   > Another challenge for this change is the possible overflow in the graph representation. If we change the ordinal value to be the offset of the multivalued position, the maximum allowed is not per doc anymore but per vector. We use int32 in the graph to represents these ordinals, which matches with the max doc limit of Lucene. Maybe not a big deal for small scale but that's something to keep in mind when implementing things like merge or even multi-segments search.
   
   Yes, with one vector per document, the maximum amount of supported vectors was in line with the docs, this is still the case but I agree that right now we potentially support less docs, happy to change that.
   > 
   > Anyway, quite exciting discussions, Ben and Mayya are reviewing too so just adding my 2cents and happy to help if/when we reach a consensus.
   
   Thanks for the feedback!
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] alessandrobenedetti commented on a diff in pull request #12314: Multi-value support for KnnVectorField

Posted by "alessandrobenedetti (via GitHub)" <gi...@apache.org>.
alessandrobenedetti commented on code in PR #12314:
URL: https://github.com/apache/lucene/pull/12314#discussion_r1243704603


##########
lucene/core/src/java/org/apache/lucene/search/VectorScorer.java:
##########
@@ -85,6 +96,26 @@ public boolean advanceExact(int doc) throws IOException {
       return vectorDoc == doc;
     }
 
+    @Override
+    public Map<Integer, Float> scoreMultiValued(BitSet acceptedDocs) throws IOException {
+      Map<Integer, Float> docToScore = new HashMap<>();
+      for (int vectorId = values.nextDoc(); vectorId != NO_MORE_DOCS; vectorId = values.nextDoc()) {
+        int docID = values.ordToDoc(vectorId);
+        if (acceptedDocs.get(docID)) {
+          float currentScore = similarity.compare(query, values.vectorValue());
+          docToScore.putIfAbsent(docID, currentScore);
+          docToScore.computeIfPresent(docID,
+                  (key, previousScore) -> Math.max(previousScore, currentScore));
+        }
+      }
+      return docToScore;
+    }

Review Comment:
   Yep, didn't spend much time on this, I drafted a few lines of code just to make the exact search tests green.
   Happy to get suggestions/other people implementations here!
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org