benwtrent commented on code in PR #12660: URL: https://github.com/apache/lucene/pull/12660#discussion_r1368782730
########## 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: Why don't we have a `ConcurrentNeighborArray`? Always requiring locking by the external caller is not good. NeighborArray needs to change its API so external folks shouldn't worry about the lock. ########## 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: My small concern here is that now we are locking even when no executor is being used and we know its a single thread. How bad are we hitting users that aren't using multi-threaded merge? ########## lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphMerger.java: ########## @@ -0,0 +1,56 @@ +/* + * 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.codecs.KnnVectorsReader; +import org.apache.lucene.index.MergeState; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.InfoStream; + +/** + * Abstraction of merging multiple graphs into one on-heap graph + * + * @lucene.experimental + */ +public interface HnswGraphMerger { Review Comment: NICE! ########## 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: I realize this might just be for testing, but it seems to me that an actual `ExecutorService` should be passed here. Additionally, it seems weird to for this to be on the format. `IndexWriterConfig` needs to control this. I am not 100% sure what the API would look like, however. -- 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