msokolov commented on code in PR #12660: URL: https://github.com/apache/lucene/pull/12660#discussion_r1370054483
########## lucene/core/src/java/org/apache/lucene/codecs/lucene95/Lucene95HnswVectorsFormat.java: ########## @@ -146,18 +148,24 @@ public final class Lucene95HnswVectorsFormat extends KnnVectorsFormat { */ private final int beamWidth; + private final int mergeWorkers; + /** Constructs a format using default graph construction parameters */ public Lucene95HnswVectorsFormat() { this(DEFAULT_MAX_CONN, DEFAULT_BEAM_WIDTH); } + public Lucene95HnswVectorsFormat(int maxConn, int beamWidth) { + this(maxConn, beamWidth, DEFAULT_MERGE_WORKER); + } + /** * Constructs a format using the given graph construction parameters. * * @param maxConn the maximum number of connections to a node in the HNSW graph * @param beamWidth the size of the queue maintained during graph construction. */ - public Lucene95HnswVectorsFormat(int maxConn, int beamWidth) { + public Lucene95HnswVectorsFormat(int maxConn, int beamWidth, int mergeWorkers) { Review Comment: eventually I would think we could have something like `IndexWriterConfig.setMergeThreadConcurrency(int)` that would provide a shared thread pool for any work done during merging. Or maybe it could accept an ExecutorService. But agreed since we only have this one use now we could tie it to the KnnVector/Hnsw API. A consequence of exposing this on the format is you would be able to turn it on and off per-field. Is that a benefit or a bug? ########## lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java: ########## @@ -221,34 +296,50 @@ private long printGraphBuildStatus(int node, long start, long t) { return now; } - private void addDiverseNeighbors(int level, int node, GraphBuilderKnnCollector candidates) + private void addDiverseNeighbors(int level, int node, NeighborArray candidates) throws IOException { /* For each of the beamWidth nearest candidates (going from best to worst), select it only if it * is closer to target than it is to any of the already-selected neighbors (ie selected in this method, * since the node is new and has no prior neighbors). */ NeighborArray neighbors = hnsw.getNeighbors(level, node); assert neighbors.size() == 0; // new node - popToScratch(candidates); int maxConnOnLevel = level == 0 ? M * 2 : M; - selectAndLinkDiverse(neighbors, scratch, maxConnOnLevel); + boolean[] mask = selectAndLinkDiverse(neighbors, candidates, maxConnOnLevel); // Link the selected nodes to the new node, and the new node to the selected nodes (again // applying diversity heuristic) - int size = neighbors.size(); - for (int i = 0; i < size; i++) { - int nbr = neighbors.node[i]; + // NOTE: here we're using candidates and mask but not the neighbour array because once we have + // added incoming link there will be possibilities of this node being discovered and neighbour + // array being modified. So using local candidates and mask is a safer option. + for (int i = 0; i < candidates.size(); i++) { + if (mask[i] == false) { + continue; + } + int nbr = candidates.node[i]; NeighborArray nbrsOfNbr = hnsw.getNeighbors(level, nbr); - nbrsOfNbr.addOutOfOrder(node, neighbors.score[i]); - if (nbrsOfNbr.size() > maxConnOnLevel) { - int indexToRemove = findWorstNonDiverse(nbrsOfNbr, nbr); - nbrsOfNbr.removeIndex(indexToRemove); + long start = System.nanoTime(); + nbrsOfNbr.rwlock.writeLock().lock(); Review Comment: I had previously posted a NeighborIterator abstraction that was designed to hide the locking (somewhat) from the HnswSearcher/Builder. That way we don't do any locking in the serial implementation. But @zhaih pointed out it was too aggressively holding the lock around a large critical section. I think we could enhance the concurrent implementation of that abstraction by having it copy the neighbors as this PR does, and then release the lock immediately after copying rather than holding the lock for the entire iteration. -- 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