Skip to content

Commit

Permalink
Reuse neighborqueue during hnsw index build (attempt 2) (apache#12372)
Browse files Browse the repository at this point in the history
This changes HnswGraphBuilder to re-use the same candidates queues for adding nodes by allocating them in the Builder instance.

This saves about 2.5% of build time and takes memory allocations of NQ long[] from 25% of total to 0%. JFR runs are attached.

The difference from the first attempt (which actually made things slower for some graphs) is that it preserves the original code's behavior of using a 1-sized queue for the search in the levels above where the node actually gets added.

* Re-use NeighborQueue during build's search

* improve javadoc for OnHeapHnswGraphSearcher

* assert that results parameter is minheap as expected

* update CHANGES
  • Loading branch information
jbellis authored Jun 20, 2023
1 parent 8703e44 commit fe0278e
Show file tree
Hide file tree
Showing 4 changed files with 42 additions and 13 deletions.
2 changes: 2 additions & 0 deletions lucene/CHANGES.txt
Original file line number Diff line number Diff line change
Expand Up @@ -78,6 +78,8 @@ Optimizations

* GITHUB#11857, GITHUB#11859, GITHUB#11893, GITHUB#11909: Hunspell: improved suggestion performance (Peter Gromov)

* GITHUB#12372: Reduce allocation during HNSW construction (Jonathan Ellis)

Bug Fixes
---------------------

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -68,6 +68,8 @@ public final class HnswGraphBuilder<T> {
private final RandomAccessVectorValues<T> vectors;
private final SplittableRandom random;
private final HnswGraphSearcher<T> graphSearcher;
private final NeighborQueue entryCandidates; // for upper levels of graph search
private final NeighborQueue beamCandidates; // for levels of graph where we add the node

final OnHeapHnswGraph hnsw;

Expand Down Expand Up @@ -149,6 +151,8 @@ private HnswGraphBuilder(
new FixedBitSet(this.vectors.size()));
// in scratch we store candidates in reverse order: worse candidates are first
scratch = new NeighborArray(Math.max(beamWidth, M + 1), false);
entryCandidates = new NeighborQueue(1, false);
beamCandidates = new NeighborQueue(beamWidth, false);
this.initializedNodes = new HashSet<>();
}

Expand Down Expand Up @@ -250,7 +254,6 @@ public OnHeapHnswGraph getGraph() {

/** Inserts a doc with vector value to the graph */
public void addGraphNode(int node, T value) throws IOException {
NeighborQueue candidates;
final int nodeLevel = getRandomGraphLevel(ml, random);
int curMaxLevel = hnsw.numLevels() - 1;

Expand All @@ -269,13 +272,19 @@ public void addGraphNode(int node, T value) throws IOException {
}

// for levels > nodeLevel search with topk = 1
NeighborQueue candidates = entryCandidates;
for (int level = curMaxLevel; level > nodeLevel; level--) {
candidates = graphSearcher.searchLevel(value, 1, level, eps, vectors, hnsw);
candidates.clear();
graphSearcher.searchLevel(
candidates, value, 1, level, eps, vectors, hnsw, null, Integer.MAX_VALUE);
eps = new int[] {candidates.pop()};
}
// for levels <= nodeLevel search with topk = beamWidth, and add connections
candidates = beamCandidates;
for (int level = Math.min(nodeLevel, curMaxLevel); level >= 0; level--) {
candidates = graphSearcher.searchLevel(value, beamWidth, level, eps, vectors, hnsw);
candidates.clear();
graphSearcher.searchLevel(
candidates, value, beamWidth, level, eps, vectors, hnsw, null, Integer.MAX_VALUE);
eps = candidates.nodes();
hnsw.addNode(level, node);
addDiverseNeighbors(level, node, candidates);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -205,10 +205,12 @@ private static <T> NeighborQueue search(
return new NeighborQueue(1, true);
}
NeighborQueue results;
results = new NeighborQueue(1, false);
int[] eps = new int[] {graph.entryNode()};
int numVisited = 0;
for (int level = graph.numLevels() - 1; level >= 1; level--) {
results = graphSearcher.searchLevel(query, 1, level, eps, vectors, graph, null, visitedLimit);
results.clear();
graphSearcher.searchLevel(results, query, 1, level, eps, vectors, graph, null, visitedLimit);

numVisited += results.visitedCount();
visitedLimit -= results.visitedCount();
Expand All @@ -219,8 +221,9 @@ private static <T> NeighborQueue search(
}
eps[0] = results.pop();
}
results =
graphSearcher.searchLevel(query, topK, 0, eps, vectors, graph, acceptOrds, visitedLimit);
results = new NeighborQueue(topK, false);
graphSearcher.searchLevel(
results, query, topK, 0, eps, vectors, graph, acceptOrds, visitedLimit);
results.setVisitedCount(results.visitedCount() + numVisited);
return results;
}
Expand Down Expand Up @@ -248,10 +251,19 @@ public NeighborQueue searchLevel(
RandomAccessVectorValues<T> vectors,
HnswGraph graph)
throws IOException {
return searchLevel(query, topK, level, eps, vectors, graph, null, Integer.MAX_VALUE);
NeighborQueue results = new NeighborQueue(topK, false);
searchLevel(results, query, topK, level, eps, vectors, graph, null, Integer.MAX_VALUE);
return results;
}

private NeighborQueue searchLevel(
/**
* Add the closest neighbors found to a priority queue (heap). These are returned in REVERSE
* proximity order -- the most distant neighbor of the topK found, i.e. the one with the lowest
* score/comparison value, will be at the top of the heap, while the closest neighbor will be the
* last to be popped.
*/
void searchLevel(
NeighborQueue results,
T query,
int topK,
int level,
Expand All @@ -261,8 +273,9 @@ private NeighborQueue searchLevel(
Bits acceptOrds,
int visitedLimit)
throws IOException {
assert results.isMinHeap();

int size = graph.size();
NeighborQueue results = new NeighborQueue(topK, false);
prepareScratchState(vectors.size());

int numVisited = 0;
Expand Down Expand Up @@ -323,7 +336,6 @@ private NeighborQueue searchLevel(
results.pop();
}
results.setVisitedCount(numVisited);
return results;
}

private float compare(T query, RandomAccessVectorValues<T> vectors, int ord) throws IOException {
Expand Down Expand Up @@ -365,10 +377,12 @@ int graphNextNeighbor(HnswGraph graph) throws IOException {
}

/**
* This class allow {@link OnHeapHnswGraph} to be searched in a thread-safe manner.
* This class allows {@link OnHeapHnswGraph} to be searched in a thread-safe manner by avoiding
* the unsafe methods (seek and nextNeighbor, which maintain state in the graph object) and
* instead maintaining the state in the searcher object.
*
* <p>Note the class itself is NOT thread safe, but since each search will create one new graph
* searcher the search method is thread safe.
* <p>Note the class itself is NOT thread safe, but since each search will create a new Searcher,
* the search methods using this class are thread safe.
*/
private static class OnHeapHnswGraphSearcher<C> extends HnswGraphSearcher<C> {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -175,6 +175,10 @@ public void markIncomplete() {
this.incomplete = true;
}

boolean isMinHeap() {
return order == Order.MIN_HEAP;
}

@Override
public String toString() {
return "Neighbors[" + heap.size() + "]";
Expand Down

0 comments on commit fe0278e

Please sign in to comment.