You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2022/12/30 22:08:11 UTC

[GitHub] [lucene] jmazanec15 opened a new pull request, #12050: Reuse HNSW graph for intialization during merge

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

   ### Description
   
   Related to #11354 (performance metrics can be found here). I also started a draft PR in #11719, but decided to refactor into a new PR.
   
   This PR adds the functionality to initialize a merged segment's HNSW graph from the largest HNSW graph from the segments being merged. The graph selected must not contain any dead documents. If no suitable intiailizer graph is found, it will fall back to creating the graph from scratch.
   
   To support this functionality, a couple of changes to current graph construction process needed to be made. OnHeapHnswGraph had to support out of order insertion. This is because the mapped ordinals of the nodes in the graph used for initialization are not necessarily the first X ordinals in the new graph.
   
   I also removed the implicit addition of the first node into the graph. Implicitly adding the first node created a lot of complexity for initialization. In #11719, I got it to work without changing this but thought it was cleaner to switch to require the first node to be added explicitly.
   
   In addition to this, graphs produced by merging two segments are no longer necessarily going to be equivalent to indexing one segment directly. This is caused by both differences in assigned random values as well as insertion order dictating which neighbors are selected for which nodes.
   


-- 
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] jmazanec15 commented on a diff in pull request #12050: Reuse HNSW graph for intialization during merge

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


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/Lucene95HnswVectorsWriter.java:
##########
@@ -489,6 +485,220 @@ public void mergeOneField(FieldInfo fieldInfo, MergeState mergeState) throws IOE
     }
   }
 
+  private HnswGraphBuilder<float[]> createFloatVectorHnswGraphBuilder(

Review Comment:
   Oh I see. Makes sense. I updated.



-- 
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] jmazanec15 commented on a diff in pull request #12050: Reuse HNSW graph for intialization during merge

Posted by GitBox <gi...@apache.org>.
jmazanec15 commented on code in PR #12050:
URL: https://github.com/apache/lucene/pull/12050#discussion_r1081896861


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/OnHeapHnswGraph.java:
##########
@@ -94,36 +93,83 @@ public int size() {
   }
 
   /**
-   * Add node on the given level
+   * Add node on the given level. Nodes can be inserted out of order, but it requires that the nodes

Review Comment:
   Updated structure to use treemap to represent upper levels of graph.



-- 
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] msokolov commented on pull request #12050: Reuse HNSW graph for intialization during merge

Posted by GitBox <gi...@apache.org>.
msokolov commented on PR #12050:
URL: https://github.com/apache/lucene/pull/12050#issuecomment-1379112305

   > To support this functionality, a couple of changes to current graph construction process needed to be made. OnHeapHnswGraph had to support out of order insertion. This is because the mapped ordinals of the nodes in the graph used for initialization are not necessarily the first X ordinals in the new graph.
   
   I'm having trouble wrapping my head around this. When we start merging some field, each segment seg has a graph with ordinals in [0,seg.size]. Why can't we preserve the ordinals from the largest segment, and then let the others fall where they may?


-- 
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] jmazanec15 commented on a diff in pull request #12050: Reuse HNSW graph for intialization during merge

Posted by GitBox <gi...@apache.org>.
jmazanec15 commented on code in PR #12050:
URL: https://github.com/apache/lucene/pull/12050#discussion_r1061916872


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/OnHeapHnswGraph.java:
##########
@@ -94,36 +93,83 @@ public int size() {
   }
 
   /**
-   * Add node on the given level
+   * Add node on the given level. Nodes can be inserted out of order, but it requires that the nodes

Review Comment:
   Im not sure on this. For random insertion for the graph, I think a BST would be better. 
   
   However, the insertion pattern for merge typically wont be random. It will be more like first, nodes [15-73] are inserted, and then nodes [0-14] and then nodes [74-100]. This assumes that the MergedVectorValues are a concatenation of the segments to be merged vectors. For this, I added the small optimization of the "lastAddedPosInLayer" list, which will skip binary search during locally ordered insertion.
   
   That being said, I am not sure that the "concatenation" property is guaranteed, specifically in the case when the mergeState.needsIndexSort == true. Given this case, it seems like insertion pattern might be more random.
   
   All that being said, do you think it would be better to build for the concatenation pattern or more random insert pattern?



-- 
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] zhaih commented on pull request #12050: Reuse HNSW graph for intialization during merge

Posted by GitBox <gi...@apache.org>.
zhaih commented on PR #12050:
URL: https://github.com/apache/lucene/pull/12050#issuecomment-1382268387

   +1, That sounds good!
   
   On Fri, Jan 13, 2023, 11:10 John Mazanec ***@***.***> wrote:
   
   > ***@***.**** commented on this pull request.
   > ------------------------------
   >
   > In lucene/core/src/java/org/apache/lucene/util/hnsw/OnHeapHnswGraph.java
   > <https://github.com/apache/lucene/pull/12050#discussion_r1069901702>:
   >
   > > @@ -94,36 +93,83 @@ public int size() {
   >    }
   >
   >    /**
   > -   * Add node on the given level
   > +   * Add node on the given level. Nodes can be inserted out of order, but it requires that the nodes
   >
   > Oh I see what you mean. Yes, that makes sense.
   >
   > I think for level 0, we will still want to use a List because all nodes
   > will eventually be present in this level. However, for levels > 0, we can
   > use a TreeMap and then add an iterator over the keys of that map.
   >
   > —
   > Reply to this email directly, view it on GitHub
   > <https://github.com/apache/lucene/pull/12050#discussion_r1069901702>, or
   > unsubscribe
   > <https://github.com/notifications/unsubscribe-auth/AEFSB7CTOHHMPL4YNBM6ZLTWSGSC7ANCNFSM6AAAAAATNF2BCI>
   > .
   > You are receiving this because you were mentioned.Message ID:
   > ***@***.***>
   >
   


-- 
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 #12050: Reuse HNSW graph for intialization during merge

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


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/NeighborQueue.java:
##########
@@ -56,6 +56,8 @@ long apply(long v) {
   // Whether the search stopped early because it reached the visited nodes limit
   private boolean incomplete;
 
+  public static final NeighborQueue EMPTY_MAX_HEAP_NEIGHBOR_QUEUE = new NeighborQueue(1, true);

Review Comment:
   It is nice to have a static thing like this. But, `EMPTY_MAX_HEAP_NEIGHBOR_QUEUE#add(int float)` is possible. This seems dangerous to me as somebody might accidentally call `search` and then add values to this static object.
   
   If we are going to have a static object like this, it would be good if it was `EmptyNeighborQueue` that disallows `add` or any mutable action. 



-- 
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] jmazanec15 commented on a diff in pull request #12050: Reuse HNSW graph for intialization during merge

Posted by GitBox <gi...@apache.org>.
jmazanec15 commented on code in PR #12050:
URL: https://github.com/apache/lucene/pull/12050#discussion_r1062966074


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/OnHeapHnswGraph.java:
##########
@@ -94,36 +93,83 @@ public int size() {
   }
 
   /**
-   * Add node on the given level
+   * Add node on the given level. Nodes can be inserted out of order, but it requires that the nodes

Review Comment:
   > but still because in L156 we need to copy the rest of array again and again as long as that is a non-appending action
   
   Right, this could be expensive for out of order insertion. I can try switching the nodeByLevel int array to a TreeSet and compare performance to https://github.com/apache/lucene/issues/11354.
   
   One complication with this approach is that the NodesIterator expects an int array: https://github.com/apache/lucene/blob/main/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraph.java#L134. Given this is a public interface, we might need to either convert the treeset to an int array every time [getNodesOnLevel](https://github.com/apache/lucene/blob/main/lucene/core/src/java/org/apache/lucene/util/hnsw/OnHeapHnswGraph.java#L165) gets called, or alter the NodesIterator interface to support both an int array and an Iterator produced from the TreeSet.
   
   @zhaih What do you think of this approach? Is there better way to do this?



-- 
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 #12050: Reuse HNSW graph for intialization during merge

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


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java:
##########
@@ -143,10 +148,64 @@ public OnHeapHnswGraph build(RandomAccessVectorValues vectorsToAdd) throws IOExc
     return hnsw;
   }
 
+  /**
+   * Initializes the graph of this builder. Transfers the nodes and their neighbors from the
+   * initializer graph into the graph being produced by this builder, mapping ordinals from the
+   * initializer graph to their new ordinals in this builder's graph. The builder's graph must be
+   * empty before calling this method.
+   *
+   * @param initializerGraph graph used for initialization
+   * @param oldToNewOrdinalMap map for converting from ordinals in the initializerGraph to this
+   *     builder's graph
+   */
+  public void initializeFromGraph(
+      HnswGraph initializerGraph, Map<Integer, Integer> oldToNewOrdinalMap) throws IOException {
+    assert hnsw.size() == 0;

Review Comment:
   Could you make this a new static method that also constructs the graph builder?



-- 
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] jmazanec15 commented on a diff in pull request #12050: Reuse HNSW graph for intialization during merge

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


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java:
##########
@@ -143,10 +148,64 @@ public OnHeapHnswGraph build(RandomAccessVectorValues vectorsToAdd) throws IOExc
     return hnsw;
   }
 
+  /**
+   * Initializes the graph of this builder. Transfers the nodes and their neighbors from the
+   * initializer graph into the graph being produced by this builder, mapping ordinals from the
+   * initializer graph to their new ordinals in this builder's graph. The builder's graph must be
+   * empty before calling this method.
+   *
+   * @param initializerGraph graph used for initialization
+   * @param oldToNewOrdinalMap map for converting from ordinals in the initializerGraph to this
+   *     builder's graph
+   */
+  public void initializeFromGraph(
+      HnswGraph initializerGraph, Map<Integer, Integer> oldToNewOrdinalMap) throws IOException {
+    assert hnsw.size() == 0;

Review Comment:
   Makes sense. Will update.



-- 
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] jmazanec15 commented on a diff in pull request #12050: Reuse HNSW graph for intialization during merge

Posted by GitBox <gi...@apache.org>.
jmazanec15 commented on code in PR #12050:
URL: https://github.com/apache/lucene/pull/12050#discussion_r1061902120


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/Lucene95HnswVectorsWriter.java:
##########
@@ -461,6 +467,126 @@ public void mergeOneField(FieldInfo fieldInfo, MergeState mergeState) throws IOE
     }
   }
 
+  private void maybeInitializeFromGraph(
+      HnswGraphBuilder<?> hnswGraphBuilder, MergeState mergeState, FieldInfo fieldInfo)
+      throws IOException {
+    int initializerIndex = selectGraphForInitialization(mergeState, fieldInfo);
+    if (initializerIndex == -1) {
+      return;
+    }
+
+    HnswGraph initializerGraph =
+        getHnswGraphFromReader(fieldInfo.name, mergeState.knnVectorsReaders[initializerIndex]);
+    Map<Integer, Integer> ordinalMapper =
+        getOldToNewOrdinalMap(mergeState, fieldInfo, initializerIndex);
+    hnswGraphBuilder.initializeFromGraph(initializerGraph, ordinalMapper);
+  }
+
+  private int selectGraphForInitialization(MergeState mergeState, FieldInfo fieldInfo)
+      throws IOException {
+    // Find the KnnVectorReader with the most docs that meets the following criteria:
+    //  1. Does not contain any deleted docs
+    //  2. Is a Lucene95HnswVectorsReader/PerFieldKnnVectorReader
+    // If no readers exist that meet this criteria, return -1. If they do, return their index in
+    // merge state
+    int maxCandidateVectorCount = 0;
+    int initializerIndex = -1;
+
+    for (int i = 0; i < mergeState.liveDocs.length; i++) {
+      KnnVectorsReader currKnnVectorsReader = mergeState.knnVectorsReaders[i];
+      if (mergeState.knnVectorsReaders[i]
+          instanceof PerFieldKnnVectorsFormat.FieldsReader candidateReader) {
+        currKnnVectorsReader = candidateReader.getFieldReader(fieldInfo.name);
+      }
+
+      if (!allMatch(mergeState.liveDocs[i])
+          || !(currKnnVectorsReader instanceof Lucene95HnswVectorsReader candidateReader)) {
+        continue;
+      }
+
+      VectorValues vectorValues = candidateReader.getVectorValues(fieldInfo.name);
+      if (vectorValues == null) {
+        continue;
+      }
+
+      int candidateVectorCount = vectorValues.size();
+      if (candidateVectorCount > maxCandidateVectorCount) {
+        maxCandidateVectorCount = candidateVectorCount;
+        initializerIndex = i;
+      }
+    }
+    return initializerIndex;
+  }
+
+  private HnswGraph getHnswGraphFromReader(String fieldName, KnnVectorsReader knnVectorsReader)
+      throws IOException {
+    if (knnVectorsReader instanceof PerFieldKnnVectorsFormat.FieldsReader perFieldReader
+        && perFieldReader.getFieldReader(fieldName)
+            instanceof Lucene95HnswVectorsReader fieldReader) {
+      return fieldReader.getGraph(fieldName);
+    }
+
+    if (knnVectorsReader instanceof Lucene95HnswVectorsReader) {
+      return ((Lucene95HnswVectorsReader) knnVectorsReader).getGraph(fieldName);
+    }
+
+    throw new IllegalArgumentException(
+        "Invalid KnnVectorsReader. Must be of type PerFieldKnnVectorsFormat.FieldsReader or Lucene94HnswVectorsReader");

Review Comment:
   Makes sense. Will update.



-- 
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] jmazanec15 commented on a diff in pull request #12050: Reuse HNSW graph for intialization during merge

Posted by GitBox <gi...@apache.org>.
jmazanec15 commented on code in PR #12050:
URL: https://github.com/apache/lucene/pull/12050#discussion_r1069901702


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/OnHeapHnswGraph.java:
##########
@@ -94,36 +93,83 @@ public int size() {
   }
 
   /**
-   * Add node on the given level
+   * Add node on the given level. Nodes can be inserted out of order, but it requires that the nodes

Review Comment:
   Oh I see what you mean. Yes, that makes sense. 
   
   I think for level 0, we will still want to use a List<NeighborArrays> because all nodes will eventually be present in this level. However, for levels > 0, we can use a TreeMap and then add an iterator over the keys of that map.



-- 
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] zhaih commented on a diff in pull request #12050: Reuse HNSW graph for intialization during merge

Posted by GitBox <gi...@apache.org>.
zhaih commented on code in PR #12050:
URL: https://github.com/apache/lucene/pull/12050#discussion_r1061976538


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java:
##########
@@ -143,10 +148,64 @@ public OnHeapHnswGraph build(RandomAccessVectorValues vectorsToAdd) throws IOExc
     return hnsw;
   }
 
+  /**
+   * Initializes the graph of this builder. Transfers the nodes and their neighbors from the
+   * initializer graph into the graph being produced by this builder, mapping ordinals from the
+   * initializer graph to their new ordinals in this builder's graph. The builder's graph must be
+   * empty before calling this method.
+   *
+   * @param initializerGraph graph used for initialization
+   * @param oldToNewOrdinalMap map for converting from ordinals in the initializerGraph to this
+   *     builder's graph
+   */
+  public void initializeFromGraph(
+      HnswGraph initializerGraph, Map<Integer, Integer> oldToNewOrdinalMap) throws IOException {
+    assert hnsw.size() == 0;
+    float[] vectorValue = null;
+    BytesRef binaryValue = null;
+    for (int level = 0; level < initializerGraph.numLevels(); level++) {
+      HnswGraph.NodesIterator it = initializerGraph.getNodesOnLevel(level);
+
+      while (it.hasNext()) {
+        int oldOrd = it.nextInt();
+        int newOrd = oldToNewOrdinalMap.get(oldOrd);
+
+        hnsw.addNode(level, newOrd);
+
+        if (level == 0) {
+          initializedNodes.add(newOrd);
+        }
+
+        switch (this.vectorEncoding) {
+          case FLOAT32 -> vectorValue = vectors.vectorValue(newOrd);
+          case BYTE -> binaryValue = vectors.binaryValue(newOrd);
+        }
+
+        NeighborArray newNeighbors = this.hnsw.getNeighbors(level, newOrd);
+        initializerGraph.seek(level, oldOrd);
+        for (int oldNeighbor = initializerGraph.nextNeighbor();
+            oldNeighbor != NO_MORE_DOCS;
+            oldNeighbor = initializerGraph.nextNeighbor()) {
+          int newNeighbor = oldToNewOrdinalMap.get(oldNeighbor);
+          float score =
+              switch (this.vectorEncoding) {
+                case FLOAT32 -> this.similarityFunction.compare(

Review Comment:
   Oh ok, I see. Initially I thought the neighbor ordering is for searching but seems that is not the case. Is this sorted order only used for calculating diversity easier?
   Anyway that can be a later topic and I think we can live with the existing logic for 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] zhaih commented on a diff in pull request #12050: Reuse HNSW graph for intialization during merge

Posted by GitBox <gi...@apache.org>.
zhaih commented on code in PR #12050:
URL: https://github.com/apache/lucene/pull/12050#discussion_r1063681335


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/OnHeapHnswGraph.java:
##########
@@ -94,36 +93,83 @@ public int size() {
   }
 
   /**
-   * Add node on the given level
+   * Add node on the given level. Nodes can be inserted out of order, but it requires that the nodes

Review Comment:
   For `NodesIterator` I think once we have confirmed there's benefit of using tree structure we can further subclass it into `Array/TreeBasedNodesIterator`. Because it seems the `NodesIterator` still only provides what normal iterator is providing but not some exotic random access or other operation that can't be achieved by a tree, so I guess shouldn't become a big problem?



-- 
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] zhaih commented on a diff in pull request #12050: Reuse HNSW graph for intialization during merge

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


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/Lucene95HnswVectorsWriter.java:
##########
@@ -489,6 +485,220 @@ public void mergeOneField(FieldInfo fieldInfo, MergeState mergeState) throws IOE
     }
   }
 
+  private HnswGraphBuilder<float[]> createFloatVectorHnswGraphBuilder(

Review Comment:
   nit: can we merge this function with L519 using generic?



##########
lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraph.java:
##########
@@ -182,10 +201,43 @@ public int nextInt() {
     public boolean hasNext() {
       return cur < size;
     }
+  }
 
-    /** The number of elements in this iterator * */
-    public int size() {
-      return size;
+  /** Nodes iterator based on set representation of nodes. */
+  public static class SetNodesIterator extends NodesIterator {

Review Comment:
   nit: I feel like this can be some more general name, since the only requirement is that the input provides a int iterator. But anyway this shouldn't block the PR.



##########
lucene/core/src/java/org/apache/lucene/util/hnsw/OnHeapHnswGraph.java:
##########
@@ -33,44 +32,36 @@
 public final class OnHeapHnswGraph extends HnswGraph implements Accountable {
 
   private int numLevels; // the current number of levels in the graph
-  private int entryNode; // the current graph entry node on the top level
+  private int entryNode; // the current graph entry node on the top level. -1 if not set
 
-  // Nodes by level expressed as the level 0's nodes' ordinals.
-  // As level 0 contains all nodes, nodesByLevel.get(0) is null.
-  private final List<int[]> nodesByLevel;
-
-  // graph is a list of graph levels.
-  // Each level is represented as List<NeighborArray> – nodes' connections on this level.
+  // Level 0 is represented as List<NeighborArray> – nodes' connections on level 0.
   // Each entry in the list has the top maxConn/maxConn0 neighbors of a node. The nodes correspond
   // to vectors
   // added to HnswBuilder, and the node values are the ordinals of those vectors.
   // Thus, on all levels, neighbors expressed as the level 0's nodes' ordinals.
-  private final List<List<NeighborArray>> graph;
+  private final List<NeighborArray> graphLevel0;
+  // Represents levels 1-N. Each level is represented with a TreeMap that maps a levels level 0
+  // ordinal to its
+  // neighbors on that level.
+  private final List<TreeMap<Integer, NeighborArray>> graphUpperLevels;

Review Comment:
   I think to maintain the index we're still adding a fake "level 0" as `null`, can we reflect that in the comment? (By explicitly state that we'll always insert a `null` to fill level 0's index?)



-- 
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] zhaih commented on a diff in pull request #12050: Reuse HNSW graph for intialization during merge

Posted by GitBox <gi...@apache.org>.
zhaih commented on code in PR #12050:
URL: https://github.com/apache/lucene/pull/12050#discussion_r1061955216


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/OnHeapHnswGraph.java:
##########
@@ -94,36 +93,83 @@ public int size() {
   }
 
   /**
-   * Add node on the given level
+   * Add node on the given level. Nodes can be inserted out of order, but it requires that the nodes

Review Comment:
   Good point, for concatenation pattern it might be better to not use a naive BST, but still because in L156 we need to copy the rest of array again and again as long as that is a non-appending action, I think it still is better to use some tree-like structure, maybe RB tree or other balanced tree?
   
   But we can also leave it as a separate issue for now, because seems like it is another mid-big change and this one need not to be blocked by that I guess?



-- 
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] zhaih commented on a diff in pull request #12050: Reuse HNSW graph for intialization during merge

Posted by GitBox <gi...@apache.org>.
zhaih commented on code in PR #12050:
URL: https://github.com/apache/lucene/pull/12050#discussion_r1068747982


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/OnHeapHnswGraph.java:
##########
@@ -94,36 +93,83 @@ public int size() {
   }
 
   /**
-   * Add node on the given level
+   * Add node on the given level. Nodes can be inserted out of order, but it requires that the nodes

Review Comment:
   > We need this index to query the OnHeapHnswGraph.graph to get the NeighborArray for a particular element ([ref](https://github.com/jmazanec15/lucene/blob/hnsw-merge-from-graph/lucene/core/src/java/org/apache/lucene/util/hnsw/OnHeapHnswGraph.java#L87)).
   
   I think we generally are not exposing the `graph` outside so the usage is only inside the OnHeapHnswGraph I believe? And inside the class I think that's the only place we use the nodeIndex, which is used for retrieving certain node after binary search. I think this won't be a problem for TreeSet or TreeMap (if we use node id as key and neighbor array as value)? Or have I overlooked some other usage of retrieving via nodeIndex?



-- 
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] jmazanec15 commented on a diff in pull request #12050: Reuse HNSW graph for intialization during merge

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


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/NeighborQueue.java:
##########
@@ -56,6 +56,8 @@ long apply(long v) {
   // Whether the search stopped early because it reached the visited nodes limit
   private boolean incomplete;
 
+  public static final NeighborQueue EMPTY_MAX_HEAP_NEIGHBOR_QUEUE = new NeighborQueue(1, true);

Review Comment:
   You are right, I did not think about this. Given how much mutable state there is, I am wondering if it might just be better to get rid of this. What do you think?



-- 
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] jmazanec15 commented on pull request #12050: Reuse HNSW graph for intialization during merge

Posted by GitBox <gi...@apache.org>.
jmazanec15 commented on PR #12050:
URL: https://github.com/apache/lucene/pull/12050#issuecomment-1379247267

   @msokolov The main reason I did not do this was to avoid having to modify the ordering of the vectors from the MergedVectorValues. I believe that the ordinals in the graph map to the positioning in the the vector values, so they need to be synchronized. 


-- 
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] jmazanec15 commented on a diff in pull request #12050: Reuse HNSW graph for intialization during merge

Posted by GitBox <gi...@apache.org>.
jmazanec15 commented on code in PR #12050:
URL: https://github.com/apache/lucene/pull/12050#discussion_r1061902971


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/Lucene95HnswVectorsWriter.java:
##########
@@ -461,6 +467,126 @@ public void mergeOneField(FieldInfo fieldInfo, MergeState mergeState) throws IOE
     }
   }
 
+  private void maybeInitializeFromGraph(
+      HnswGraphBuilder<?> hnswGraphBuilder, MergeState mergeState, FieldInfo fieldInfo)
+      throws IOException {
+    int initializerIndex = selectGraphForInitialization(mergeState, fieldInfo);
+    if (initializerIndex == -1) {
+      return;
+    }
+
+    HnswGraph initializerGraph =
+        getHnswGraphFromReader(fieldInfo.name, mergeState.knnVectorsReaders[initializerIndex]);
+    Map<Integer, Integer> ordinalMapper =
+        getOldToNewOrdinalMap(mergeState, fieldInfo, initializerIndex);
+    hnswGraphBuilder.initializeFromGraph(initializerGraph, ordinalMapper);
+  }
+
+  private int selectGraphForInitialization(MergeState mergeState, FieldInfo fieldInfo)
+      throws IOException {
+    // Find the KnnVectorReader with the most docs that meets the following criteria:
+    //  1. Does not contain any deleted docs
+    //  2. Is a Lucene95HnswVectorsReader/PerFieldKnnVectorReader
+    // If no readers exist that meet this criteria, return -1. If they do, return their index in
+    // merge state
+    int maxCandidateVectorCount = 0;
+    int initializerIndex = -1;
+
+    for (int i = 0; i < mergeState.liveDocs.length; i++) {
+      KnnVectorsReader currKnnVectorsReader = mergeState.knnVectorsReaders[i];
+      if (mergeState.knnVectorsReaders[i]
+          instanceof PerFieldKnnVectorsFormat.FieldsReader candidateReader) {
+        currKnnVectorsReader = candidateReader.getFieldReader(fieldInfo.name);
+      }
+
+      if (!allMatch(mergeState.liveDocs[i])
+          || !(currKnnVectorsReader instanceof Lucene95HnswVectorsReader candidateReader)) {
+        continue;
+      }
+
+      VectorValues vectorValues = candidateReader.getVectorValues(fieldInfo.name);
+      if (vectorValues == null) {
+        continue;
+      }
+
+      int candidateVectorCount = vectorValues.size();
+      if (candidateVectorCount > maxCandidateVectorCount) {
+        maxCandidateVectorCount = candidateVectorCount;
+        initializerIndex = i;
+      }
+    }
+    return initializerIndex;
+  }
+
+  private HnswGraph getHnswGraphFromReader(String fieldName, KnnVectorsReader knnVectorsReader)
+      throws IOException {
+    if (knnVectorsReader instanceof PerFieldKnnVectorsFormat.FieldsReader perFieldReader
+        && perFieldReader.getFieldReader(fieldName)
+            instanceof Lucene95HnswVectorsReader fieldReader) {
+      return fieldReader.getGraph(fieldName);
+    }
+
+    if (knnVectorsReader instanceof Lucene95HnswVectorsReader) {
+      return ((Lucene95HnswVectorsReader) knnVectorsReader).getGraph(fieldName);
+    }
+
+    throw new IllegalArgumentException(
+        "Invalid KnnVectorsReader. Must be of type PerFieldKnnVectorsFormat.FieldsReader or Lucene94HnswVectorsReader");
+  }
+
+  private Map<Integer, Integer> getOldToNewOrdinalMap(
+      MergeState mergeState, FieldInfo fieldInfo, int initializerIndex) throws IOException {
+    VectorValues initializerVectorValues =
+        mergeState.knnVectorsReaders[initializerIndex].getVectorValues(fieldInfo.name);
+    MergeState.DocMap initializerDocMap = mergeState.docMaps[initializerIndex];
+
+    Map<Integer, Integer> newIdToOldOrdinal = new HashMap<>();
+    int oldOrd = 0;
+    for (int oldId = initializerVectorValues.nextDoc();
+        oldId != NO_MORE_DOCS;
+        oldId = initializerVectorValues.nextDoc()) {
+      if (initializerVectorValues.vectorValue() == null) {
+        continue;
+      }
+      int newId = initializerDocMap.get(oldId);
+      newIdToOldOrdinal.put(newId, oldOrd);
+      oldOrd++;
+    }
+
+    Map<Integer, Integer> oldToNewOrdinalMap = new HashMap<>();
+    int newOrd = 0;
+    int maxNewDocID = Collections.max(newIdToOldOrdinal.keySet());

Review Comment:
   Good idea, I will update this.



-- 
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 #12050: Reuse HNSW graph for intialization during merge

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

   @jmazanec15 merged and I backported to branch_9x (some minor changes for java version stuff around switch statements). 
   
   Good stuff!


-- 
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] jmazanec15 commented on a diff in pull request #12050: Reuse HNSW graph for intialization during merge

Posted by GitBox <gi...@apache.org>.
jmazanec15 commented on code in PR #12050:
URL: https://github.com/apache/lucene/pull/12050#discussion_r1080646383


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/OnHeapHnswGraph.java:
##########
@@ -94,36 +93,83 @@ public int size() {
   }
 
   /**
-   * Add node on the given level
+   * Add node on the given level. Nodes can be inserted out of order, but it requires that the nodes

Review Comment:
   Added a commit for it here: https://github.com/jmazanec15/lucene/commit/9c54de56fa37a35bdff241abd9ebe3a6f1d8ba3a. Running some performance tests to compare results.



-- 
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] jmazanec15 commented on a diff in pull request #12050: Reuse HNSW graph for intialization during merge

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


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/Lucene95HnswVectorsWriter.java:
##########
@@ -489,6 +485,220 @@ public void mergeOneField(FieldInfo fieldInfo, MergeState mergeState) throws IOE
     }
   }
 
+  private HnswGraphBuilder<float[]> createFloatVectorHnswGraphBuilder(

Review Comment:
   I cant think of a good way to do this. HnswGraphBuilder  already uses generics and requires that the same generic be passed in for the RandomVectorValues. So I think some branching logic will be required no matter what. Did you have an idea for this?



-- 
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] jmazanec15 commented on a diff in pull request #12050: Reuse HNSW graph for intialization during merge

Posted by GitBox <gi...@apache.org>.
jmazanec15 commented on code in PR #12050:
URL: https://github.com/apache/lucene/pull/12050#discussion_r1067742826


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/OnHeapHnswGraph.java:
##########
@@ -94,36 +93,83 @@ public int size() {
   }
 
   /**
-   * Add node on the given level
+   * Add node on the given level. Nodes can be inserted out of order, but it requires that the nodes

Review Comment:
   That makes sense. This shouldnt be a big problem.



-- 
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] jmazanec15 commented on a diff in pull request #12050: Reuse HNSW graph for intialization during merge

Posted by GitBox <gi...@apache.org>.
jmazanec15 commented on code in PR #12050:
URL: https://github.com/apache/lucene/pull/12050#discussion_r1068562367


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/OnHeapHnswGraph.java:
##########
@@ -94,36 +93,83 @@ public int size() {
   }
 
   /**
-   * Add node on the given level
+   * Add node on the given level. Nodes can be inserted out of order, but it requires that the nodes

Review Comment:
   Thinking about this more, one issue with a TreeSet based approach would be that we wouldnt be able to look up the index of a particular element (unless we converted the set to a list each time we need to do lookup). We need this index to query the OnHeapHnswGraph.graph to get the NeighborArray for a particular element ([ref](https://github.com/jmazanec15/lucene/blob/hnsw-merge-from-graph/lucene/core/src/java/org/apache/lucene/util/hnsw/OnHeapHnswGraph.java#L87)). I am not sure if there is a good way around this - are you aware of any?
   
   Alternatively, I was thinking if we wanted to avoid the expensive copy for inserting when position < idx, we could switch from using an int[] to an ArrayList<Integer> to represent nodesByLevel for a particular level. This should avoid in most cases the extreme copy that out of order insertion would require.



-- 
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 #12050: Reuse HNSW graph for intialization during merge

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

   > Ah since Lucene95 has just been released, I think we should move this to Lucene 96?
   
   @zhaih 
   
   Do you mean create a new Codec version? From what I can tell, nothing in the underlying storage format has changed and the only reason `Lucene95HnswVectorsReader` is cast is for `Lucene95HnswVectorsReader#getGraph`, which already existed.
   
   Could you clarify your concern?
   


-- 
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] jmazanec15 commented on pull request #12050: Reuse HNSW graph for intialization during merge

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

   @benwtrent thanks! I do not mind a coauthor. Was working on the rebase and just finished 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] jpountz commented on pull request #12050: Reuse HNSW graph for intialization during merge

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

   Nightlies have failed for the last couple days, complaining that KNN searches now return different hits. Is it expected that given the exact same indexing conditions (flushing on doc count and serial merge scheduler), KNN searches may return different hits for the same query with this change?
   
   Here's the error I'm seeing in the log for reference (can be retrieved via `curl -r -10000 http://people.apache.org/~mikemccand/lucenebench/nightly.log`):
   
   ```
   RuntimeError: search result differences: ["query=KnnFloatVectorQuery:vector[0.024077624,...][100] filter=None sort=None groupField=None hitCount=100: hit 6 has wrong field/score value ([19995955], '0.82841617') vs ([19404640], '0.8304943')", "query=KnnFloatVectorQuery:vector[0.028473025,...][100] filter=None sort=None groupField=None hitCount=100: hit 1 has wrong field/score value ([2139705], '0.9640273') vs ([20795785], '0.9655802')", "query=KnnFloatVectorQuery:vector[0.02227773,...][100] filter=None sort=None groupField=None hitCount=100: hit 19 has wrong field/score value ([20249582], '0.9433427') vs ([8538823], '0.94324553')", "query=KnnFloatVectorQuery:vector[-0.047548626,...][100] filter=None sort=None groupField=None hitCount=100: hit 0 has wrong field/score value ([24831434], '0.84341675') vs ([20712471], '0.8335463')", "query=KnnFloatVectorQuery:vector[0.02625591,...][100] filter=None sort=None groupField=None hitCount=100: hit 6 has wrong field/score value ([25459412], '
 0.8309758') vs ([15548210], '0.8312737')"]
   ```


-- 
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] jmazanec15 commented on a diff in pull request #12050: Reuse HNSW graph for intialization during merge

Posted by GitBox <gi...@apache.org>.
jmazanec15 commented on code in PR #12050:
URL: https://github.com/apache/lucene/pull/12050#discussion_r1061991840


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java:
##########
@@ -143,10 +148,64 @@ public OnHeapHnswGraph build(RandomAccessVectorValues vectorsToAdd) throws IOExc
     return hnsw;
   }
 
+  /**
+   * Initializes the graph of this builder. Transfers the nodes and their neighbors from the
+   * initializer graph into the graph being produced by this builder, mapping ordinals from the
+   * initializer graph to their new ordinals in this builder's graph. The builder's graph must be
+   * empty before calling this method.
+   *
+   * @param initializerGraph graph used for initialization
+   * @param oldToNewOrdinalMap map for converting from ordinals in the initializerGraph to this
+   *     builder's graph
+   */
+  public void initializeFromGraph(
+      HnswGraph initializerGraph, Map<Integer, Integer> oldToNewOrdinalMap) throws IOException {
+    assert hnsw.size() == 0;
+    float[] vectorValue = null;
+    BytesRef binaryValue = null;
+    for (int level = 0; level < initializerGraph.numLevels(); level++) {
+      HnswGraph.NodesIterator it = initializerGraph.getNodesOnLevel(level);
+
+      while (it.hasNext()) {
+        int oldOrd = it.nextInt();
+        int newOrd = oldToNewOrdinalMap.get(oldOrd);
+
+        hnsw.addNode(level, newOrd);
+
+        if (level == 0) {
+          initializedNodes.add(newOrd);
+        }
+
+        switch (this.vectorEncoding) {
+          case FLOAT32 -> vectorValue = vectors.vectorValue(newOrd);
+          case BYTE -> binaryValue = vectors.binaryValue(newOrd);
+        }
+
+        NeighborArray newNeighbors = this.hnsw.getNeighbors(level, newOrd);
+        initializerGraph.seek(level, oldOrd);
+        for (int oldNeighbor = initializerGraph.nextNeighbor();
+            oldNeighbor != NO_MORE_DOCS;
+            oldNeighbor = initializerGraph.nextNeighbor()) {
+          int newNeighbor = oldToNewOrdinalMap.get(oldNeighbor);
+          float score =
+              switch (this.vectorEncoding) {
+                case FLOAT32 -> this.similarityFunction.compare(

Review Comment:
   > Is this sorted order only used for calculating diversity easier?
   
   Yes, I think you are correct. The  reason for sorting order by distance during construction is that the neighbor arrays of an inserted node continue to get updated as more nodes are inserted in. So keeping it sorted will allow the worst node or nodes will allow it to be more easily identified. 



-- 
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] jmazanec15 commented on pull request #12050: Reuse HNSW graph for intialization during merge

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

   @jpountz yes that's correct. The random number assignment is no longer going to be the same when merging multiple graphs together, because the segment whose graph is being used to initialize won't take any random numbers. Additionally, depending on the ordinals the vectors map to in the initializer graph, the neighbor assignment may be different. 


-- 
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] zhaih commented on pull request #12050: Reuse HNSW graph for intialization during merge

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

   > Do you mean create a new Codec version? From what I can tell, nothing in the underlying storage format has changed and the only reason Lucene95HnswVectorsReader is cast is for Lucene95HnswVectorsReader#getGraph, which already existed.
   
   @benwtrent You're right, I had an impression of this work was based on the newly created codec but yeah we don't need a new codec for it. Sorry for the confusion.


-- 
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] zhaih commented on a diff in pull request #12050: Reuse HNSW graph for intialization during merge

Posted by GitBox <gi...@apache.org>.
zhaih commented on code in PR #12050:
URL: https://github.com/apache/lucene/pull/12050#discussion_r1061007997


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/Lucene95HnswVectorsWriter.java:
##########
@@ -461,6 +467,126 @@ public void mergeOneField(FieldInfo fieldInfo, MergeState mergeState) throws IOE
     }
   }
 
+  private void maybeInitializeFromGraph(
+      HnswGraphBuilder<?> hnswGraphBuilder, MergeState mergeState, FieldInfo fieldInfo)
+      throws IOException {
+    int initializerIndex = selectGraphForInitialization(mergeState, fieldInfo);
+    if (initializerIndex == -1) {
+      return;
+    }
+
+    HnswGraph initializerGraph =
+        getHnswGraphFromReader(fieldInfo.name, mergeState.knnVectorsReaders[initializerIndex]);
+    Map<Integer, Integer> ordinalMapper =
+        getOldToNewOrdinalMap(mergeState, fieldInfo, initializerIndex);
+    hnswGraphBuilder.initializeFromGraph(initializerGraph, ordinalMapper);
+  }
+
+  private int selectGraphForInitialization(MergeState mergeState, FieldInfo fieldInfo)
+      throws IOException {
+    // Find the KnnVectorReader with the most docs that meets the following criteria:
+    //  1. Does not contain any deleted docs
+    //  2. Is a Lucene95HnswVectorsReader/PerFieldKnnVectorReader
+    // If no readers exist that meet this criteria, return -1. If they do, return their index in
+    // merge state
+    int maxCandidateVectorCount = 0;
+    int initializerIndex = -1;
+
+    for (int i = 0; i < mergeState.liveDocs.length; i++) {
+      KnnVectorsReader currKnnVectorsReader = mergeState.knnVectorsReaders[i];
+      if (mergeState.knnVectorsReaders[i]
+          instanceof PerFieldKnnVectorsFormat.FieldsReader candidateReader) {
+        currKnnVectorsReader = candidateReader.getFieldReader(fieldInfo.name);
+      }
+
+      if (!allMatch(mergeState.liveDocs[i])
+          || !(currKnnVectorsReader instanceof Lucene95HnswVectorsReader candidateReader)) {
+        continue;
+      }
+
+      VectorValues vectorValues = candidateReader.getVectorValues(fieldInfo.name);
+      if (vectorValues == null) {
+        continue;
+      }
+
+      int candidateVectorCount = vectorValues.size();
+      if (candidateVectorCount > maxCandidateVectorCount) {
+        maxCandidateVectorCount = candidateVectorCount;
+        initializerIndex = i;
+      }
+    }
+    return initializerIndex;
+  }
+
+  private HnswGraph getHnswGraphFromReader(String fieldName, KnnVectorsReader knnVectorsReader)
+      throws IOException {
+    if (knnVectorsReader instanceof PerFieldKnnVectorsFormat.FieldsReader perFieldReader
+        && perFieldReader.getFieldReader(fieldName)
+            instanceof Lucene95HnswVectorsReader fieldReader) {
+      return fieldReader.getGraph(fieldName);
+    }
+
+    if (knnVectorsReader instanceof Lucene95HnswVectorsReader) {
+      return ((Lucene95HnswVectorsReader) knnVectorsReader).getGraph(fieldName);
+    }
+
+    throw new IllegalArgumentException(
+        "Invalid KnnVectorsReader. Must be of type PerFieldKnnVectorsFormat.FieldsReader or Lucene94HnswVectorsReader");
+  }
+
+  private Map<Integer, Integer> getOldToNewOrdinalMap(
+      MergeState mergeState, FieldInfo fieldInfo, int initializerIndex) throws IOException {
+    VectorValues initializerVectorValues =
+        mergeState.knnVectorsReaders[initializerIndex].getVectorValues(fieldInfo.name);
+    MergeState.DocMap initializerDocMap = mergeState.docMaps[initializerIndex];
+
+    Map<Integer, Integer> newIdToOldOrdinal = new HashMap<>();
+    int oldOrd = 0;
+    for (int oldId = initializerVectorValues.nextDoc();
+        oldId != NO_MORE_DOCS;
+        oldId = initializerVectorValues.nextDoc()) {
+      if (initializerVectorValues.vectorValue() == null) {
+        continue;
+      }
+      int newId = initializerDocMap.get(oldId);
+      newIdToOldOrdinal.put(newId, oldOrd);
+      oldOrd++;
+    }
+
+    Map<Integer, Integer> oldToNewOrdinalMap = new HashMap<>();
+    int newOrd = 0;
+    int maxNewDocID = Collections.max(newIdToOldOrdinal.keySet());

Review Comment:
   It might be a bit faster to calculate this max in the previous loop?



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/Lucene95HnswVectorsWriter.java:
##########
@@ -461,6 +467,126 @@ public void mergeOneField(FieldInfo fieldInfo, MergeState mergeState) throws IOE
     }
   }
 
+  private void maybeInitializeFromGraph(
+      HnswGraphBuilder<?> hnswGraphBuilder, MergeState mergeState, FieldInfo fieldInfo)
+      throws IOException {
+    int initializerIndex = selectGraphForInitialization(mergeState, fieldInfo);
+    if (initializerIndex == -1) {
+      return;
+    }
+
+    HnswGraph initializerGraph =
+        getHnswGraphFromReader(fieldInfo.name, mergeState.knnVectorsReaders[initializerIndex]);
+    Map<Integer, Integer> ordinalMapper =
+        getOldToNewOrdinalMap(mergeState, fieldInfo, initializerIndex);
+    hnswGraphBuilder.initializeFromGraph(initializerGraph, ordinalMapper);
+  }
+
+  private int selectGraphForInitialization(MergeState mergeState, FieldInfo fieldInfo)
+      throws IOException {
+    // Find the KnnVectorReader with the most docs that meets the following criteria:
+    //  1. Does not contain any deleted docs
+    //  2. Is a Lucene95HnswVectorsReader/PerFieldKnnVectorReader
+    // If no readers exist that meet this criteria, return -1. If they do, return their index in
+    // merge state
+    int maxCandidateVectorCount = 0;
+    int initializerIndex = -1;
+
+    for (int i = 0; i < mergeState.liveDocs.length; i++) {
+      KnnVectorsReader currKnnVectorsReader = mergeState.knnVectorsReaders[i];
+      if (mergeState.knnVectorsReaders[i]
+          instanceof PerFieldKnnVectorsFormat.FieldsReader candidateReader) {
+        currKnnVectorsReader = candidateReader.getFieldReader(fieldInfo.name);
+      }
+
+      if (!allMatch(mergeState.liveDocs[i])
+          || !(currKnnVectorsReader instanceof Lucene95HnswVectorsReader candidateReader)) {
+        continue;
+      }
+
+      VectorValues vectorValues = candidateReader.getVectorValues(fieldInfo.name);
+      if (vectorValues == null) {
+        continue;
+      }
+
+      int candidateVectorCount = vectorValues.size();
+      if (candidateVectorCount > maxCandidateVectorCount) {
+        maxCandidateVectorCount = candidateVectorCount;
+        initializerIndex = i;
+      }
+    }
+    return initializerIndex;
+  }
+
+  private HnswGraph getHnswGraphFromReader(String fieldName, KnnVectorsReader knnVectorsReader)
+      throws IOException {
+    if (knnVectorsReader instanceof PerFieldKnnVectorsFormat.FieldsReader perFieldReader
+        && perFieldReader.getFieldReader(fieldName)
+            instanceof Lucene95HnswVectorsReader fieldReader) {
+      return fieldReader.getGraph(fieldName);
+    }
+
+    if (knnVectorsReader instanceof Lucene95HnswVectorsReader) {
+      return ((Lucene95HnswVectorsReader) knnVectorsReader).getGraph(fieldName);
+    }
+
+    throw new IllegalArgumentException(
+        "Invalid KnnVectorsReader. Must be of type PerFieldKnnVectorsFormat.FieldsReader or Lucene94HnswVectorsReader");

Review Comment:
   Maybe say:
   `"Invalid KnnVectorsReader type for field: " + fieldName + ". Must be Lucene95HnswVectorsReader or newer"`?



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/Lucene95HnswVectorsWriter.java:
##########
@@ -461,6 +467,126 @@ public void mergeOneField(FieldInfo fieldInfo, MergeState mergeState) throws IOE
     }
   }
 
+  private void maybeInitializeFromGraph(
+      HnswGraphBuilder<?> hnswGraphBuilder, MergeState mergeState, FieldInfo fieldInfo)
+      throws IOException {
+    int initializerIndex = selectGraphForInitialization(mergeState, fieldInfo);
+    if (initializerIndex == -1) {
+      return;
+    }
+
+    HnswGraph initializerGraph =
+        getHnswGraphFromReader(fieldInfo.name, mergeState.knnVectorsReaders[initializerIndex]);
+    Map<Integer, Integer> ordinalMapper =
+        getOldToNewOrdinalMap(mergeState, fieldInfo, initializerIndex);
+    hnswGraphBuilder.initializeFromGraph(initializerGraph, ordinalMapper);
+  }
+
+  private int selectGraphForInitialization(MergeState mergeState, FieldInfo fieldInfo)
+      throws IOException {
+    // Find the KnnVectorReader with the most docs that meets the following criteria:
+    //  1. Does not contain any deleted docs
+    //  2. Is a Lucene95HnswVectorsReader/PerFieldKnnVectorReader
+    // If no readers exist that meet this criteria, return -1. If they do, return their index in
+    // merge state
+    int maxCandidateVectorCount = 0;
+    int initializerIndex = -1;
+
+    for (int i = 0; i < mergeState.liveDocs.length; i++) {
+      KnnVectorsReader currKnnVectorsReader = mergeState.knnVectorsReaders[i];
+      if (mergeState.knnVectorsReaders[i]
+          instanceof PerFieldKnnVectorsFormat.FieldsReader candidateReader) {
+        currKnnVectorsReader = candidateReader.getFieldReader(fieldInfo.name);
+      }
+
+      if (!allMatch(mergeState.liveDocs[i])
+          || !(currKnnVectorsReader instanceof Lucene95HnswVectorsReader candidateReader)) {
+        continue;
+      }
+
+      VectorValues vectorValues = candidateReader.getVectorValues(fieldInfo.name);
+      if (vectorValues == null) {
+        continue;
+      }
+
+      int candidateVectorCount = vectorValues.size();
+      if (candidateVectorCount > maxCandidateVectorCount) {
+        maxCandidateVectorCount = candidateVectorCount;
+        initializerIndex = i;
+      }
+    }
+    return initializerIndex;
+  }
+
+  private HnswGraph getHnswGraphFromReader(String fieldName, KnnVectorsReader knnVectorsReader)
+      throws IOException {
+    if (knnVectorsReader instanceof PerFieldKnnVectorsFormat.FieldsReader perFieldReader
+        && perFieldReader.getFieldReader(fieldName)
+            instanceof Lucene95HnswVectorsReader fieldReader) {
+      return fieldReader.getGraph(fieldName);
+    }
+
+    if (knnVectorsReader instanceof Lucene95HnswVectorsReader) {
+      return ((Lucene95HnswVectorsReader) knnVectorsReader).getGraph(fieldName);
+    }
+

Review Comment:
   Can we also add a comment indicating we shouldn't really reach here because the reader type should be already checked inside `selectGraphForInitialization`?



##########
lucene/core/src/java/org/apache/lucene/util/hnsw/OnHeapHnswGraph.java:
##########
@@ -94,36 +93,83 @@ public int size() {
   }
 
   /**
-   * Add node on the given level
+   * Add node on the given level. Nodes can be inserted out of order, but it requires that the nodes

Review Comment:
   Since we need out-of-order insertion, I wonder whether it could be better if we have another implementation of OnHeapHnswGraph where it uses BST for all layers other than layer 0?



##########
lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java:
##########
@@ -143,10 +148,64 @@ public OnHeapHnswGraph build(RandomAccessVectorValues vectorsToAdd) throws IOExc
     return hnsw;
   }
 
+  /**
+   * Initializes the graph of this builder. Transfers the nodes and their neighbors from the
+   * initializer graph into the graph being produced by this builder, mapping ordinals from the
+   * initializer graph to their new ordinals in this builder's graph. The builder's graph must be
+   * empty before calling this method.
+   *
+   * @param initializerGraph graph used for initialization
+   * @param oldToNewOrdinalMap map for converting from ordinals in the initializerGraph to this
+   *     builder's graph
+   */
+  public void initializeFromGraph(
+      HnswGraph initializerGraph, Map<Integer, Integer> oldToNewOrdinalMap) throws IOException {
+    assert hnsw.size() == 0;
+    float[] vectorValue = null;
+    BytesRef binaryValue = null;
+    for (int level = 0; level < initializerGraph.numLevels(); level++) {
+      HnswGraph.NodesIterator it = initializerGraph.getNodesOnLevel(level);
+
+      while (it.hasNext()) {
+        int oldOrd = it.nextInt();
+        int newOrd = oldToNewOrdinalMap.get(oldOrd);
+
+        hnsw.addNode(level, newOrd);
+
+        if (level == 0) {
+          initializedNodes.add(newOrd);
+        }
+
+        switch (this.vectorEncoding) {
+          case FLOAT32 -> vectorValue = vectors.vectorValue(newOrd);
+          case BYTE -> binaryValue = vectors.binaryValue(newOrd);
+        }
+
+        NeighborArray newNeighbors = this.hnsw.getNeighbors(level, newOrd);
+        initializerGraph.seek(level, oldOrd);
+        for (int oldNeighbor = initializerGraph.nextNeighbor();
+            oldNeighbor != NO_MORE_DOCS;
+            oldNeighbor = initializerGraph.nextNeighbor()) {
+          int newNeighbor = oldToNewOrdinalMap.get(oldNeighbor);
+          float score =
+              switch (this.vectorEncoding) {
+                case FLOAT32 -> this.similarityFunction.compare(

Review Comment:
   I wonder if those scores are lazily calculated whether we can save some time here?
   Since for all the nodes in the initializer we already know their order and we don't need their score as long as there's no new nodes inserted?
   When there's a new node we just do the binary search as usual and calculate scores 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] jmazanec15 commented on a diff in pull request #12050: Reuse HNSW graph for intialization during merge

Posted by GitBox <gi...@apache.org>.
jmazanec15 commented on code in PR #12050:
URL: https://github.com/apache/lucene/pull/12050#discussion_r1061902422


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/Lucene95HnswVectorsWriter.java:
##########
@@ -461,6 +467,126 @@ public void mergeOneField(FieldInfo fieldInfo, MergeState mergeState) throws IOE
     }
   }
 
+  private void maybeInitializeFromGraph(
+      HnswGraphBuilder<?> hnswGraphBuilder, MergeState mergeState, FieldInfo fieldInfo)
+      throws IOException {
+    int initializerIndex = selectGraphForInitialization(mergeState, fieldInfo);
+    if (initializerIndex == -1) {
+      return;
+    }
+
+    HnswGraph initializerGraph =
+        getHnswGraphFromReader(fieldInfo.name, mergeState.knnVectorsReaders[initializerIndex]);
+    Map<Integer, Integer> ordinalMapper =
+        getOldToNewOrdinalMap(mergeState, fieldInfo, initializerIndex);
+    hnswGraphBuilder.initializeFromGraph(initializerGraph, ordinalMapper);
+  }
+
+  private int selectGraphForInitialization(MergeState mergeState, FieldInfo fieldInfo)
+      throws IOException {
+    // Find the KnnVectorReader with the most docs that meets the following criteria:
+    //  1. Does not contain any deleted docs
+    //  2. Is a Lucene95HnswVectorsReader/PerFieldKnnVectorReader
+    // If no readers exist that meet this criteria, return -1. If they do, return their index in
+    // merge state
+    int maxCandidateVectorCount = 0;
+    int initializerIndex = -1;
+
+    for (int i = 0; i < mergeState.liveDocs.length; i++) {
+      KnnVectorsReader currKnnVectorsReader = mergeState.knnVectorsReaders[i];
+      if (mergeState.knnVectorsReaders[i]
+          instanceof PerFieldKnnVectorsFormat.FieldsReader candidateReader) {
+        currKnnVectorsReader = candidateReader.getFieldReader(fieldInfo.name);
+      }
+
+      if (!allMatch(mergeState.liveDocs[i])
+          || !(currKnnVectorsReader instanceof Lucene95HnswVectorsReader candidateReader)) {
+        continue;
+      }
+
+      VectorValues vectorValues = candidateReader.getVectorValues(fieldInfo.name);
+      if (vectorValues == null) {
+        continue;
+      }
+
+      int candidateVectorCount = vectorValues.size();
+      if (candidateVectorCount > maxCandidateVectorCount) {
+        maxCandidateVectorCount = candidateVectorCount;
+        initializerIndex = i;
+      }
+    }
+    return initializerIndex;
+  }
+
+  private HnswGraph getHnswGraphFromReader(String fieldName, KnnVectorsReader knnVectorsReader)
+      throws IOException {
+    if (knnVectorsReader instanceof PerFieldKnnVectorsFormat.FieldsReader perFieldReader
+        && perFieldReader.getFieldReader(fieldName)
+            instanceof Lucene95HnswVectorsReader fieldReader) {
+      return fieldReader.getGraph(fieldName);
+    }
+
+    if (knnVectorsReader instanceof Lucene95HnswVectorsReader) {
+      return ((Lucene95HnswVectorsReader) knnVectorsReader).getGraph(fieldName);
+    }
+

Review Comment:
   Good idea, I will add this comment 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] jpountz commented on pull request #12050: Reuse HNSW graph for intialization during merge

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

   I think that the answer to my question is "yes" given this paragraph in the issue description: "In addition to this, graphs produced by merging two segments are no longer necessarily going to be equivalent to indexing one segment directly. This is caused by both differences in assigned random values as well as insertion order dictating which neighbors are selected for which nodes."
   
   @mikemccand Could you kick off a re-gold of nightly benchmarks?


-- 
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 merged pull request #12050: Reuse HNSW graph for intialization during merge

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent merged PR #12050:
URL: https://github.com/apache/lucene/pull/12050


-- 
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] jmazanec15 commented on pull request #12050: Reuse HNSW graph for intialization during merge

Posted by GitBox <gi...@apache.org>.
jmazanec15 commented on PR #12050:
URL: https://github.com/apache/lucene/pull/12050#issuecomment-1397643952

   Per [this discussion](https://github.com/apache/lucene/pull/12050#discussion_r1061034056), I refactored OnHeapHnswGraph to use a TreeMap to represent the graph structure for levels greater than 0. I ran performance tests with the same setup as https://github.com/apache/lucene/issues/11354#issuecomment-1239961308, and the results did not show a significant difference in indexing time between my previous implementation, the implementation using the map, and the current implementation with no merge optimization. Additionally, the results did not show a difference in merge time between by previous implementation and the implementation using the map.
   
   Here are the results:
   
   ###  Segment Size 10K
   
   
   Exper. | Total indexing time (s) | Total time to merge numeric vectors (ms) | Recall
   -- | -- | -- | --
   Control-1 | 189s | 697280 | 0.979
   Control-2 | 190s | 722042 | 0.979
   Control-3 | 191s | 713402 | 0.979
   Test-array 1 | 190s | 683966 | 0.98
   Test-array 2 | 187s | 683584 | 0.98
   Test-array 3 | 190s | 702458 | 0.98
   Test-map 1 | 189s | 723582 | 0.98
   Test-map 2 | 187s | 658196 | 0.98
   Test-map 3 | 190s | 667777 | 0.98
   
   ###  Segment Size 100K
   
   Exper. | Total indexing time (s) | Total time to merge numeric vectors (ms) | Recall
   -- | -- | -- | --
   Control-1 | 366s | 675361 | 0.981
   Control-2 | 370s | 695974 | 0.981
   Control-3 | 367s | 684418 | 0.981
   Test-array 1 | 368s | 651814 | 0.981
   Test-array 2 | 368s | 654862 | 0.981
   Test-array 3 | 368s | 656062 | 0.981
   Test-map 1  | 364s | 637257 | 0.981
   Test-map 2  | 370s | 628755 | 0.981
   Test-map 3 | 366s | 647569 | 0.981
   
   ###  Segment Size 500K
   
   Exper. | Total indexing time (s) | Total time to merge numeric vectors (ms) | Recall
   -- | -- | -- | --
   Control-1 | 633s | 655538 | 0.98
   Control-2 | 631s | 664622 | 0.98
   Control-3 | 627s | 635919 | 0.98
   Test-array 1 | 639s | 376139 | 0.98
   Test-array 2 | 636s | 378071 | 0.98
   Test-array 3 | 638s | 352633 | 0.98
   Test-map 1  | 645s | 373572 | 0.98
   Test-map 2  | 635s | 374309 | 0.98
   Test-map 3 | 633s | 381212 | 0.98
   
   Given that the results do not show a significant difference, I switched to use the treemap to avoid multiple large array copies.


-- 
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] zhaih commented on a diff in pull request #12050: Reuse HNSW graph for intialization during merge

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


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/Lucene95HnswVectorsWriter.java:
##########
@@ -489,6 +485,220 @@ public void mergeOneField(FieldInfo fieldInfo, MergeState mergeState) throws IOE
     }
   }
 
+  private HnswGraphBuilder<float[]> createFloatVectorHnswGraphBuilder(

Review Comment:
   I thought
   ```
   private <T> HnswGraphBuilder<T> createHnswGraphBuilder(
   ...
   RandomAccessVectorValues<T> vectorValues,
   ...)
   ```
   might work? Am I missing anything?



-- 
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] jmazanec15 commented on a diff in pull request #12050: Reuse HNSW graph for intialization during merge

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


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraph.java:
##########
@@ -182,10 +201,43 @@ public int nextInt() {
     public boolean hasNext() {
       return cur < size;
     }
+  }
 
-    /** The number of elements in this iterator * */
-    public int size() {
-      return size;
+  /** Nodes iterator based on set representation of nodes. */
+  public static class SetNodesIterator extends NodesIterator {

Review Comment:
   Yes, this is a good point. What about WrappedNodesIterator?



##########
lucene/core/src/java/org/apache/lucene/util/hnsw/OnHeapHnswGraph.java:
##########
@@ -33,44 +32,36 @@
 public final class OnHeapHnswGraph extends HnswGraph implements Accountable {
 
   private int numLevels; // the current number of levels in the graph
-  private int entryNode; // the current graph entry node on the top level
+  private int entryNode; // the current graph entry node on the top level. -1 if not set
 
-  // Nodes by level expressed as the level 0's nodes' ordinals.
-  // As level 0 contains all nodes, nodesByLevel.get(0) is null.
-  private final List<int[]> nodesByLevel;
-
-  // graph is a list of graph levels.
-  // Each level is represented as List<NeighborArray> – nodes' connections on this level.
+  // Level 0 is represented as List<NeighborArray> – nodes' connections on level 0.
   // Each entry in the list has the top maxConn/maxConn0 neighbors of a node. The nodes correspond
   // to vectors
   // added to HnswBuilder, and the node values are the ordinals of those vectors.
   // Thus, on all levels, neighbors expressed as the level 0's nodes' ordinals.
-  private final List<List<NeighborArray>> graph;
+  private final List<NeighborArray> graphLevel0;
+  // Represents levels 1-N. Each level is represented with a TreeMap that maps a levels level 0
+  // ordinal to its
+  // neighbors on that level.
+  private final List<TreeMap<Integer, NeighborArray>> graphUpperLevels;

Review Comment:
   Will add a 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] jmazanec15 commented on a diff in pull request #12050: Reuse HNSW graph for intialization during merge

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


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraph.java:
##########
@@ -182,10 +201,43 @@ public int nextInt() {
     public boolean hasNext() {
       return cur < size;
     }
+  }
 
-    /** The number of elements in this iterator * */
-    public int size() {
-      return size;
+  /** Nodes iterator based on set representation of nodes. */
+  public static class SetNodesIterator extends NodesIterator {

Review Comment:
   Actually, it is required to have a "size()" method, so a better name might be CollectionNodesIterator.



-- 
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] jmazanec15 commented on a diff in pull request #12050: Reuse HNSW graph for intialization during merge

Posted by GitBox <gi...@apache.org>.
jmazanec15 commented on code in PR #12050:
URL: https://github.com/apache/lucene/pull/12050#discussion_r1061905607


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java:
##########
@@ -143,10 +148,64 @@ public OnHeapHnswGraph build(RandomAccessVectorValues vectorsToAdd) throws IOExc
     return hnsw;
   }
 
+  /**
+   * Initializes the graph of this builder. Transfers the nodes and their neighbors from the
+   * initializer graph into the graph being produced by this builder, mapping ordinals from the
+   * initializer graph to their new ordinals in this builder's graph. The builder's graph must be
+   * empty before calling this method.
+   *
+   * @param initializerGraph graph used for initialization
+   * @param oldToNewOrdinalMap map for converting from ordinals in the initializerGraph to this
+   *     builder's graph
+   */
+  public void initializeFromGraph(
+      HnswGraph initializerGraph, Map<Integer, Integer> oldToNewOrdinalMap) throws IOException {
+    assert hnsw.size() == 0;
+    float[] vectorValue = null;
+    BytesRef binaryValue = null;
+    for (int level = 0; level < initializerGraph.numLevels(); level++) {
+      HnswGraph.NodesIterator it = initializerGraph.getNodesOnLevel(level);
+
+      while (it.hasNext()) {
+        int oldOrd = it.nextInt();
+        int newOrd = oldToNewOrdinalMap.get(oldOrd);
+
+        hnsw.addNode(level, newOrd);
+
+        if (level == 0) {
+          initializedNodes.add(newOrd);
+        }
+
+        switch (this.vectorEncoding) {
+          case FLOAT32 -> vectorValue = vectors.vectorValue(newOrd);
+          case BYTE -> binaryValue = vectors.binaryValue(newOrd);
+        }
+
+        NeighborArray newNeighbors = this.hnsw.getNeighbors(level, newOrd);
+        initializerGraph.seek(level, oldOrd);
+        for (int oldNeighbor = initializerGraph.nextNeighbor();
+            oldNeighbor != NO_MORE_DOCS;
+            oldNeighbor = initializerGraph.nextNeighbor()) {
+          int newNeighbor = oldToNewOrdinalMap.get(oldNeighbor);
+          float score =
+              switch (this.vectorEncoding) {
+                case FLOAT32 -> this.similarityFunction.compare(

Review Comment:
   Ah I thought about this, but the HnswGraph does not guarantee ordering of neighbors. For example, during writing, the neighbors get sorted to improve compression: https://github.com/apache/lucene/blob/main/lucene/core/src/java/org/apache/lucene/codecs/lucene95/Lucene95HnswVectorsWriter.java#L486-L493.



-- 
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 #12050: Reuse HNSW graph for intialization during merge

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


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/NeighborQueue.java:
##########
@@ -56,6 +56,8 @@ long apply(long v) {
   // Whether the search stopped early because it reached the visited nodes limit
   private boolean incomplete;
 
+  public static final NeighborQueue EMPTY_MAX_HEAP_NEIGHBOR_QUEUE = new NeighborQueue(1, true);

Review Comment:
   @jmazanec15 simply removing it and going back to the way it was (since all the following loops would be empty) should be OK imo. Either way I am good.



-- 
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] jmazanec15 commented on pull request #12050: Reuse HNSW graph for intialization during merge

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

   Thanks @benwtrent!


-- 
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 #12050: Reuse HNSW graph for intialization during merge

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

   @jmazanec15 did his due diligence, just being paranoid :). 
   
   I have confirmed that for the following ann-benchmarks datasets the recall before and after this change are 1-1: `mnist-784-euclidean`, `sift-128-euclidean`, `glove-100-angular`. However, all these datasets are pretty small, and may not kick off many segment merges, etc.
   
   So I tested with `deep-image-96-angular` and it took some time. 
   
   But here are the results:
   
   | parameters                                  | test recall | control recall |
   |---------------------------------------------|-------------|----------------|
   | {'M': 48, 'efConstruction': 100} fanout=100 | 0.995       | 0.994          |
   | {'M': 16, 'efConstruction': 100} fanout=100 | 0.986       | 0.986          |
   | {'M': 16, 'efConstruction': 100} fanout=50  | 0.969       | 0.969          |
   | {'M': 16, 'efConstruction': 100} fanout=500 | 0.998       | 0.998          |
   | {'M': 48, 'efConstruction': 100} fanout=500 | 0.999       | 0.999          |
   | {'M': 16, 'efConstruction': 100} fanout=10  | 0.892       | 0.892          |
   | {'M': 48, 'efConstruction': 100} fanout=50  | 0.986       | 0.986          |
   | {'M': 48, 'efConstruction': 100} fanout=10  | 0.941       | 0.940          |
   
   So, there are no significant changes in recall. So, I think this change is good and we should update the test.
   
   @jpountz 


-- 
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