That's interesting! One thing I'd say is we don't want to be optimizing for the random vector use case, so from that perspective this is less concerning. However we also don't want to have poor worst-case performance, so we should address this somehow. If you want to probe for degenerate cases, you might also try indexing many copies of the *same* vector. In this case, since the connectComponents() call is best-effort only we could probably address by adding an iteration limit to avoid a runaway execution. Have you checked what is going on? I'm suspecting the graph is radically disconnected and requires repeated attempts at reconnection??
On Wed, Nov 27, 2024 at 8:55 AM Viliam Ďurina <viliam.dur...@gmail.com> wrote: > > Hi all, > > I'm experimenting with parallel HNSW graph building, however I'm experiencing > very long execution times in the `connectComponents` method when using random > vectors. When using SIFT1M dataset, it does not occur. > > I'm adding 100k documents with FLOAT vectors. Dimensionality is 128. With the > SIFT dataset, I'm using the first 100k vectors. I'm using 4 threads. The > execution time for random vectors is around 239s. When using the SIFT > vectors, the execution time is around 4s. That's a huge difference. My CPU is > Core Ultra 9 185H. > > Below is a reproducer for random vectors. The code adds 100k vectors, > committing after each 10k vectors. This creates 10 segments, which are merged > into 1 segment when `forceMerge` is called, with 4 workers. I'm using Lucene > 10.0 on Java 21: > > > public class ConcurrentMergeTest { > private static final Random RANDOM = new Random(42); > > public static void main(String[] args) throws Exception { > Path path = Files.createTempDirectory("conc-merge-test"); > Directory directory = new MMapDirectory(path); > > InfoStream.setDefault(new PrintStreamInfoStream(System.out)); > IndexWriterConfig config = new IndexWriterConfig(); > try (ExecutorService executor = Executors.newFixedThreadPool(4)) { > config.setCodec(new FilterCodec(Codec.getDefault().getName(), > Codec.getDefault()) { > @Override > public KnnVectorsFormat knnVectorsFormat() { > return new Lucene99HnswVectorsFormat(DEFAULT_MAX_CONN, > DEFAULT_BEAM_WIDTH, 4, executor); > } > }); > long start = System.nanoTime(); > > try (IndexWriter writer = new IndexWriter(directory, config)) { > int batchSize = 10_000; > int numVectors = 100_000; > for (int i = 0; i < numVectors; i++) { > float[] vec = randomVector(128); > Document doc = new Document(); > doc.add(new KnnFloatVectorField("vector", vec, > VectorSimilarityFunction.DOT_PRODUCT)); > writer.addDocument(doc); > > if ((i + 1) % batchSize == 0) { // commit after each batch > writer.commit(); > } > } > > System.out.println("Merging all segments..."); > writer.forceMerge(1); > writer.commit(); > System.out.println("Merge completed!"); > } > > System.out.println("Elapsed time: " + > NANOSECONDS.toMillis(System.nanoTime() - start)); > } > > // Verify the index > try (DirectoryReader reader = DirectoryReader.open(directory)) { > System.out.println("Number of documents in the index: " + > reader.numDocs()); > } > } > > private static float[] randomVector(int dim) { > float[] v = new float[dim]; > for (int i = 0; i < dim; i++) { > v[i] = RANDOM.nextInt(219); > } > return v; > } > } > > > > The debug output includes the following, not sure if that's a problem or not. > But no exception is reported to my code: > HNSW 0 [2024-11-27T13:20:12.852323560Z; Lucene Merge Thread #0]: > connectComponents failed on level 1 > HNSW 0 [2024-11-27T13:20:19.833511543Z; Lucene Merge Thread #0]: > connectComponents failed on level 2 > HNSW 0 [2024-11-27T13:20:19.833621748Z; Lucene Merge Thread #0]: > connectComponents 230758 ms > > I'm not attaching the code to use the SIFT dataset as it is a lot longer, > parsing the files. Anybody has any ideas? I'm attaching the whole debug > output. > > Thanks, > Viliam > > --------------------------------------------------------------------- > To unsubscribe, e-mail: java-user-unsubscr...@lucene.apache.org > For additional commands, e-mail: java-user-h...@lucene.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: java-user-unsubscr...@lucene.apache.org For additional commands, e-mail: java-user-h...@lucene.apache.org