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 2021/03/30 15:41:01 UTC

[GitHub] [lucene] nitirajrathore opened a new pull request #55: LUCENE-9798 : Fix looping bug and made Full Knn calculation parallelizable

nitirajrathore opened a new pull request #55:
URL: https://github.com/apache/lucene/pull/55


   
   # Description
   There was a bug in the KNN Tester where in for large document vector files only first set of documents will be considered for Full Knn calculation
   
   # Solution
   I have corrected the bug and also made the code to execute parallelly, so as to take less time for large document vector files. 
   
   # Tests
   I am yet to figure out how to properly Mock the FileChannel. May be I will write a dummy class myself? 
   
   # Checklist
   TODO: Tests.
   


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

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 a change in pull request #55: LUCENE-9798 : Fix looping bug and made Full Knn calculation parallelizable

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #55:
URL: https://github.com/apache/lucene/pull/55#discussion_r604386537



##########
File path: lucene/test-framework/src/java/org/apache/lucene/util/FullKnn.java
##########
@@ -0,0 +1,156 @@
+package org.apache.lucene.util;
+
+import org.apache.lucene.index.VectorValues;
+import org.apache.lucene.util.hnsw.NeighborQueue;
+
+import java.io.IOException;
+import java.nio.ByteOrder;
+import java.nio.FloatBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+/**
+ * A utility class to calculate the Full KNN / Exact KNN over a set of query vectors and document vectors.
+ */
+public class FullKnn {
+
+  private final int dim;
+  private final int topK;
+  private final VectorValues.SearchStrategy searchStrategy;
+  private final boolean quiet;
+
+  public FullKnn(int dim, int topK, VectorValues.SearchStrategy searchStrategy, boolean quiet) {
+    this.dim = dim;
+    this.topK = topK;
+    this.searchStrategy = searchStrategy;
+    this.quiet = quiet;
+  }
+
+  /**
+   * internal object to track KNN calculation for one query
+   */
+  private static class KnnJob {
+    public int currDocIndex;
+    float[] queryVector;
+    float[] currDocVector;
+    int queryIndex;
+    NeighborQueue queue;
+    FloatBuffer docVectors;
+    VectorValues.SearchStrategy searchStrategy;
+
+    public KnnJob(int queryIndex, float[] queryVector, int topK, VectorValues.SearchStrategy searchStrategy) {
+      this.queryIndex = queryIndex;
+      this.queryVector = queryVector;
+      this.currDocVector = new float[queryVector.length];
+      queue = new NeighborQueue(topK, searchStrategy.reversed);
+      this.searchStrategy = searchStrategy;
+    }
+
+    public void execute() {
+      while (this.docVectors.hasRemaining()) {
+        this.docVectors.get(this.currDocVector);
+        float d = this.searchStrategy.compare(this.queryVector, this.currDocVector);
+        this.queue.insertWithOverflow(this.currDocIndex, d);
+        this.currDocIndex++;
+      }
+    }
+  }
+
+  /**
+   * computes the exact KNN match for each query vector in queryPath for all the document vectors in docPath
+   *
+   * @param docPath   : path to the file containing the float 32 document vectors in bytes with little-endian byte order
+   *                  Throws exception if topK is greater than number of documents in this file

Review comment:
       what kind of exception does it throw?

##########
File path: lucene/test-framework/src/java/org/apache/lucene/util/FullKnn.java
##########
@@ -0,0 +1,156 @@
+package org.apache.lucene.util;

Review comment:
       Does this belong in o.a.l.util.hnsw?

##########
File path: lucene/test-framework/src/java/org/apache/lucene/util/FullKnn.java
##########
@@ -0,0 +1,156 @@
+package org.apache.lucene.util;
+
+import org.apache.lucene.index.VectorValues;
+import org.apache.lucene.util.hnsw.NeighborQueue;
+
+import java.io.IOException;
+import java.nio.ByteOrder;
+import java.nio.FloatBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+/**
+ * A utility class to calculate the Full KNN / Exact KNN over a set of query vectors and document vectors.
+ */
+public class FullKnn {
+
+  private final int dim;
+  private final int topK;
+  private final VectorValues.SearchStrategy searchStrategy;
+  private final boolean quiet;
+
+  public FullKnn(int dim, int topK, VectorValues.SearchStrategy searchStrategy, boolean quiet) {
+    this.dim = dim;
+    this.topK = topK;
+    this.searchStrategy = searchStrategy;
+    this.quiet = quiet;
+  }
+
+  /**
+   * internal object to track KNN calculation for one query
+   */
+  private static class KnnJob {
+    public int currDocIndex;
+    float[] queryVector;
+    float[] currDocVector;
+    int queryIndex;
+    NeighborQueue queue;
+    FloatBuffer docVectors;
+    VectorValues.SearchStrategy searchStrategy;
+
+    public KnnJob(int queryIndex, float[] queryVector, int topK, VectorValues.SearchStrategy searchStrategy) {
+      this.queryIndex = queryIndex;
+      this.queryVector = queryVector;
+      this.currDocVector = new float[queryVector.length];
+      queue = new NeighborQueue(topK, searchStrategy.reversed);
+      this.searchStrategy = searchStrategy;
+    }
+
+    public void execute() {
+      while (this.docVectors.hasRemaining()) {
+        this.docVectors.get(this.currDocVector);
+        float d = this.searchStrategy.compare(this.queryVector, this.currDocVector);
+        this.queue.insertWithOverflow(this.currDocIndex, d);
+        this.currDocIndex++;
+      }
+    }
+  }
+
+  /**
+   * computes the exact KNN match for each query vector in queryPath for all the document vectors in docPath
+   *
+   * @param docPath   : path to the file containing the float 32 document vectors in bytes with little-endian byte order
+   *                  Throws exception if topK is greater than number of documents in this file
+   * @param numDocs   : number of vectors in the document vector file at docPath
+   * @param queryPath : path to the file containing the containing 32-bit floating point vectors in little-endian byte order
+   * @param numIters  : number of vectors in the query vector file at queryPath

Review comment:
       similar comment for numDocs above. Also let's rename to numQueries

##########
File path: lucene/core/src/java/org/apache/lucene/util/hnsw/NeighborQueue.java
##########
@@ -34,15 +34,15 @@
   // Used to track the number of neighbors visited during a single graph traversal
   private int visitedCount;
 
-  NeighborQueue(int initialSize, boolean reversed) {
+  public NeighborQueue(int initialSize, boolean reversed) {

Review comment:
       I think we can avoid making these public if we keep all these classes in the same package?

##########
File path: lucene/test-framework/src/java/org/apache/lucene/util/FullKnn.java
##########
@@ -0,0 +1,156 @@
+package org.apache.lucene.util;
+
+import org.apache.lucene.index.VectorValues;
+import org.apache.lucene.util.hnsw.NeighborQueue;
+
+import java.io.IOException;
+import java.nio.ByteOrder;
+import java.nio.FloatBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+/**
+ * A utility class to calculate the Full KNN / Exact KNN over a set of query vectors and document vectors.
+ */
+public class FullKnn {
+
+  private final int dim;
+  private final int topK;
+  private final VectorValues.SearchStrategy searchStrategy;
+  private final boolean quiet;
+
+  public FullKnn(int dim, int topK, VectorValues.SearchStrategy searchStrategy, boolean quiet) {
+    this.dim = dim;
+    this.topK = topK;
+    this.searchStrategy = searchStrategy;
+    this.quiet = quiet;
+  }
+
+  /**
+   * internal object to track KNN calculation for one query
+   */
+  private static class KnnJob {
+    public int currDocIndex;
+    float[] queryVector;
+    float[] currDocVector;
+    int queryIndex;
+    NeighborQueue queue;
+    FloatBuffer docVectors;
+    VectorValues.SearchStrategy searchStrategy;
+
+    public KnnJob(int queryIndex, float[] queryVector, int topK, VectorValues.SearchStrategy searchStrategy) {
+      this.queryIndex = queryIndex;
+      this.queryVector = queryVector;
+      this.currDocVector = new float[queryVector.length];
+      queue = new NeighborQueue(topK, searchStrategy.reversed);
+      this.searchStrategy = searchStrategy;
+    }
+
+    public void execute() {
+      while (this.docVectors.hasRemaining()) {
+        this.docVectors.get(this.currDocVector);
+        float d = this.searchStrategy.compare(this.queryVector, this.currDocVector);
+        this.queue.insertWithOverflow(this.currDocIndex, d);
+        this.currDocIndex++;
+      }
+    }
+  }
+
+  /**
+   * computes the exact KNN match for each query vector in queryPath for all the document vectors in docPath
+   *
+   * @param docPath   : path to the file containing the float 32 document vectors in bytes with little-endian byte order
+   *                  Throws exception if topK is greater than number of documents in this file
+   * @param numDocs   : number of vectors in the document vector file at docPath
+   * @param queryPath : path to the file containing the containing 32-bit floating point vectors in little-endian byte order
+   * @param numIters  : number of vectors in the query vector file at queryPath
+   * @param numThreads : create numThreads to parallelize work
+   * @return : returns an int 2D array ( int matches[][]) of size 'numIters x topK'. matches[i] is an array containing
+   * the indexes of the topK most similar document vectors to the ith query vector, and is sorted by similarity, with
+   * the most similar vector first. Similarity is defined by the searchStrategy used to construct this FullKnn.
+   * @throws IOException : if topK is greater than number of documents in docPath file

Review comment:
       is it topK, or numDocs? What if numQueries exceeds the query file size?

##########
File path: lucene/test-framework/src/java/org/apache/lucene/util/FullKnn.java
##########
@@ -0,0 +1,156 @@
+package org.apache.lucene.util;
+
+import org.apache.lucene.index.VectorValues;
+import org.apache.lucene.util.hnsw.NeighborQueue;
+
+import java.io.IOException;
+import java.nio.ByteOrder;
+import java.nio.FloatBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+/**
+ * A utility class to calculate the Full KNN / Exact KNN over a set of query vectors and document vectors.
+ */
+public class FullKnn {
+
+  private final int dim;
+  private final int topK;
+  private final VectorValues.SearchStrategy searchStrategy;
+  private final boolean quiet;
+
+  public FullKnn(int dim, int topK, VectorValues.SearchStrategy searchStrategy, boolean quiet) {
+    this.dim = dim;
+    this.topK = topK;
+    this.searchStrategy = searchStrategy;
+    this.quiet = quiet;
+  }
+
+  /**
+   * internal object to track KNN calculation for one query
+   */
+  private static class KnnJob {
+    public int currDocIndex;
+    float[] queryVector;
+    float[] currDocVector;
+    int queryIndex;
+    NeighborQueue queue;
+    FloatBuffer docVectors;
+    VectorValues.SearchStrategy searchStrategy;
+
+    public KnnJob(int queryIndex, float[] queryVector, int topK, VectorValues.SearchStrategy searchStrategy) {
+      this.queryIndex = queryIndex;
+      this.queryVector = queryVector;
+      this.currDocVector = new float[queryVector.length];
+      queue = new NeighborQueue(topK, searchStrategy.reversed);
+      this.searchStrategy = searchStrategy;
+    }
+
+    public void execute() {
+      while (this.docVectors.hasRemaining()) {
+        this.docVectors.get(this.currDocVector);
+        float d = this.searchStrategy.compare(this.queryVector, this.currDocVector);
+        this.queue.insertWithOverflow(this.currDocIndex, d);
+        this.currDocIndex++;
+      }
+    }
+  }
+
+  /**
+   * computes the exact KNN match for each query vector in queryPath for all the document vectors in docPath
+   *
+   * @param docPath   : path to the file containing the float 32 document vectors in bytes with little-endian byte order
+   *                  Throws exception if topK is greater than number of documents in this file
+   * @param numDocs   : number of vectors in the document vector file at docPath

Review comment:
       We could compute the number of vectors in the file by looking at its size. Does this allow computing over a subset though? Maybe we should rephrase it as "number of vectors in the document vector file for which to compute matches"




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

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] nitirajrathore commented on a change in pull request #55: LUCENE-9798 : Fix looping bug and made Full Knn calculation parallelizable

Posted by GitBox <gi...@apache.org>.
nitirajrathore commented on a change in pull request #55:
URL: https://github.com/apache/lucene/pull/55#discussion_r604604635



##########
File path: lucene/core/src/java/org/apache/lucene/util/hnsw/NeighborQueue.java
##########
@@ -34,15 +34,15 @@
   // Used to track the number of neighbors visited during a single graph traversal
   private int visitedCount;
 
-  NeighborQueue(int initialSize, boolean reversed) {
+  public NeighborQueue(int initialSize, boolean reversed) {

Review comment:
       Yes, Initially I created FullKnn.java in same package called `hnsw`. But it does not make sense to have different algorithm in the package name and different algorithm (exact knn) in class. I think we don't need the NeighborQueue at all here, it can directly be implemented with LongHeap, I will attempt 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.

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] nitirajrathore commented on a change in pull request #55: LUCENE-9798 : Fix looping bug and made Full Knn calculation parallelizable

Posted by GitBox <gi...@apache.org>.
nitirajrathore commented on a change in pull request #55:
URL: https://github.com/apache/lucene/pull/55#discussion_r607813173



##########
File path: lucene/test-framework/src/java/org/apache/lucene/util/FullKnn.java
##########
@@ -0,0 +1,156 @@
+package org.apache.lucene.util;
+
+import org.apache.lucene.index.VectorValues;
+import org.apache.lucene.util.hnsw.NeighborQueue;
+
+import java.io.IOException;
+import java.nio.ByteOrder;
+import java.nio.FloatBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+/**
+ * A utility class to calculate the Full KNN / Exact KNN over a set of query vectors and document vectors.
+ */
+public class FullKnn {
+
+  private final int dim;
+  private final int topK;
+  private final VectorValues.SearchStrategy searchStrategy;
+  private final boolean quiet;
+
+  public FullKnn(int dim, int topK, VectorValues.SearchStrategy searchStrategy, boolean quiet) {
+    this.dim = dim;
+    this.topK = topK;
+    this.searchStrategy = searchStrategy;
+    this.quiet = quiet;
+  }
+
+  /**
+   * internal object to track KNN calculation for one query
+   */
+  private static class KnnJob {
+    public int currDocIndex;
+    float[] queryVector;
+    float[] currDocVector;
+    int queryIndex;
+    NeighborQueue queue;
+    FloatBuffer docVectors;
+    VectorValues.SearchStrategy searchStrategy;
+
+    public KnnJob(int queryIndex, float[] queryVector, int topK, VectorValues.SearchStrategy searchStrategy) {
+      this.queryIndex = queryIndex;
+      this.queryVector = queryVector;
+      this.currDocVector = new float[queryVector.length];
+      queue = new NeighborQueue(topK, searchStrategy.reversed);
+      this.searchStrategy = searchStrategy;
+    }
+
+    public void execute() {
+      while (this.docVectors.hasRemaining()) {
+        this.docVectors.get(this.currDocVector);
+        float d = this.searchStrategy.compare(this.queryVector, this.currDocVector);
+        this.queue.insertWithOverflow(this.currDocIndex, d);
+        this.currDocIndex++;
+      }
+    }
+  }
+
+  /**
+   * computes the exact KNN match for each query vector in queryPath for all the document vectors in docPath
+   *
+   * @param docPath   : path to the file containing the float 32 document vectors in bytes with little-endian byte order
+   *                  Throws exception if topK is greater than number of documents in this file
+   * @param numDocs   : number of vectors in the document vector file at docPath

Review comment:
       done!




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

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 #55: LUCENE-9798 : Fix looping bug and made Full Knn calculation parallelizable

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


   Thanks for the patch! Looks like spotless formatting check failed; if you run `./gradlew spotlessApply` it should automatically reformat the patch in accordance with our formatting standard


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

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 a change in pull request #55: LUCENE-9798 : Fix looping bug and made Full Knn calculation parallelizable

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #55:
URL: https://github.com/apache/lucene/pull/55#discussion_r605192900



##########
File path: lucene/test-framework/src/java/org/apache/lucene/util/FullKnn.java
##########
@@ -0,0 +1,156 @@
+package org.apache.lucene.util;
+
+import org.apache.lucene.index.VectorValues;
+import org.apache.lucene.util.hnsw.NeighborQueue;
+
+import java.io.IOException;
+import java.nio.ByteOrder;
+import java.nio.FloatBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+/**
+ * A utility class to calculate the Full KNN / Exact KNN over a set of query vectors and document vectors.
+ */
+public class FullKnn {
+
+  private final int dim;
+  private final int topK;
+  private final VectorValues.SearchStrategy searchStrategy;
+  private final boolean quiet;
+
+  public FullKnn(int dim, int topK, VectorValues.SearchStrategy searchStrategy, boolean quiet) {
+    this.dim = dim;
+    this.topK = topK;
+    this.searchStrategy = searchStrategy;
+    this.quiet = quiet;
+  }
+
+  /**
+   * internal object to track KNN calculation for one query
+   */
+  private static class KnnJob {
+    public int currDocIndex;
+    float[] queryVector;
+    float[] currDocVector;
+    int queryIndex;
+    NeighborQueue queue;
+    FloatBuffer docVectors;
+    VectorValues.SearchStrategy searchStrategy;
+
+    public KnnJob(int queryIndex, float[] queryVector, int topK, VectorValues.SearchStrategy searchStrategy) {
+      this.queryIndex = queryIndex;
+      this.queryVector = queryVector;
+      this.currDocVector = new float[queryVector.length];
+      queue = new NeighborQueue(topK, searchStrategy.reversed);
+      this.searchStrategy = searchStrategy;
+    }
+
+    public void execute() {
+      while (this.docVectors.hasRemaining()) {
+        this.docVectors.get(this.currDocVector);
+        float d = this.searchStrategy.compare(this.queryVector, this.currDocVector);
+        this.queue.insertWithOverflow(this.currDocIndex, d);
+        this.currDocIndex++;
+      }
+    }
+  }
+
+  /**
+   * computes the exact KNN match for each query vector in queryPath for all the document vectors in docPath
+   *
+   * @param docPath   : path to the file containing the float 32 document vectors in bytes with little-endian byte order
+   *                  Throws exception if topK is greater than number of documents in this file
+   * @param numDocs   : number of vectors in the document vector file at docPath

Review comment:
       Hm I think Java's nio API will always tell you the number of bytes in a file as its size, whatever the filesystem actually consumes on disk due to page quantization.




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

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] nitirajrathore commented on a change in pull request #55: LUCENE-9798 : Fix looping bug and made Full Knn calculation parallelizable

Posted by GitBox <gi...@apache.org>.
nitirajrathore commented on a change in pull request #55:
URL: https://github.com/apache/lucene/pull/55#discussion_r604606544



##########
File path: lucene/test-framework/src/java/org/apache/lucene/util/FullKnn.java
##########
@@ -0,0 +1,156 @@
+package org.apache.lucene.util;
+
+import org.apache.lucene.index.VectorValues;
+import org.apache.lucene.util.hnsw.NeighborQueue;
+
+import java.io.IOException;
+import java.nio.ByteOrder;
+import java.nio.FloatBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+/**
+ * A utility class to calculate the Full KNN / Exact KNN over a set of query vectors and document vectors.
+ */
+public class FullKnn {
+
+  private final int dim;
+  private final int topK;
+  private final VectorValues.SearchStrategy searchStrategy;
+  private final boolean quiet;
+
+  public FullKnn(int dim, int topK, VectorValues.SearchStrategy searchStrategy, boolean quiet) {
+    this.dim = dim;
+    this.topK = topK;
+    this.searchStrategy = searchStrategy;
+    this.quiet = quiet;
+  }
+
+  /**
+   * internal object to track KNN calculation for one query
+   */
+  private static class KnnJob {
+    public int currDocIndex;
+    float[] queryVector;
+    float[] currDocVector;
+    int queryIndex;
+    NeighborQueue queue;
+    FloatBuffer docVectors;
+    VectorValues.SearchStrategy searchStrategy;
+
+    public KnnJob(int queryIndex, float[] queryVector, int topK, VectorValues.SearchStrategy searchStrategy) {
+      this.queryIndex = queryIndex;
+      this.queryVector = queryVector;
+      this.currDocVector = new float[queryVector.length];
+      queue = new NeighborQueue(topK, searchStrategy.reversed);
+      this.searchStrategy = searchStrategy;
+    }
+
+    public void execute() {
+      while (this.docVectors.hasRemaining()) {
+        this.docVectors.get(this.currDocVector);
+        float d = this.searchStrategy.compare(this.queryVector, this.currDocVector);
+        this.queue.insertWithOverflow(this.currDocIndex, d);
+        this.currDocIndex++;
+      }
+    }
+  }
+
+  /**
+   * computes the exact KNN match for each query vector in queryPath for all the document vectors in docPath
+   *
+   * @param docPath   : path to the file containing the float 32 document vectors in bytes with little-endian byte order
+   *                  Throws exception if topK is greater than number of documents in this file
+   * @param numDocs   : number of vectors in the document vector file at docPath

Review comment:
       I am not sure if file size always corresponds exactly to the size of its contents. For example file size may accomodate for padding / page size of OS. Will confirm this.
   If we have to make it subset, then we have to take start index and num of vectors rather than just num docs. I think for this utility we can skip that trouble. Caller can always create a smaller file and use that instead.




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

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] nitirajrathore commented on a change in pull request #55: LUCENE-9798 : Fix looping bug and made Full Knn calculation parallelizable

Posted by GitBox <gi...@apache.org>.
nitirajrathore commented on a change in pull request #55:
URL: https://github.com/apache/lucene/pull/55#discussion_r607812871



##########
File path: lucene/test-framework/src/java/org/apache/lucene/util/FullKnn.java
##########
@@ -0,0 +1,156 @@
+package org.apache.lucene.util;
+
+import org.apache.lucene.index.VectorValues;
+import org.apache.lucene.util.hnsw.NeighborQueue;
+
+import java.io.IOException;
+import java.nio.ByteOrder;
+import java.nio.FloatBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+/**
+ * A utility class to calculate the Full KNN / Exact KNN over a set of query vectors and document vectors.
+ */
+public class FullKnn {
+
+  private final int dim;
+  private final int topK;
+  private final VectorValues.SearchStrategy searchStrategy;
+  private final boolean quiet;
+
+  public FullKnn(int dim, int topK, VectorValues.SearchStrategy searchStrategy, boolean quiet) {
+    this.dim = dim;
+    this.topK = topK;
+    this.searchStrategy = searchStrategy;
+    this.quiet = quiet;
+  }
+
+  /**
+   * internal object to track KNN calculation for one query
+   */
+  private static class KnnJob {
+    public int currDocIndex;
+    float[] queryVector;
+    float[] currDocVector;
+    int queryIndex;
+    NeighborQueue queue;
+    FloatBuffer docVectors;
+    VectorValues.SearchStrategy searchStrategy;
+
+    public KnnJob(int queryIndex, float[] queryVector, int topK, VectorValues.SearchStrategy searchStrategy) {
+      this.queryIndex = queryIndex;
+      this.queryVector = queryVector;
+      this.currDocVector = new float[queryVector.length];
+      queue = new NeighborQueue(topK, searchStrategy.reversed);
+      this.searchStrategy = searchStrategy;
+    }
+
+    public void execute() {
+      while (this.docVectors.hasRemaining()) {
+        this.docVectors.get(this.currDocVector);
+        float d = this.searchStrategy.compare(this.queryVector, this.currDocVector);
+        this.queue.insertWithOverflow(this.currDocIndex, d);
+        this.currDocIndex++;
+      }
+    }
+  }
+
+  /**
+   * computes the exact KNN match for each query vector in queryPath for all the document vectors in docPath
+   *
+   * @param docPath   : path to the file containing the float 32 document vectors in bytes with little-endian byte order
+   *                  Throws exception if topK is greater than number of documents in this file

Review comment:
       fixed.




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

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 a change in pull request #55: LUCENE-9798 : Fix looping bug and made Full Knn calculation parallelizable

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #55:
URL: https://github.com/apache/lucene/pull/55#discussion_r605194409



##########
File path: lucene/test-framework/src/java/org/apache/lucene/util/FullKnn.java
##########
@@ -0,0 +1,156 @@
+package org.apache.lucene.util;

Review comment:
       I see, yes that makes sense - this doesn't have anything to do with HNSW, but provides ground truth for evaluating any approximate vector search. I think that using `LongHeap` will require copying the encode/decode logic from `NeighborQuery`, but that's OK? Then we can leave that as a class dedicated to support HNSW...




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

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 #55: LUCENE-9798 : Fix looping bug and made Full Knn calculation parallelizable

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


   Thanks for addressing the issues with this patch, looks good now I'll merge.


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

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] nitirajrathore commented on a change in pull request #55: LUCENE-9798 : Fix looping bug and made Full Knn calculation parallelizable

Posted by GitBox <gi...@apache.org>.
nitirajrathore commented on a change in pull request #55:
URL: https://github.com/apache/lucene/pull/55#discussion_r604607041



##########
File path: lucene/test-framework/src/java/org/apache/lucene/util/FullKnn.java
##########
@@ -0,0 +1,156 @@
+package org.apache.lucene.util;
+
+import org.apache.lucene.index.VectorValues;
+import org.apache.lucene.util.hnsw.NeighborQueue;
+
+import java.io.IOException;
+import java.nio.ByteOrder;
+import java.nio.FloatBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+/**
+ * A utility class to calculate the Full KNN / Exact KNN over a set of query vectors and document vectors.
+ */
+public class FullKnn {
+
+  private final int dim;
+  private final int topK;
+  private final VectorValues.SearchStrategy searchStrategy;
+  private final boolean quiet;
+
+  public FullKnn(int dim, int topK, VectorValues.SearchStrategy searchStrategy, boolean quiet) {
+    this.dim = dim;
+    this.topK = topK;
+    this.searchStrategy = searchStrategy;
+    this.quiet = quiet;
+  }
+
+  /**
+   * internal object to track KNN calculation for one query
+   */
+  private static class KnnJob {
+    public int currDocIndex;
+    float[] queryVector;
+    float[] currDocVector;
+    int queryIndex;
+    NeighborQueue queue;
+    FloatBuffer docVectors;
+    VectorValues.SearchStrategy searchStrategy;
+
+    public KnnJob(int queryIndex, float[] queryVector, int topK, VectorValues.SearchStrategy searchStrategy) {
+      this.queryIndex = queryIndex;
+      this.queryVector = queryVector;
+      this.currDocVector = new float[queryVector.length];
+      queue = new NeighborQueue(topK, searchStrategy.reversed);
+      this.searchStrategy = searchStrategy;
+    }
+
+    public void execute() {
+      while (this.docVectors.hasRemaining()) {
+        this.docVectors.get(this.currDocVector);
+        float d = this.searchStrategy.compare(this.queryVector, this.currDocVector);
+        this.queue.insertWithOverflow(this.currDocIndex, d);
+        this.currDocIndex++;
+      }
+    }
+  }
+
+  /**
+   * computes the exact KNN match for each query vector in queryPath for all the document vectors in docPath
+   *
+   * @param docPath   : path to the file containing the float 32 document vectors in bytes with little-endian byte order
+   *                  Throws exception if topK is greater than number of documents in this file
+   * @param numDocs   : number of vectors in the document vector file at docPath
+   * @param queryPath : path to the file containing the containing 32-bit floating point vectors in little-endian byte order
+   * @param numIters  : number of vectors in the query vector file at queryPath
+   * @param numThreads : create numThreads to parallelize work
+   * @return : returns an int 2D array ( int matches[][]) of size 'numIters x topK'. matches[i] is an array containing
+   * the indexes of the topK most similar document vectors to the ith query vector, and is sorted by similarity, with
+   * the most similar vector first. Similarity is defined by the searchStrategy used to construct this FullKnn.
+   * @throws IOException : if topK is greater than number of documents in docPath file

Review comment:
       Here it is topK actually, but I agree other checks like numDocs and numQueries are missing.




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

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 merged pull request #55: LUCENE-9798 : Fix looping bug and made Full Knn calculation parallelizable

Posted by GitBox <gi...@apache.org>.
msokolov merged pull request #55:
URL: https://github.com/apache/lucene/pull/55


   


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

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] nitirajrathore commented on a change in pull request #55: LUCENE-9798 : Fix looping bug and made Full Knn calculation parallelizable

Posted by GitBox <gi...@apache.org>.
nitirajrathore commented on a change in pull request #55:
URL: https://github.com/apache/lucene/pull/55#discussion_r604604910



##########
File path: lucene/test-framework/src/java/org/apache/lucene/util/FullKnn.java
##########
@@ -0,0 +1,156 @@
+package org.apache.lucene.util;

Review comment:
       again, I am just not sure about the nomenclature, as the package name suggest hnsw algorith, but code is of exact knn which is independent of hnsw.




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

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