mayya-sharipova commented on code in PR #14331: URL: https://github.com/apache/lucene/pull/14331#discussion_r2005416489
########## lucene/core/src/java/org/apache/lucene/util/hnsw/ConcurrentHnswMerger.java: ########## @@ -51,19 +57,85 @@ protected HnswBuilder createBuilder(KnnVectorValues mergedVectorValues, int maxO OnHeapHnswGraph graph; BitSet initializedNodes = null; - if (initReader == null) { + if (graphReaders.size() == 0) { graph = new OnHeapHnswGraph(M, maxOrd); } else { + graphReaders.sort(Comparator.comparingInt(GraphReader::graphSize).reversed()); + GraphReader initGraphReader = graphReaders.get(0); + KnnVectorsReader initReader = initGraphReader.reader(); + MergeState.DocMap initDocMap = initGraphReader.initDocMap(); + int initGraphSize = initGraphReader.graphSize(); HnswGraph initializerGraph = ((HnswGraphProvider) initReader).getGraph(fieldInfo.name); + if (initializerGraph.size() == 0) { graph = new OnHeapHnswGraph(M, maxOrd); } else { initializedNodes = new FixedBitSet(maxOrd); - int[] oldToNewOrdinalMap = getNewOrdMapping(mergedVectorValues, initializedNodes); + int[] oldToNewOrdinalMap = + getNewOrdMapping( + fieldInfo, + initReader, + initDocMap, + initGraphSize, + mergedVectorValues, + initializedNodes); graph = InitializedHnswGraphBuilder.initGraph(initializerGraph, oldToNewOrdinalMap, maxOrd); } } return new HnswConcurrentMergeBuilder( taskExecutor, numWorker, scorerSupplier, beamWidth, graph, initializedNodes); } + + /** + * Creates a new mapping from old ordinals to new ordinals and returns the total number of vectors + * in the newly merged segment. + * + * @param mergedVectorValues vector values in the merged segment + * @param initializedNodes track what nodes have been initialized + * @return the mapping from old ordinals to new ordinals + * @throws IOException If an error occurs while reading from the merge state + */ + private static final int[] getNewOrdMapping( + FieldInfo fieldInfo, + KnnVectorsReader initReader, + MergeState.DocMap initDocMap, + int initGraphSize, + KnnVectorValues mergedVectorValues, + BitSet initializedNodes) + throws IOException { + KnnVectorValues.DocIndexIterator initializerIterator = null; + + switch (fieldInfo.getVectorEncoding()) { + case BYTE -> initializerIterator = initReader.getByteVectorValues(fieldInfo.name).iterator(); + case FLOAT32 -> + initializerIterator = initReader.getFloatVectorValues(fieldInfo.name).iterator(); + } + + IntIntHashMap newIdToOldOrdinal = new IntIntHashMap(initGraphSize); + int maxNewDocID = -1; + for (int docId = initializerIterator.nextDoc(); + docId != NO_MORE_DOCS; + docId = initializerIterator.nextDoc()) { + int newId = initDocMap.get(docId); + maxNewDocID = Math.max(newId, maxNewDocID); + newIdToOldOrdinal.put(newId, initializerIterator.index()); + } + + if (maxNewDocID == -1) { + return new int[0]; + } + final int[] oldToNewOrdinalMap = new int[initGraphSize]; + KnnVectorValues.DocIndexIterator mergedVectorIterator = mergedVectorValues.iterator(); + for (int newDocId = mergedVectorIterator.nextDoc(); + newDocId <= maxNewDocID; + newDocId = mergedVectorIterator.nextDoc()) { + int hashDocIndex = newIdToOldOrdinal.indexOf(newDocId); + if (newIdToOldOrdinal.indexExists(hashDocIndex)) { Review Comment: @benwtrent Thanks for the comment. This piece is moved from [IncrementalHNSWGraphMerger](https://github.com/apache/lucene/blob/main/lucene/core/src/java/org/apache/lucene/util/hnsw/IncrementalHnswGraphMerger.java#L154) without modifications. > Is this stuff around indexOf indexExists, etc. just performance improvements over a simple newIdToOldOrdinal.get(...) Agree about `IntIntHashMap` and may be this piece can be optimized. -- 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