You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by "jimczi (via GitHub)" <gi...@apache.org> on 2023/08/30 07:26:52 UTC

[GitHub] [lucene] jimczi opened a new pull request, #12529: Introduce a random vector scorer in HNSW builder/searcher

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

   This pull request (PR) involves the refactoring of the HNSW builder and searcher, aiming to create an abstraction for the random access and vector comparisons conducted during graph traversal.
   
   The newly added RandomVectorScorer provides a means to directly compare ordinals, eliminating the need to expose the raw vector primitive type. This scorer takes charge of vector retrieval and comparison during the graph's construction and search processes.
   
   The primary purpose of this abstraction is to enable the implementation of various strategies. For example, it opens the door to constructing the graph using the original float vectors while performing searches using their quantized int8 vector counterparts.


-- 
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 #12529: Introduce a random vector scorer in HNSW builder/searcher

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


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java:
##########
@@ -231,30 +184,34 @@ private void initializeFromGraph(
     }
   }
 
-  private void addVectors(RandomAccessVectorValues<T> vectorsToAdd) throws IOException {
+  /** Set info-stream to output debugging information * */
+  public void setInfoStream(InfoStream infoStream) {
+    this.infoStream = infoStream;
+  }
+
+  public OnHeapHnswGraph getGraph() {
+    return hnsw;
+  }
+
+  private void addVectors(RandomAccessVectorValues<?> vectorsToAdd) throws IOException {

Review Comment:
   Seems like this could be `private void addVectors(int maxOrd)`



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

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

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


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


[GitHub] [lucene] jimczi commented on pull request #12529: Introduce a random vector scorer in HNSW builder/searcher

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

   I merged with the latest changes in main, the new random vector scorer integrates nicely with the changes added `https://github.com/apache/lucene/pull/12480`. The only difference is that the scorer is now exposed in the API.


-- 
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 #12529: Introduce a random vector scorer in HNSW builder/searcher

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


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/RandomVectorScorerProvider.java:
##########
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.util.hnsw;
+
+import java.io.IOException;
+import org.apache.lucene.index.VectorSimilarityFunction;
+
+/** A provider that creates {@link RandomVectorScorer} from an ordinal. */
+public interface RandomVectorScorerProvider {
+  /**
+   * Creates a {@link RandomVectorScorer} to compare random nodes with the provided ordinal.
+   *
+   * @param ord the ordinal of the node to compare
+   * @return a new {@link RandomVectorScorer}
+   */
+  RandomVectorScorer scorer(int ord) throws IOException;

Review Comment:
   If we have this, do we really need `RandomVectorScorer#symmetricScore`? It seems `RandomVectorScorer#symmetricScore` is just `RandomVectorScorer#queryScore(int)` for when the stored `query` vector is another node in the 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] jimczi commented on a diff in pull request #12529: Introduce a random vector scorer in HNSW builder/searcher

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


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java:
##########
@@ -165,11 +134,7 @@ private HnswGraphBuilder(
    * @param vectorsToAdd the vectors for which to build a nearest neighbors graph. Must be an
    *     independent accessor for the vectors
    */
-  public OnHeapHnswGraph build(RandomAccessVectorValues<T> vectorsToAdd) throws IOException {
-    if (vectorsToAdd == this.vectors) {
-      throw new IllegalArgumentException(
-          "Vectors to build must be independent of the source of vectors provided to HnswGraphBuilder()");
-    }
+  public OnHeapHnswGraph build(RandomAccessVectorValues<?> vectorsToAdd) throws IOException {

Review Comment:
   +1, I pushed a change to modify the signature 



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

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

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


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


[GitHub] [lucene] jimczi commented on a diff in pull request #12529: Introduce a random vector scorer in HNSW builder/searcher

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


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/RandomVectorScorerProvider.java:
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.util.hnsw;
+
+import java.io.IOException;
+import org.apache.lucene.index.VectorSimilarityFunction;
+
+/** A provider that creates {@link RandomVectorScorer} from an ordinal. */
+public interface RandomVectorScorerProvider {
+  /**
+   * This creates a {@link RandomVectorScorer} for scoring random nodes in batches against the given
+   * ordinal.
+   *
+   * @param ord the ordinal of the node to compare
+   * @return a new {@link RandomVectorScorer}
+   */
+  RandomVectorScorer scorer(int ord) throws IOException;
+
+  /**
+   * Creates a {@link RandomVectorScorerProvider} to compare float vectors.
+   *
+   * <p>WARNING: The {@link RandomAccessVectorValues} given can contain stateful buffers. Avoid
+   * using it after calling this function. If you plan to use it again outside the returned {@link
+   * RandomVectorScorer}, think about passing a copied version ({@link
+   * RandomAccessVectorValues#copy}).
+   *
+   * @param vectors the underlying storage for vectors
+   * @param similarityFunction the similarity function to score vectors
+   */
+  static RandomVectorScorerProvider createFloats(
+      final RandomAccessVectorValues<float[]> vectors,
+      final VectorSimilarityFunction similarityFunction)
+      throws IOException {
+    final RandomAccessVectorValues<float[]> vectorsCopy = vectors.copy();
+    return queryOrd ->
+        (RandomVectorScorer)
+            cand ->
+                similarityFunction.compare(
+                    vectors.vectorValue(queryOrd), vectorsCopy.vectorValue(cand));

Review Comment:
   I'm trying limit the number of copies we make. In this model, we only make one copy for the entire supplier. We depend on the fact that when we call vectors.vectorValue twice in a row with the same order, it will use the previous value.
   
   Your suggestion would result in making a copy every time the supplier creates a scorer. Since we create a scorer every time we diversify a node, I believe this would have a noticeable impact.



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

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

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


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


[GitHub] [lucene] jimczi merged pull request #12529: Introduce a random vector scorer in HNSW builder/searcher

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


-- 
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 #12529: Introduce a random vector scorer in HNSW builder/searcher

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


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/RandomVectorScorerProvider.java:
##########
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.util.hnsw;
+
+import java.io.IOException;
+import org.apache.lucene.index.VectorSimilarityFunction;
+
+/** A provider that creates {@link RandomVectorScorer} from an ordinal. */
+public interface RandomVectorScorerProvider {
+  /**
+   * Creates a {@link RandomVectorScorer} to compare random nodes with the provided ordinal.
+   *
+   * @param ord the ordinal of the node to compare
+   * @return a new {@link RandomVectorScorer}
+   */
+  RandomVectorScorer scorer(int ord) throws IOException;
+
+  /**
+   * Creates a {@link RandomVectorScorerProvider} to compare float vectors.
+   *
+   * @param vectors the underlying storage for vectors
+   * @param similarityFunction the similarity function to score vectors
+   */
+  static RandomVectorScorerProvider createFloats(
+      final RandomAccessVectorValues<float[]> vectors,
+      final VectorSimilarityFunction similarityFunction) {
+    return ord ->
+        RandomVectorScorer.createFloats(
+            vectors.copy(), similarityFunction, vectors.vectorValue(ord));
+  }
+
+  /**
+   * Creates a {@link RandomVectorScorerProvider} to compare byte vectors.
+   *
+   * @param vectors the underlying storage for vectors
+   * @param similarityFunction the similarity function to score vectors
+   */
+  static RandomVectorScorerProvider createBytes(
+      final RandomAccessVectorValues<byte[]> vectors,
+      final VectorSimilarityFunction similarityFunction) {
+    return ord ->
+        RandomVectorScorer.createBytes(
+            vectors.copy(), similarityFunction, vectors.vectorValue(ord));

Review Comment:
   `RandomVectorScorer.createBytes(` also makes a `vectors.copy()`. I think this copy here in the provider should be removed.



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

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

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


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


[GitHub] [lucene] jimczi commented on a diff in pull request #12529: Introduce a random vector scorer in HNSW builder/searcher

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


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/RandomVectorScorerProvider.java:
##########
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.util.hnsw;
+
+import java.io.IOException;
+import org.apache.lucene.index.VectorSimilarityFunction;
+
+/** A provider that creates {@link RandomVectorScorer} from an ordinal. */
+public interface RandomVectorScorerProvider {
+  /**
+   * Creates a {@link RandomVectorScorer} to compare random nodes with the provided ordinal.
+   *
+   * @param ord the ordinal of the node to compare
+   * @return a new {@link RandomVectorScorer}
+   */
+  RandomVectorScorer scorer(int ord) throws IOException;
+
+  /**
+   * Creates a {@link RandomVectorScorerProvider} to compare float vectors.
+   *
+   * @param vectors the underlying storage for vectors
+   * @param similarityFunction the similarity function to score vectors
+   */
+  static RandomVectorScorerProvider createFloats(
+      final RandomAccessVectorValues<float[]> vectors,
+      final VectorSimilarityFunction similarityFunction) {
+    return ord ->
+        RandomVectorScorer.createFloats(
+            vectors.copy(), similarityFunction, vectors.vectorValue(ord));
+  }
+
+  /**
+   * Creates a {@link RandomVectorScorerProvider} to compare byte vectors.
+   *
+   * @param vectors the underlying storage for vectors
+   * @param similarityFunction the similarity function to score vectors
+   */
+  static RandomVectorScorerProvider createBytes(
+      final RandomAccessVectorValues<byte[]> vectors,
+      final VectorSimilarityFunction similarityFunction) {
+    return ord ->
+        RandomVectorScorer.createBytes(
+            vectors.copy(), similarityFunction, vectors.vectorValue(ord));

Review Comment:
   This whole copy dance is not easy to get right, thanks. I changed the methodology slightly to ensure that we're not overriding the buffer of another scorer in the latest change.



-- 
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 #12529: Introduce a random vector scorer in HNSW builder/searcher

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

   Well, actually looking at the JFR, I cannot see anything that stands out. The percentages of compute time are still VERY similar when building index & querying. I may just be detecting noise.
   
   `lucene_candidate` jfr is this PR, `lucene_baseline` jfr is latest main.
   [Archive.zip](https://github.com/apache/lucene/files/12550717/Archive.zip)
   


-- 
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 #12529: Introduce a random vector scorer in HNSW builder/searcher

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


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java:
##########
@@ -205,24 +168,14 @@ private void initializeFromGraph(
           initializedNodes.add(newOrd);
         }
 
-        switch (this.vectorEncoding) {
-          case FLOAT32 -> vectorValue = (float[]) vectors.vectorValue(newOrd);
-          case BYTE -> binaryValue = (byte[]) vectors.vectorValue(newOrd);
-        }
-
         NeighborArray newNeighbors = this.hnsw.getNeighbors(level, newOrd);
         initializerGraph.seek(level, oldOrd);
+        RandomVectorScorer scorer = scorerProvider.scorer(newOrd);
         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(
-                    vectorValue, (float[]) vectorsCopy.vectorValue(newNeighbor));
-                case BYTE -> this.similarityFunction.compare(
-                    binaryValue, (byte[]) vectorsCopy.vectorValue(newNeighbor));
-              };
+          float score = scorer.symmetricScore(newOrd, newNeighbor);

Review Comment:
   seems like since `scorer` is already wrapping `newOrd`, couldn't this be `scorer.score(newNeighbor)`?



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

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

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


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


[GitHub] [lucene] jimczi commented on a diff in pull request #12529: Introduce a random vector scorer in HNSW builder/searcher

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


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java:
##########
@@ -231,30 +184,34 @@ private void initializeFromGraph(
     }
   }
 
-  private void addVectors(RandomAccessVectorValues<T> vectorsToAdd) throws IOException {
+  /** Set info-stream to output debugging information * */
+  public void setInfoStream(InfoStream infoStream) {
+    this.infoStream = infoStream;
+  }
+
+  public OnHeapHnswGraph getGraph() {
+    return hnsw;
+  }
+
+  private void addVectors(RandomAccessVectorValues<?> vectorsToAdd) throws IOException {

Review Comment:
   +1, I pushed a change



-- 
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 a diff in pull request #12529: Introduce a random vector scorer in HNSW builder/searcher

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


##########
lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswVectorsReader.java:
##########
@@ -423,8 +422,12 @@ public RandomAccessVectorValues<float[]> copy() {
 
     @Override
     public float[] vectorValue(int targetOrd) throws IOException {
+      if (lastOrd == targetOrd) {
+        return value;
+      }

Review Comment:
   I'm curious, is it common to read the same ord multiple times in a row?



##########
lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/word2vec/Word2VecSynonymProvider.java:
##########
@@ -23,14 +23,11 @@
 import java.io.IOException;
 import java.util.LinkedList;
 import java.util.List;
-import org.apache.lucene.index.VectorEncoding;
 import org.apache.lucene.index.VectorSimilarityFunction;
 import org.apache.lucene.search.KnnCollector;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.hnsw.HnswGraphBuilder;
-import org.apache.lucene.util.hnsw.HnswGraphSearcher;
-import org.apache.lucene.util.hnsw.OnHeapHnswGraph;
+import org.apache.lucene.util.hnsw.*;

Review Comment:
   Nit: We prefer to avoid star imports.



##########
lucene/core/src/java/org/apache/lucene/util/hnsw/RandomVectorScorerProvider.java:
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.util.hnsw;
+
+import java.io.IOException;
+import org.apache.lucene.index.VectorSimilarityFunction;
+
+/** A provider that creates {@link RandomVectorScorer} from an ordinal. */
+public interface RandomVectorScorerProvider {

Review Comment:
   Naming nit: the naming convention seems to be to call things "Supplier"s rather than "Provider"s.



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/OffHeapByteVectorValues.java:
##########
@@ -60,13 +61,17 @@ public int size() {
 
   @Override
   public byte[] vectorValue(int targetOrd) throws IOException {
-    readValue(targetOrd);
+    if (lastOrd != targetOrd) {
+      readValue(targetOrd);
+      lastOrd = targetOrd;
+    }
     return binaryValue;
   }
 
   private void readValue(int targetOrd) throws IOException {
     slice.seek((long) targetOrd * byteSize);
     slice.readBytes(byteBuffer.array(), byteBuffer.arrayOffset(), byteSize);
+    lastOrd = targetOrd;

Review Comment:
   we already set `lastOrd` in `vectorValue()`, so it's not needed here?



##########
lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene94/Lucene94HnswVectorsWriter.java:
##########
@@ -47,12 +47,8 @@
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.InfoStream;
 import org.apache.lucene.util.RamUsageEstimator;
-import org.apache.lucene.util.hnsw.HnswGraph;
+import org.apache.lucene.util.hnsw.*;

Review Comment:
   nit: star import



##########
lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene92/Lucene92HnswVectorsWriter.java:
##########
@@ -34,16 +34,12 @@
 import org.apache.lucene.index.FloatVectorValues;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentWriteState;
-import org.apache.lucene.index.VectorEncoding;
 import org.apache.lucene.index.VectorSimilarityFunction;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.hnsw.HnswGraphBuilder;
-import org.apache.lucene.util.hnsw.NeighborArray;
-import org.apache.lucene.util.hnsw.OnHeapHnswGraph;
-import org.apache.lucene.util.hnsw.RandomAccessVectorValues;
+import org.apache.lucene.util.hnsw.*;

Review Comment:
   nit: star import



##########
lucene/core/src/java/org/apache/lucene/util/hnsw/RandomVectorScorerProvider.java:
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.util.hnsw;
+
+import java.io.IOException;
+import org.apache.lucene.index.VectorSimilarityFunction;
+
+/** A provider that creates {@link RandomVectorScorer} from an ordinal. */
+public interface RandomVectorScorerProvider {
+  /**
+   * This creates a {@link RandomVectorScorer} for scoring random nodes in batches against the given
+   * ordinal.
+   *
+   * @param ord the ordinal of the node to compare
+   * @return a new {@link RandomVectorScorer}
+   */
+  RandomVectorScorer scorer(int ord) throws IOException;
+
+  /**
+   * Creates a {@link RandomVectorScorerProvider} to compare float vectors.
+   *
+   * <p>WARNING: The {@link RandomAccessVectorValues} given can contain stateful buffers. Avoid
+   * using it after calling this function. If you plan to use it again outside the returned {@link
+   * RandomVectorScorer}, think about passing a copied version ({@link
+   * RandomAccessVectorValues#copy}).
+   *
+   * @param vectors the underlying storage for vectors
+   * @param similarityFunction the similarity function to score vectors
+   */
+  static RandomVectorScorerProvider createFloats(
+      final RandomAccessVectorValues<float[]> vectors,
+      final VectorSimilarityFunction similarityFunction)
+      throws IOException {
+    final RandomAccessVectorValues<float[]> vectorsCopy = vectors.copy();
+    return queryOrd ->
+        (RandomVectorScorer)
+            cand ->
+                similarityFunction.compare(
+                    vectors.vectorValue(queryOrd), vectorsCopy.vectorValue(cand));

Review Comment:
   Should we retrieve the query vector only once, rather than on every call? Then we could skip making a copy of the `RandomAccessVectorValues` as well? Or would it break something?



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/Lucene95HnswVectorsWriter.java:
##########
@@ -41,12 +41,8 @@
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.*;
-import org.apache.lucene.util.hnsw.HnswGraph;
+import org.apache.lucene.util.hnsw.*;

Review Comment:
   nit: star import



##########
lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene91/Lucene91HnswVectorsReader.java:
##########
@@ -42,9 +41,7 @@
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.RamUsageEstimator;
-import org.apache.lucene.util.hnsw.HnswGraph;
-import org.apache.lucene.util.hnsw.HnswGraphSearcher;
-import org.apache.lucene.util.hnsw.RandomAccessVectorValues;
+import org.apache.lucene.util.hnsw.*;

Review Comment:
   nit: star import



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

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

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


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


[GitHub] [lucene] jimczi commented on a diff in pull request #12529: Introduce a random vector scorer in HNSW builder/searcher

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


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/RandomVectorScorerProvider.java:
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.util.hnsw;
+
+import java.io.IOException;
+import org.apache.lucene.index.VectorSimilarityFunction;
+
+/** A provider that creates {@link RandomVectorScorer} from an ordinal. */
+public interface RandomVectorScorerProvider {

Review Comment:
   Addressed in https://github.com/apache/lucene/pull/12529/commits/31d8b549e62a288358c58cd666767d09bf17f65c



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

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

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


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


[GitHub] [lucene] jimczi commented on a diff in pull request #12529: Introduce a random vector scorer in HNSW builder/searcher

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


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java:
##########
@@ -366,38 +324,11 @@ private void popToScratch(GraphBuilderKnnCollector candidates) {
    * @param neighbors the neighbors selected so far
    * @return whether the candidate is diverse given the existing neighbors
    */
-  private boolean diversityCheck(int candidate, float score, NeighborArray neighbors)
-      throws IOException {
-    return isDiverse(candidate, neighbors, score);
-  }
-
-  private boolean isDiverse(int candidate, NeighborArray neighbors, float score)
-      throws IOException {
-    return switch (vectorEncoding) {
-      case BYTE -> isDiverse((byte[]) vectors.vectorValue(candidate), neighbors, score);
-      case FLOAT32 -> isDiverse((float[]) vectors.vectorValue(candidate), neighbors, score);
-    };
-  }
-
-  private boolean isDiverse(float[] candidate, NeighborArray neighbors, float score)
-      throws IOException {
-    for (int i = 0; i < neighbors.size(); i++) {
-      float neighborSimilarity =
-          similarityFunction.compare(
-              candidate, (float[]) vectorsCopy.vectorValue(neighbors.node[i]));
-      if (neighborSimilarity >= score) {
-        return false;
-      }
-    }
-    return true;
-  }
-
-  private boolean isDiverse(byte[] candidate, NeighborArray neighbors, float score)
+  private boolean diversityCheck(
+      RandomVectorScorer scorer, int candidate, float score, NeighborArray neighbors)
       throws IOException {
     for (int i = 0; i < neighbors.size(); i++) {
-      float neighborSimilarity =
-          similarityFunction.compare(
-              candidate, (byte[]) vectorsCopy.vectorValue(neighbors.node[i]));
+      float neighborSimilarity = scorer.symmetricScore(candidate, neighbors.node[i]);

Review Comment:
   That's a lot cleaner, I pushed a change to reflect your idea.



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

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

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12529: Introduce a random vector scorer in HNSW builder/searcher

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


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/RandomVectorScorer.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.util.hnsw;
+
+import java.io.IOException;
+import org.apache.lucene.index.VectorSimilarityFunction;
+import org.apache.lucene.search.Query;
+
+/**
+ * The {@link RandomVectorScorer} calculates scores between an abstract query and a random node. It
+ * can also score two random nodes stored in the graph. The scores returned must be strictly
+ * positive for them to be usable in a {@link Query}.
+ */
+public interface RandomVectorScorer {
+  /**
+   * Returns the score between the query and the provided node.
+   *
+   * @param node a random node in the graph
+   * @return the computed score
+   */
+  float queryScore(int node) throws IOException;
+
+  /**
+   * Returns the score between two nodes.
+   *
+   * @param node1 the first node
+   * @param node2 the second node
+   * @return the computed score
+   */
+  float symmetricScore(int node1, int node2) throws IOException;
+
+  /**
+   * Creates a default scorer for float vectors.
+   *
+   * @param vectors the underlying storage for vectors
+   * @param similarityFunction the similarity function to score vectors
+   * @param query the actual query
+   */
+  static RandomVectorScorer createFloats(
+      final RandomAccessVectorValues<float[]> vectors,
+      final VectorSimilarityFunction similarityFunction,
+      final float[] query)
+      throws IOException {
+    if (query.length != vectors.dimension()) {
+      throw new IllegalArgumentException(
+          "vector query dimension: "
+              + query.length
+              + " differs from field dimension: "
+              + vectors.dimension());
+    }
+    final RandomAccessVectorValues<float[]> vectorsCopy = vectors.copy();
+    return new RandomVectorScorer() {
+      @Override
+      public float symmetricScore(int node1, int node2) throws IOException {
+        return similarityFunction.compare(
+            vectors.vectorValue(node1), vectorsCopy.vectorValue(node2));
+      }
+
+      @Override
+      public float queryScore(int node) throws IOException {
+        return similarityFunction.compare(query, vectorsCopy.vectorValue(node));
+      }
+    };
+  }
+
+  /**
+   * Creates a default scorer for byte vectors.
+   *
+   * @param vectors the underlying storage for vectors
+   * @param similarityFunction the similarity function to use to score vectors
+   * @param query the actual query
+   */
+  static RandomVectorScorer createBytes(
+      final RandomAccessVectorValues<byte[]> vectors,
+      final VectorSimilarityFunction similarityFunction,
+      final byte[] query)

Review Comment:
   This `byte[]` reference is held through the lifetime of the `RandomVectorScorer`, but the provider of the array could change things. 
   
   For example `RandomVectorScorerProvider` provides the current query object from `RandomAccessVectorValues` and that `RandomAccessVectorValues` could call `.vectorValue(someOtherOrd)`.
   
   Seems like a fragile API.
   
   I think we should:
    - Be very clear that the array isn't copied
    - Copy the array here in this method
    - Copy the array when `RandomVectorScorerProvider` calls it.
   
   Part of me worries somebody will misuse this slightly and we just need to be careful.



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

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

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


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


[GitHub] [lucene] jimczi commented on a diff in pull request #12529: Introduce a random vector scorer in HNSW builder/searcher

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


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java:
##########
@@ -205,24 +168,14 @@ private void initializeFromGraph(
           initializedNodes.add(newOrd);
         }
 
-        switch (this.vectorEncoding) {
-          case FLOAT32 -> vectorValue = (float[]) vectors.vectorValue(newOrd);
-          case BYTE -> binaryValue = (byte[]) vectors.vectorValue(newOrd);
-        }
-
         NeighborArray newNeighbors = this.hnsw.getNeighbors(level, newOrd);
         initializerGraph.seek(level, oldOrd);
+        RandomVectorScorer scorer = scorerProvider.scorer(newOrd);
         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(
-                    vectorValue, (float[]) vectorsCopy.vectorValue(newNeighbor));
-                case BYTE -> this.similarityFunction.compare(
-                    binaryValue, (byte[]) vectorsCopy.vectorValue(newNeighbor));
-              };
+          float score = scorer.symmetricScore(newOrd, newNeighbor);

Review Comment:
   Yep thanks, I pushed a fix (removing `symmetricScore` entirely).



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

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

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


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


[GitHub] [lucene] jimczi commented on a diff in pull request #12529: Introduce a random vector scorer in HNSW builder/searcher

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


##########
lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene91/Lucene91HnswVectorsReader.java:
##########
@@ -42,9 +41,7 @@
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.RamUsageEstimator;
-import org.apache.lucene.util.hnsw.HnswGraph;
-import org.apache.lucene.util.hnsw.HnswGraphSearcher;
-import org.apache.lucene.util.hnsw.RandomAccessVectorValues;
+import org.apache.lucene.util.hnsw.*;

Review Comment:
   I have to remove this automatic rule from my idea. I upgraded to a new version and forgot to copy my previous settings. Thanks for spotting!



-- 
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 #12529: Introduce a random vector scorer in HNSW builder/searcher

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

   Ran a benchmark on Lucene util and here are the results, candidate (this PR) is consistently slightly slower.
   
   ```
   recall	latency	nDoc	fanout	maxConn	beamWidth  visited	index ms
   lucene_baseline
   0.980	 0.65	50000	100	32	100	   110	        16397
   lucene_candidate
   0.980	 0.67	50000	100	32	100	   110	        16481
   lucene_baseline
   0.991	 0.93	50000	100	32	250	   110	        51686
   lucene_candidate
   0.991	 0.96	50000	100	32	250	   110	        51721
   lucene_baseline
   0.979	 0.69	50000	100	96	100	   110	        16675
   lucene_candidate
   0.979	 0.71	50000	100	96	100	   110	        16713
   lucene_baseline 
   0.992	 0.99	50000	100	96	250	   110	        54563
   lucene_candidate
   0.992	 1.04	50000	100	96	250	   110	        54599
   ```
   
   Almost for sure it has to do with creating more objects. Not sure how to prevent that with a nice api


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

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

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


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


[GitHub] [lucene] jimczi commented on a diff in pull request #12529: Introduce a random vector scorer in HNSW builder/searcher

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


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/RandomVectorScorerProvider.java:
##########
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.util.hnsw;
+
+import java.io.IOException;
+import org.apache.lucene.index.VectorSimilarityFunction;
+
+/** A provider that creates {@link RandomVectorScorer} from an ordinal. */
+public interface RandomVectorScorerProvider {
+  /**
+   * Creates a {@link RandomVectorScorer} to compare random nodes with the provided ordinal.
+   *
+   * @param ord the ordinal of the node to compare
+   * @return a new {@link RandomVectorScorer}
+   */
+  RandomVectorScorer scorer(int ord) throws IOException;

Review Comment:
   I agree that it would be cleaner. It's a bit tricky if we want to avoid copying the vectors but I pushed a change to try this approach.



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

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

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


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


[GitHub] [lucene] jimczi commented on a diff in pull request #12529: Introduce a random vector scorer in HNSW builder/searcher

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


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/RandomVectorScorer.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.util.hnsw;
+
+import java.io.IOException;
+import org.apache.lucene.index.VectorSimilarityFunction;
+import org.apache.lucene.search.Query;
+
+/**
+ * The {@link RandomVectorScorer} calculates scores between an abstract query and a random node. It
+ * can also score two random nodes stored in the graph. The scores returned must be strictly
+ * positive for them to be usable in a {@link Query}.
+ */
+public interface RandomVectorScorer {
+  /**
+   * Returns the score between the query and the provided node.
+   *
+   * @param node a random node in the graph
+   * @return the computed score
+   */
+  float queryScore(int node) throws IOException;
+
+  /**
+   * Returns the score between two nodes.
+   *
+   * @param node1 the first node
+   * @param node2 the second node
+   * @return the computed score
+   */
+  float symmetricScore(int node1, int node2) throws IOException;
+
+  /**
+   * Creates a default scorer for float vectors.
+   *
+   * @param vectors the underlying storage for vectors
+   * @param similarityFunction the similarity function to score vectors
+   * @param query the actual query
+   */
+  static RandomVectorScorer createFloats(
+      final RandomAccessVectorValues<float[]> vectors,
+      final VectorSimilarityFunction similarityFunction,
+      final float[] query)
+      throws IOException {
+    if (query.length != vectors.dimension()) {
+      throw new IllegalArgumentException(
+          "vector query dimension: "
+              + query.length
+              + " differs from field dimension: "
+              + vectors.dimension());
+    }
+    final RandomAccessVectorValues<float[]> vectorsCopy = vectors.copy();
+    return new RandomVectorScorer() {
+      @Override
+      public float symmetricScore(int node1, int node2) throws IOException {
+        return similarityFunction.compare(
+            vectors.vectorValue(node1), vectorsCopy.vectorValue(node2));
+      }
+
+      @Override
+      public float queryScore(int node) throws IOException {
+        return similarityFunction.compare(query, vectorsCopy.vectorValue(node));
+      }
+    };
+  }
+
+  /**
+   * Creates a default scorer for byte vectors.
+   *
+   * @param vectors the underlying storage for vectors
+   * @param similarityFunction the similarity function to use to score vectors
+   * @param query the actual query
+   */
+  static RandomVectorScorer createBytes(
+      final RandomAccessVectorValues<byte[]> vectors,
+      final VectorSimilarityFunction similarityFunction,
+      final byte[] query)

Review Comment:
   I added a warning in the `RandomVectorScorerProvider` functions. I think it's fine to expect that the array is not modified during the lifecycle of `RandomAccessVectorValues`. For `RandomVectorScorerProvider` I changed the initialiser into a lazy lookup at query time to ensure that we don't modify the buffer of an existing `RandomVectorScorer`.



-- 
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 a diff in pull request #12529: Introduce a random vector scorer in HNSW builder/searcher

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


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/RandomVectorScorerProvider.java:
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.util.hnsw;
+
+import java.io.IOException;
+import org.apache.lucene.index.VectorSimilarityFunction;
+
+/** A provider that creates {@link RandomVectorScorer} from an ordinal. */
+public interface RandomVectorScorerProvider {
+  /**
+   * This creates a {@link RandomVectorScorer} for scoring random nodes in batches against the given
+   * ordinal.
+   *
+   * @param ord the ordinal of the node to compare
+   * @return a new {@link RandomVectorScorer}
+   */
+  RandomVectorScorer scorer(int ord) throws IOException;
+
+  /**
+   * Creates a {@link RandomVectorScorerProvider} to compare float vectors.
+   *
+   * <p>WARNING: The {@link RandomAccessVectorValues} given can contain stateful buffers. Avoid
+   * using it after calling this function. If you plan to use it again outside the returned {@link
+   * RandomVectorScorer}, think about passing a copied version ({@link
+   * RandomAccessVectorValues#copy}).
+   *
+   * @param vectors the underlying storage for vectors
+   * @param similarityFunction the similarity function to score vectors
+   */
+  static RandomVectorScorerProvider createFloats(
+      final RandomAccessVectorValues<float[]> vectors,
+      final VectorSimilarityFunction similarityFunction)
+      throws IOException {
+    final RandomAccessVectorValues<float[]> vectorsCopy = vectors.copy();
+    return queryOrd ->
+        (RandomVectorScorer)
+            cand ->
+                similarityFunction.compare(
+                    vectors.vectorValue(queryOrd), vectorsCopy.vectorValue(cand));

Review Comment:
   OK, thanks for explaining. Maybe leave a small comment about this since it was not obvious to me from looking at the code?



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

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

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


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


[GitHub] [lucene] jimczi commented on a diff in pull request #12529: Introduce a random vector scorer in HNSW builder/searcher

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


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphSearcher.java:
##########
@@ -172,106 +83,36 @@ public static KnnCollector search(
    * @return a set of collected vectors holding the nearest neighbors found
    */
   public static KnnCollector search(
-      byte[] query,
-      int topK,
-      RandomAccessVectorValues<byte[]> vectors,
-      VectorEncoding vectorEncoding,
-      VectorSimilarityFunction similarityFunction,
-      HnswGraph graph,
-      Bits acceptOrds,
-      int visitedLimit)
-      throws IOException {
-    KnnCollector collector = new TopKnnCollector(topK, visitedLimit);
-    search(query, collector, vectors, vectorEncoding, similarityFunction, graph, acceptOrds);
-    return collector;
-  }
-
-  /**
-   * Searches HNSW graph for the nearest neighbors of a query vector.
-   *
-   * @param query search query vector
-   * @param knnCollector a collector of top knn results to be returned
-   * @param vectors the vector values
-   * @param similarityFunction the similarity function to compare vectors
-   * @param graph the graph values. May represent the entire graph, or a level in a hierarchical
-   *     graph.
-   * @param acceptOrds {@link Bits} that represents the allowed document ordinals to match, or
-   *     {@code null} if they are all allowed to match.
-   */
-  public static void search(
-      byte[] query,
-      KnnCollector knnCollector,
-      RandomAccessVectorValues<byte[]> vectors,
-      VectorEncoding vectorEncoding,
-      VectorSimilarityFunction similarityFunction,
-      HnswGraph graph,
-      Bits acceptOrds)
-      throws IOException {
-    if (query.length != vectors.dimension()) {
-      throw new IllegalArgumentException(
-          "vector query dimension: "
-              + query.length
-              + " differs from field dimension: "
-              + vectors.dimension());
-    }
-    HnswGraphSearcher<byte[]> graphSearcher =
-        new HnswGraphSearcher<>(
-            vectorEncoding,
-            similarityFunction,
-            new NeighborQueue(knnCollector.k(), true),
-            new SparseFixedBitSet(vectors.size()));
-    search(query, knnCollector, vectors, graph, graphSearcher, acceptOrds);
-  }
-
-  /**
-   * Search {@link OnHeapHnswGraph}, this method is thread safe, for parameters please refer to
-   * {@link #search(byte[], int, RandomAccessVectorValues, VectorEncoding, VectorSimilarityFunction,
-   * HnswGraph, Bits, int)}
-   */
-  public static KnnCollector search(
-      byte[] query,
-      int topK,
-      RandomAccessVectorValues<byte[]> vectors,
-      VectorEncoding vectorEncoding,
-      VectorSimilarityFunction similarityFunction,
-      OnHeapHnswGraph graph,
-      Bits acceptOrds,
-      int visitedLimit)
+      RandomVectorScorer scorer, int topK, OnHeapHnswGraph graph, Bits acceptOrds, int visitedLimit)

Review Comment:
   This function is only used by the `Word2VecSynonymProvider` to search the on-heap builder/searcher with multiple threads.
   I think we can simplify things a bit to not require to have an `OnHeapHnswGraph` for this purpose. It's outside the scope of this PR but the main reason this function exists is because the `HnswGraph` API exposes `seek` and `nextNeighbor` as separate function so the implementer needs to keep a state in the object. We could get rid of this by changing the API in a follow up and only expose a `PrimitiveIterator.OfInt getNeighbors()`. In the latest codec we also eagerly load the entire list on each `seek` call (for the off heap version) so we don't really need the iterator API either. Maybe exposing it as a simple `int[] getNeighbors()` could be enough.



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

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

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


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


[GitHub] [lucene] jimczi commented on pull request #12529: Introduce a random vector scorer in HNSW builder/searcher

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

   Thanks for running the benchmarks @benwtrent . I agree that the difference seems to be in the noise.


-- 
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 #12529: Introduce a random vector scorer in HNSW builder/searcher

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

   I reran from latest main and the most recent merge here:
   
   ```
   recall	latency	nDoc	fanout	maxConn	beamWidth	visited	index ms
   lucene_baseline
   0.980	 0.68	50000	100	32	100	110	16655	1.00	post-filter
   lucene_candidate
   0.980	 0.68	50000	100	32	100	110	16644	1.00	post-filter
   lucene_baseline
   0.991	 0.93	50000	100	32	250	110	52158	1.00	post-filter
   lucene_candidate
   0.991	 0.93	50000	100	32	250	110	52342	1.00	post-filter
   lucene_baseline
   0.979	 0.69	50000	100	96	100	110	16652	1.00	post-filter
   lucene_candidate
   0.979	 0.71	50000	100	96	100	110	16854	1.00	post-filter
   lucene_baseline
   0.992	 1.02	50000	100	96	250	110	54944	1.00	post-filter
   lucene_candidate
   0.992	 1.02	50000	100	96	250	110	55163	1.00	post-filter
   ```
   
   There is basically no query time difference between this PR and main.
   
   However, building the index for 50000 768 float32 vectors is around 2% slower on larger beamWidth & maxConn.
   
   It is either caused by the number of objects we are creating, or the random vector reader isn't being copied correctly and thus something funky is happening when comparing.
   
   I am not sure. 2% isn't that big of a hit IMO for a much cleaner API. 


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

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

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


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


[GitHub] [lucene] jimczi commented on a diff in pull request #12529: Introduce a random vector scorer in HNSW builder/searcher

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


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/OffHeapByteVectorValues.java:
##########
@@ -60,13 +61,17 @@ public int size() {
 
   @Override
   public byte[] vectorValue(int targetOrd) throws IOException {
-    readValue(targetOrd);
+    if (lastOrd != targetOrd) {
+      readValue(targetOrd);
+      lastOrd = targetOrd;
+    }
     return binaryValue;
   }
 
   private void readValue(int targetOrd) throws IOException {
     slice.seek((long) targetOrd * byteSize);
     slice.readBytes(byteBuffer.array(), byteBuffer.arrayOffset(), byteSize);
+    lastOrd = targetOrd;

Review Comment:
   thanks, addressed in https://github.com/apache/lucene/pull/12529/commits/31d8b549e62a288358c58cd666767d09bf17f65c



-- 
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 #12529: Introduce a random vector scorer in HNSW builder/searcher

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


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/RandomVectorScorer.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.util.hnsw;
+
+import java.io.IOException;
+import org.apache.lucene.index.VectorSimilarityFunction;
+import org.apache.lucene.search.Query;
+
+/**
+ * The {@link RandomVectorScorer} calculates scores between an abstract query and a random node. It
+ * can also score two random nodes stored in the graph. The scores returned must be strictly
+ * positive for them to be usable in a {@link Query}.
+ */
+public interface RandomVectorScorer {
+  /**
+   * Returns the score between the query and the provided node.
+   *
+   * @param node a random node in the graph
+   * @return the computed score
+   */
+  float queryScore(int node) throws IOException;
+
+  /**
+   * Returns the score between two nodes.
+   *
+   * @param node1 the first node
+   * @param node2 the second node
+   * @return the computed score
+   */
+  float symmetricScore(int node1, int node2) throws IOException;
+
+  /**
+   * Creates a default scorer for float vectors.
+   *
+   * @param vectors the underlying storage for vectors
+   * @param similarityFunction the similarity function to score vectors
+   * @param query the actual query
+   */
+  static RandomVectorScorer createFloats(
+      final RandomAccessVectorValues<float[]> vectors,
+      final VectorSimilarityFunction similarityFunction,
+      final float[] query)
+      throws IOException {
+    if (query.length != vectors.dimension()) {
+      throw new IllegalArgumentException(
+          "vector query dimension: "
+              + query.length
+              + " differs from field dimension: "
+              + vectors.dimension());
+    }
+    final RandomAccessVectorValues<float[]> vectorsCopy = vectors.copy();
+    return new RandomVectorScorer() {
+      @Override
+      public float symmetricScore(int node1, int node2) throws IOException {
+        return similarityFunction.compare(
+            vectors.vectorValue(node1), vectorsCopy.vectorValue(node2));
+      }
+
+      @Override
+      public float queryScore(int node) throws IOException {
+        return similarityFunction.compare(query, vectorsCopy.vectorValue(node));
+      }
+    };
+  }
+
+  /**
+   * Creates a default scorer for byte vectors.
+   *
+   * @param vectors the underlying storage for vectors
+   * @param similarityFunction the similarity function to use to score vectors
+   * @param query the actual query
+   */
+  static RandomVectorScorer createBytes(
+      final RandomAccessVectorValues<byte[]> vectors,
+      final VectorSimilarityFunction similarityFunction,
+      final byte[] query)

Review Comment:
   This comment also applies to the `createFloats` method.



-- 
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 #12529: Introduce a random vector scorer in HNSW builder/searcher

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


##########
lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene94/Lucene94HnswVectorsWriter.java:
##########
@@ -630,7 +621,8 @@ private abstract static class FieldWriter<T> extends KnnFieldVectorsWriter<T> {
     private final int dim;
     private final DocsWithFieldSet docsWithField;
     private final List<T> vectors;
-    private final HnswGraphBuilder<T> hnswGraphBuilder;
+    private final RandomAccessVectorValues<T> raVectors;

Review Comment:
   This could be a local variable. It isn't used in anything other than the ctor



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/Lucene95HnswVectorsWriter.java:
##########
@@ -868,7 +859,8 @@ private abstract static class FieldWriter<T> extends KnnFieldVectorsWriter<T> {
     private final int dim;
     private final DocsWithFieldSet docsWithField;
     private final List<T> vectors;
-    private final HnswGraphBuilder<T> hnswGraphBuilder;
+    private final RAVectorValues<T> raVectors;

Review Comment:
   Can be a local variable. Its only used in the ctor.



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/Lucene95HnswVectorsWriter.java:
##########
@@ -482,35 +478,30 @@ public void mergeOneField(FieldInfo fieldInfo, MergeState mergeState) throws IOE
     }
   }
 
-  private <T> HnswGraphBuilder<T> createHnswGraphBuilder(
+  @SuppressWarnings("unchecked")
+  private HnswGraphBuilder createHnswGraphBuilder(
       MergeState mergeState,
       FieldInfo fieldInfo,
-      RandomAccessVectorValues<T> floatVectorValues,
+      RandomAccessVectorValues<?> vectors,
       int initializerIndex)
       throws IOException {
+    RandomVectorScorerProvider scorerProvider =

Review Comment:
   We should move this up into `mergeOneField`. We already switch on the encoding and gather things of the correct type. 
   
   Then we can pass the constructed `RandomVectorScorerProvider scorerProvider` to this private method instead of the `RandomAccessVectorValues<?> vectors,`



##########
lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphSearcher.java:
##########
@@ -172,106 +83,36 @@ public static KnnCollector search(
    * @return a set of collected vectors holding the nearest neighbors found
    */
   public static KnnCollector search(
-      byte[] query,
-      int topK,
-      RandomAccessVectorValues<byte[]> vectors,
-      VectorEncoding vectorEncoding,
-      VectorSimilarityFunction similarityFunction,
-      HnswGraph graph,
-      Bits acceptOrds,
-      int visitedLimit)
-      throws IOException {
-    KnnCollector collector = new TopKnnCollector(topK, visitedLimit);
-    search(query, collector, vectors, vectorEncoding, similarityFunction, graph, acceptOrds);
-    return collector;
-  }
-
-  /**
-   * Searches HNSW graph for the nearest neighbors of a query vector.
-   *
-   * @param query search query vector
-   * @param knnCollector a collector of top knn results to be returned
-   * @param vectors the vector values
-   * @param similarityFunction the similarity function to compare vectors
-   * @param graph the graph values. May represent the entire graph, or a level in a hierarchical
-   *     graph.
-   * @param acceptOrds {@link Bits} that represents the allowed document ordinals to match, or
-   *     {@code null} if they are all allowed to match.
-   */
-  public static void search(
-      byte[] query,
-      KnnCollector knnCollector,
-      RandomAccessVectorValues<byte[]> vectors,
-      VectorEncoding vectorEncoding,
-      VectorSimilarityFunction similarityFunction,
-      HnswGraph graph,
-      Bits acceptOrds)
-      throws IOException {
-    if (query.length != vectors.dimension()) {
-      throw new IllegalArgumentException(
-          "vector query dimension: "
-              + query.length
-              + " differs from field dimension: "
-              + vectors.dimension());
-    }
-    HnswGraphSearcher<byte[]> graphSearcher =
-        new HnswGraphSearcher<>(
-            vectorEncoding,
-            similarityFunction,
-            new NeighborQueue(knnCollector.k(), true),
-            new SparseFixedBitSet(vectors.size()));
-    search(query, knnCollector, vectors, graph, graphSearcher, acceptOrds);
-  }
-
-  /**
-   * Search {@link OnHeapHnswGraph}, this method is thread safe, for parameters please refer to
-   * {@link #search(byte[], int, RandomAccessVectorValues, VectorEncoding, VectorSimilarityFunction,
-   * HnswGraph, Bits, int)}
-   */
-  public static KnnCollector search(
-      byte[] query,
-      int topK,
-      RandomAccessVectorValues<byte[]> vectors,
-      VectorEncoding vectorEncoding,
-      VectorSimilarityFunction similarityFunction,
-      OnHeapHnswGraph graph,
-      Bits acceptOrds,
-      int visitedLimit)
+      RandomVectorScorer scorer, int topK, OnHeapHnswGraph graph, Bits acceptOrds, int visitedLimit)

Review Comment:
   I think this breaks with the new change :(
   
   Here we create a `new TopKnnCollector(topK, visitedLimit);`, but the underlying vector storage could be sparse requiring `OrdinalTranslatedKnnCollector`. We need to ensure that users that call for `topK` directly like this correctly have their ordinals applied. 
   
   I am not 100% sure the best way to ensure this.
   
   We could provide (again, sorry for suggesting otherwise) ordinal translation in the `RandomVectorScorer`
   
   Or we remove this and require callers to provide the collector directly. We are already breaking the API by requiring a scorer now. So, it seems like requiring that they provide a collector isn't any more breaking.



-- 
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 #12529: Introduce a random vector scorer in HNSW builder/searcher

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


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java:
##########
@@ -165,11 +134,7 @@ private HnswGraphBuilder(
    * @param vectorsToAdd the vectors for which to build a nearest neighbors graph. Must be an
    *     independent accessor for the vectors
    */
-  public OnHeapHnswGraph build(RandomAccessVectorValues<T> vectorsToAdd) throws IOException {
-    if (vectorsToAdd == this.vectors) {
-      throw new IllegalArgumentException(
-          "Vectors to build must be independent of the source of vectors provided to HnswGraphBuilder()");
-    }
+  public OnHeapHnswGraph build(RandomAccessVectorValues<?> vectorsToAdd) throws IOException {

Review Comment:
   So, this actually doesn't use anything related to `RandomAccessVectorValues` other than checking the size. Is there a better interface here than passing in `RandomAccessVectorValues<?> vectorsToAdd`? Maybe `public OnHeapHnswGraph build(int maxOrd) throws IOException {` ?



##########
lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java:
##########
@@ -366,38 +324,11 @@ private void popToScratch(GraphBuilderKnnCollector candidates) {
    * @param neighbors the neighbors selected so far
    * @return whether the candidate is diverse given the existing neighbors
    */
-  private boolean diversityCheck(int candidate, float score, NeighborArray neighbors)
-      throws IOException {
-    return isDiverse(candidate, neighbors, score);
-  }
-
-  private boolean isDiverse(int candidate, NeighborArray neighbors, float score)
-      throws IOException {
-    return switch (vectorEncoding) {
-      case BYTE -> isDiverse((byte[]) vectors.vectorValue(candidate), neighbors, score);
-      case FLOAT32 -> isDiverse((float[]) vectors.vectorValue(candidate), neighbors, score);
-    };
-  }
-
-  private boolean isDiverse(float[] candidate, NeighborArray neighbors, float score)
-      throws IOException {
-    for (int i = 0; i < neighbors.size(); i++) {
-      float neighborSimilarity =
-          similarityFunction.compare(
-              candidate, (float[]) vectorsCopy.vectorValue(neighbors.node[i]));
-      if (neighborSimilarity >= score) {
-        return false;
-      }
-    }
-    return true;
-  }
-
-  private boolean isDiverse(byte[] candidate, NeighborArray neighbors, float score)
+  private boolean diversityCheck(
+      RandomVectorScorer scorer, int candidate, float score, NeighborArray neighbors)
       throws IOException {
     for (int i = 0; i < neighbors.size(); i++) {
-      float neighborSimilarity =
-          similarityFunction.compare(
-              candidate, (byte[]) vectorsCopy.vectorValue(neighbors.node[i]));
+      float neighborSimilarity = scorer.symmetricScore(candidate, neighbors.node[i]);

Review Comment:
   My thought here is that `scorerProvider` creates a new scorer for `candidate` and iterates the neighbors. 
   
   But, I wonder how many short lived objects we will be creating doing that.



-- 
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 #12529: Introduce a random vector scorer in HNSW builder/searcher

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

   @msokolov what say you? It seems like encapsulating random vector seeking & scoring into one thing makes the code simpler.


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

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

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


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


[GitHub] [lucene] jimczi commented on a diff in pull request #12529: Introduce a random vector scorer in HNSW builder/searcher

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


##########
lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswVectorsReader.java:
##########
@@ -423,8 +422,12 @@ public RandomAccessVectorValues<float[]> copy() {
 
     @Override
     public float[] vectorValue(int targetOrd) throws IOException {
+      if (lastOrd == targetOrd) {
+        return value;
+      }

Review Comment:
   Not on normal search where only the query vector is used multiple times but it happens frequently when we build the index during the diversification phase. There, we compare single ordinals with all their neighbors sequentially so instead of having the optimisation in the graph I moved it to the reader since it's straightforward to add and shouldn't add any overhead to check.



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

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

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


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


[GitHub] [lucene] jimczi commented on pull request #12529: Introduce a random vector scorer in HNSW builder/searcher

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

   Given that no further concerns have been raised, I am intending to merge this change soon.


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