benwtrent commented on code in PR #12660:
URL: https://github.com/apache/lucene/pull/12660#discussion_r1371954603


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsWriter.java:
##########
@@ -557,6 +566,12 @@ public void close() throws IOException {
                 IOUtils.close(finalVectorDataInput);
                 segmentWriteState.directory.deleteFile(tempFileName);
               }
+
+              @Override

Review Comment:
   yeah, this seems ok. We just need to be able to close the file handles and 
delete the temporary file. The internal copies should only be able to read and 
score while these still exist. 
   
   If close occurs and something is still hanging onto a copy, it should be 
undefined and throw. Especially since these are internal/experimental objects, 
this is OK by me.



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsFormat.java:
##########
@@ -198,14 +218,25 @@ public Lucene99HnswVectorsFormat(
               + "; beamWidth="
               + beamWidth);
     }
+    if (numMergeWorkers > 1 && mergeExec == null) {
+      throw new IllegalArgumentException(
+          "No executor service passed in when " + numMergeWorkers + " merge 
workers are requested");
+    }
+    if (numMergeWorkers == 1 && mergeExec != null) {
+      throw new IllegalArgumentException(
+          "No executor service is needed as we'll use single thread to merge");
+    }
     this.maxConn = maxConn;
     this.beamWidth = beamWidth;
     this.scalarQuantizedVectorsFormat = scalarQuantize;
+    this.numMergeWorkers = numMergeWorkers;
+    this.mergeExec = mergeExec;
   }
 
   @Override
   public KnnVectorsWriter fieldsWriter(SegmentWriteState state) throws 
IOException {
-    return new Lucene99HnswVectorsWriter(state, maxConn, beamWidth, 
scalarQuantizedVectorsFormat);
+    return new Lucene99HnswVectorsWriter(

Review Comment:
   seems correct to me!



##########
lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java:
##########
@@ -151,61 +159,128 @@ public OnHeapHnswGraph build(int maxOrd) throws 
IOException {
     return hnsw;
   }
 
-  /** Set info-stream to output debugging information * */
+  @Override
   public void setInfoStream(InfoStream infoStream) {
     this.infoStream = infoStream;
   }
 
+  @Override
   public OnHeapHnswGraph getGraph() {
     return hnsw;
   }
 
-  private void addVectors(int maxOrd) throws IOException {
+  protected void addVectors(int minOrd, int maxOrd) throws IOException {
     long start = System.nanoTime(), t = start;
-    for (int node = 0; node < maxOrd; node++) {
+    if (infoStream.isEnabled(HNSW_COMPONENT)) {
+      infoStream.message(HNSW_COMPONENT, "addVectors [" + minOrd + " " + 
maxOrd + ")");
+    }
+    //    System.out.println("addVectors [" + minOrd + " " + maxOrd + ") 
initialized.size=" +
+    // initializedNodes.size());

Review Comment:
   all these comments should probably be removed.



##########
lucene/core/src/java/org/apache/lucene/util/hnsw/NeighborArray.java:
##########
@@ -35,6 +38,9 @@ public class NeighborArray {
   float[] score;
   int[] node;
   private int sortedNodeSize;
+  public final ReadWriteLock rwlock = new ReentrantReadWriteLock(true);

Review Comment:
   It seems like we need a `ConcurrentHNSWGraphBuilder`. It can use much of the 
same logic, but it instead uses concurrent neighbors. But that can be a later 
refactor.



-- 
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-unsubscr...@lucene.apache.org

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


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

Reply via email to