HnswLock: access locks via hash and only use for concurrent indexing (#13581)

hnswlock: hash locks and only use for concurrent indexing
This commit is contained in:
Michael Sokolov 2024-08-01 10:22:56 -04:00 committed by GitHub
parent 3ee85a46af
commit f14eb2b2c5
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
4 changed files with 92 additions and 15 deletions

View File

@ -40,6 +40,7 @@ public class HnswConcurrentMergeBuilder implements HnswBuilder {
private final TaskExecutor taskExecutor;
private final ConcurrentMergeWorker[] workers;
private final HnswLock hnswLock;
private InfoStream infoStream = InfoStream.getDefault();
private boolean frozen;
@ -55,6 +56,7 @@ public class HnswConcurrentMergeBuilder implements HnswBuilder {
this.taskExecutor = taskExecutor;
AtomicInteger workProgress = new AtomicInteger(0);
workers = new ConcurrentMergeWorker[numWorker];
hnswLock = new HnswLock(hnsw);
for (int i = 0; i < numWorker; i++) {
workers[i] =
new ConcurrentMergeWorker(
@ -63,6 +65,7 @@ public class HnswConcurrentMergeBuilder implements HnswBuilder {
beamWidth,
HnswGraphBuilder.randSeed,
hnsw,
hnswLock,
initializedNodes,
workProgress);
}
@ -140,6 +143,7 @@ public class HnswConcurrentMergeBuilder implements HnswBuilder {
int beamWidth,
long seed,
OnHeapHnswGraph hnsw,
HnswLock hnswLock,
BitSet initializedNodes,
AtomicInteger workProgress)
throws IOException {
@ -149,8 +153,9 @@ public class HnswConcurrentMergeBuilder implements HnswBuilder {
beamWidth,
seed,
hnsw,
hnswLock,
new MergeSearcher(
new NeighborQueue(beamWidth, true), new FixedBitSet(hnsw.maxNodeId() + 1)));
new NeighborQueue(beamWidth, true), hnswLock, new FixedBitSet(hnsw.maxNodeId() + 1)));
this.workProgress = workProgress;
this.initializedNodes = initializedNodes;
}
@ -195,26 +200,25 @@ public class HnswConcurrentMergeBuilder implements HnswBuilder {
* that concurrent modification of the graph will not impact the search
*/
private static class MergeSearcher extends HnswGraphSearcher {
private final HnswLock hnswLock;
private int[] nodeBuffer;
private int upto;
private int size;
private MergeSearcher(NeighborQueue candidates, BitSet visited) {
private MergeSearcher(NeighborQueue candidates, HnswLock hnswLock, BitSet visited) {
super(candidates, visited);
this.hnswLock = hnswLock;
}
@Override
void graphSeek(HnswGraph graph, int level, int targetNode) {
NeighborArray neighborArray = ((OnHeapHnswGraph) graph).getNeighbors(level, targetNode);
neighborArray.rwlock.readLock().lock();
try {
try (HnswLock.LockedRow rowLock = hnswLock.read(level, targetNode)) {
NeighborArray neighborArray = rowLock.row;
if (nodeBuffer == null || nodeBuffer.length < neighborArray.size()) {
nodeBuffer = new int[neighborArray.size()];
}
size = neighborArray.size();
if (size >= 0) System.arraycopy(neighborArray.nodes(), 0, nodeBuffer, 0, size);
} finally {
neighborArray.rwlock.readLock().unlock();
}
upto = -1;
}

View File

@ -63,6 +63,7 @@ public class HnswGraphBuilder implements HnswBuilder {
beamCandidates; // for levels of graph where we add the node
protected final OnHeapHnswGraph hnsw;
protected final HnswLock hnswLock;
private InfoStream infoStream = InfoStream.getDefault();
private boolean frozen;
@ -110,6 +111,7 @@ public class HnswGraphBuilder implements HnswBuilder {
beamWidth,
seed,
hnsw,
null,
new HnswGraphSearcher(new NeighborQueue(beamWidth, true), new FixedBitSet(hnsw.size())));
}
@ -131,6 +133,7 @@ public class HnswGraphBuilder implements HnswBuilder {
int beamWidth,
long seed,
OnHeapHnswGraph hnsw,
HnswLock hnswLock,
HnswGraphSearcher graphSearcher)
throws IOException {
if (M <= 0) {
@ -146,6 +149,7 @@ public class HnswGraphBuilder implements HnswBuilder {
this.ml = M == 1 ? 1 : 1 / Math.log(1.0 * M);
this.random = new SplittableRandom(seed);
this.hnsw = hnsw;
this.hnswLock = hnswLock;
this.graphSearcher = graphSearcher;
entryCandidates = new GraphBuilderKnnCollector(1);
beamCandidates = new GraphBuilderKnnCollector(beamWidth);
@ -327,12 +331,14 @@ public class HnswGraphBuilder implements HnswBuilder {
continue;
}
int nbr = candidates.nodes()[i];
NeighborArray nbrsOfNbr = hnsw.getNeighbors(level, nbr);
nbrsOfNbr.rwlock.writeLock().lock();
try {
if (hnswLock != null) {
try (HnswLock.LockedRow rowLock = hnswLock.write(level, nbr)) {
NeighborArray nbrsOfNbr = rowLock.row;
nbrsOfNbr.addAndEnsureDiversity(node, candidates.scores()[i], nbr, scorerSupplier);
}
} else {
NeighborArray nbrsOfNbr = hnsw.getNeighbors(level, nbr);
nbrsOfNbr.addAndEnsureDiversity(node, candidates.scores()[i], nbr, scorerSupplier);
} finally {
nbrsOfNbr.rwlock.writeLock().unlock();
}
}
}

View File

@ -0,0 +1,70 @@
/*
* 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.Closeable;
import java.util.Objects;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
/**
* Provide (read-and-write) locked access to rows of an OnHeapHnswGraph. For use by
* HnswConcurrentMerger and its HnswGraphBuilders.
*/
public class HnswLock {
private static final int NUM_LOCKS = 512;
private final ReentrantReadWriteLock[] locks;
private final OnHeapHnswGraph graph;
HnswLock(OnHeapHnswGraph graph) {
this.graph = graph;
locks = new ReentrantReadWriteLock[NUM_LOCKS];
for (int i = 0; i < NUM_LOCKS; i++) {
locks[i] = new ReentrantReadWriteLock();
}
}
LockedRow read(int level, int node) {
int lockid = Objects.hash(level, node) % NUM_LOCKS;
Lock lock = locks[lockid].readLock();
lock.lock();
return new LockedRow(graph.getNeighbors(level, node), lock);
}
LockedRow write(int level, int node) {
int lockid = Objects.hash(level, node) % NUM_LOCKS;
Lock lock = locks[lockid].writeLock();
lock.lock();
return new LockedRow(graph.getNeighbors(level, node), lock);
}
static class LockedRow implements Closeable {
final Lock lock;
final NeighborArray row;
LockedRow(NeighborArray row, Lock lock) {
this.lock = lock;
this.row = row;
}
@Override
public void close() {
lock.unlock();
}
}
}

View File

@ -19,8 +19,6 @@ package org.apache.lucene.util.hnsw;
import java.io.IOException;
import java.util.Arrays;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.lucene.util.ArrayUtil;
/**
@ -37,7 +35,6 @@ public class NeighborArray {
private final float[] scores;
private final int[] nodes;
private int sortedNodeSize;
public final ReadWriteLock rwlock = new ReentrantReadWriteLock(true);
public NeighborArray(int maxSize, boolean descOrder) {
nodes = new int[maxSize];