You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by "benwtrent (via GitHub)" <gi...@apache.org> on 2023/10/11 19:21:28 UTC

[PR] Extract the hnsw graph merging from being part of the vector writer [lucene]

benwtrent opened a new pull request, #12657:
URL: https://github.com/apache/lucene/pull/12657

   While working on the quantization codec & thinking about how merging will evolve, it became clearer that having merging attached directly to the vector writer is weird.
   
   I extracted it out to its own class and removed the "initializedNodes" logic from the base class builder. 
   
   Also, there was on other refactoring around grabbing sorted nodes from the neighbor iterator, I just moved that static method so its not attached to the writer (as all bwc writers need it and all future HNSW writers will as well).


-- 
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-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Extract the hnsw graph merging from being part of the vector writer [lucene]

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12657:
URL: https://github.com/apache/lucene/pull/12657#discussion_r1359865451


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/IncrementalHnswGraphMerger.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.codecs.lucene95;
+
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+import java.io.IOException;
+import org.apache.lucene.codecs.HnswGraphProvider;
+import org.apache.lucene.codecs.KnnVectorsReader;
+import org.apache.lucene.codecs.KnnVectorsWriter;
+import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
+import org.apache.lucene.index.ByteVectorValues;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FloatVectorValues;
+import org.apache.lucene.index.MergeState;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.BitSet;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.hnsw.HnswGraph;
+import org.apache.lucene.util.hnsw.HnswGraphBuilder;
+import org.apache.lucene.util.hnsw.InitializedHnswGraphBuilder;
+import org.apache.lucene.util.hnsw.RandomVectorScorerSupplier;
+
+/**
+ * This selects the biggest Hnsw graph from the provided merge state and initializes a new
+ * HnswGraphBuilder with that graph as a starting point.
+ */
+public class IncrementalHnswGraphMerger {
+
+  private KnnVectorsReader initReader;
+  private MergeState.DocMap initDocMap;
+  private int initGraphSize;
+  private final FieldInfo fieldInfo;
+  private final RandomVectorScorerSupplier scorerSupplier;
+  private final int M;
+  private final int beamWidth;
+
+  /**
+   * @param fieldInfo FieldInfo for the field being merged
+   */
+  public IncrementalHnswGraphMerger(
+      FieldInfo fieldInfo, RandomVectorScorerSupplier scorerSupplier, int M, int beamWidth) {
+    this.fieldInfo = fieldInfo;
+    this.scorerSupplier = scorerSupplier;
+    this.M = M;
+    this.beamWidth = beamWidth;
+  }
+
+  /**
+   * Adds a reader to the graph merger if it meets the following criteria: 1. Does not contain any
+   * deleted docs 2. Is a HnswGraphProvider/PerFieldKnnVectorReader 3. Has the most docs of any
+   * previous reader that met the above criteria
+   *
+   * @param reader KnnVectorsReader to add to the merger
+   * @param docMap MergeState.DocMap for the reader
+   * @param liveDocs Bits representing live docs, can be null
+   * @return this
+   * @throws IOException If an error occurs while reading from the merge state
+   */
+  IncrementalHnswGraphMerger addReader(
+      KnnVectorsReader reader, MergeState.DocMap docMap, Bits liveDocs) throws IOException {
+    KnnVectorsReader currKnnVectorsReader = reader;
+    if (reader instanceof PerFieldKnnVectorsFormat.FieldsReader candidateReader) {
+      currKnnVectorsReader = candidateReader.getFieldReader(fieldInfo.name);
+    }
+
+    if (!(currKnnVectorsReader instanceof HnswGraphProvider) || !allMatch(liveDocs)) {
+      return this;
+    }
+
+    int candidateVectorCount = 0;
+    switch (fieldInfo.getVectorEncoding()) {
+      case BYTE -> {
+        ByteVectorValues byteVectorValues =
+            currKnnVectorsReader.getByteVectorValues(fieldInfo.name);
+        if (byteVectorValues == null) {
+          return this;
+        }
+        candidateVectorCount = byteVectorValues.size();
+      }
+      case FLOAT32 -> {
+        FloatVectorValues vectorValues = currKnnVectorsReader.getFloatVectorValues(fieldInfo.name);
+        if (vectorValues == null) {
+          return this;
+        }
+        candidateVectorCount = vectorValues.size();
+      }
+    }
+    if (candidateVectorCount > initGraphSize) {
+      initReader = currKnnVectorsReader;
+      initDocMap = docMap;
+      initGraphSize = candidateVectorCount;
+    }
+    return this;
+  }
+
+  /**
+   * Builds a new HnswGraphBuilder using the biggest graph from the merge state as a starting point.
+   * If no valid readers were added to the merge state, a new graph is created.
+   *
+   * @param mergeState MergeState for the merge
+   * @return HnswGraphBuilder
+   * @throws IOException If an error occurs while reading from the merge state
+   */
+  public HnswGraphBuilder createBuilder(MergeState mergeState) throws IOException {
+    if (initReader == null) {
+      return HnswGraphBuilder.create(scorerSupplier, M, beamWidth, HnswGraphBuilder.randSeed);
+    }
+
+    HnswGraph initializerGraph = ((HnswGraphProvider) initReader).getGraph(fieldInfo.name);
+    int ordBaseline = getNewOrdOffset(mergeState);
+    BitSet initializedNodes =
+        BitSet.of(
+            DocIdSetIterator.range(ordBaseline, initGraphSize + ordBaseline),
+            mergeState.segmentInfo.maxDoc() + 1);
+    return InitializedHnswGraphBuilder.fromGraph(
+        scorerSupplier,
+        M,
+        beamWidth,
+        HnswGraphBuilder.randSeed,
+        initializerGraph,
+        ordBaseline,
+        initializedNodes);
+  }
+
+  private int getNewOrdOffset(MergeState mergeState) throws IOException {
+    final DocIdSetIterator initializerIterator =
+        switch (fieldInfo.getVectorEncoding()) {
+          case BYTE -> initReader.getByteVectorValues(fieldInfo.name);
+          case FLOAT32 -> initReader.getFloatVectorValues(fieldInfo.name);
+        };
+
+    // minDoc is just the first doc in the main segment containing our graph
+    // we don't need to worry about deleted documents as deletions are not allowed when selecting
+    // the best graph
+    int minDoc = initializerIterator.nextDoc();
+    if (minDoc == NO_MORE_DOCS) {
+      return -1;
+    }
+    minDoc = initDocMap.get(minDoc);
+
+    DocIdSetIterator vectorIterator = null;
+    switch (fieldInfo.getVectorEncoding()) {
+      case BYTE -> vectorIterator =
+          KnnVectorsWriter.MergedVectorValues.mergeByteVectorValues(fieldInfo, mergeState);
+      case FLOAT32 -> vectorIterator =
+          KnnVectorsWriter.MergedVectorValues.mergeFloatVectorValues(fieldInfo, mergeState);
+    }
+
+    // Since there are no deleted documents in our chosen segment, we can assume that the ordinals
+    // are unchanged
+    // meaning we only need to know what ordinal offset to select and apply it

Review Comment:
   You are correct! I will revert back to the integer map.



-- 
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-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Extract the hnsw graph merging from being part of the vector writer [lucene]

Posted by "zhaih (via GitHub)" <gi...@apache.org>.
zhaih commented on code in PR #12657:
URL: https://github.com/apache/lucene/pull/12657#discussion_r1357779437


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/IncrementalHnswGraphMerger.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.codecs.lucene95;
+
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.lucene.codecs.HnswGraphProvider;
+import org.apache.lucene.codecs.KnnVectorsReader;
+import org.apache.lucene.codecs.KnnVectorsWriter;
+import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
+import org.apache.lucene.index.ByteVectorValues;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FloatVectorValues;
+import org.apache.lucene.index.MergeState;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.hnsw.HnswGraph;
+import org.apache.lucene.util.hnsw.HnswGraphBuilder;
+import org.apache.lucene.util.hnsw.InitializedHnswGraphBuilder;
+import org.apache.lucene.util.hnsw.RandomVectorScorerSupplier;
+
+/**
+ * This selects the biggest Hnsw graph from the provided merge state and initializes a new
+ * HnswGraphBuilder with that graph as a starting point.
+ */
+public class IncrementalHnswGraphMerger {
+
+  private KnnVectorsReader curReader;
+  private MergeState.DocMap curDocMap;
+  private int curGraphSize;

Review Comment:
   maybe rename to `initGraphReader`, `initGraphDocMap` and `initGraphSize`?



##########
lucene/core/src/java/org/apache/lucene/util/hnsw/InitializedHnswGraphBuilder.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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 static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.Set;
+import org.apache.lucene.util.CollectionUtil;
+
+/**
+ * This creates a graph builder that is initialized with the provided HnswGraph. This is useful for
+ * merging HnswGraphs from multiple segments.
+ *
+ * @lucene.internal
+ */
+public final class InitializedHnswGraphBuilder extends HnswGraphBuilder {

Review Comment:
   I wonder whether it'll be better to allow `HnswGraphBuilder` to accept a `OnHeapHnswGraph` as a constructor parameter and then when we init from graph we will use this `InitializedHnswGraphBuilder` to build a graph and then pass the built graph and a node filter (to avoid reindex the same nodes) to the normal `HnswGraphBuilder`?
   
   Then for example if we want to have a multi-thread `ConcurrentHnswGraphBuilder` we can still use this `InitializedHnswGraphBuilder` to build the init graph and pass it to the `ConcurentHnswGraphBuilder`?
   
   I mentioned this because in my draft concurrent HNSW merge PR #12660 I do need to pass the HNSW graph to a builder per thread, altho it can be done in various ways. But I still feel using parent/child class to separate this can make things a little bit hard later? Like, if I want a concurrent builder will the concurrent builder extend from this class? If so we need to be quite careful not to inherit a wrong behavior from the original `HnswGraphBuilder` and things can become quite complex?



-- 
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-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Extract the hnsw graph merging from being part of the vector writer [lucene]

Posted by "zhaih (via GitHub)" <gi...@apache.org>.
zhaih commented on code in PR #12657:
URL: https://github.com/apache/lucene/pull/12657#discussion_r1359606481


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/IncrementalHnswGraphMerger.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.codecs.lucene95;
+
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+import java.io.IOException;
+import org.apache.lucene.codecs.HnswGraphProvider;
+import org.apache.lucene.codecs.KnnVectorsReader;
+import org.apache.lucene.codecs.KnnVectorsWriter;
+import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
+import org.apache.lucene.index.ByteVectorValues;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FloatVectorValues;
+import org.apache.lucene.index.MergeState;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.BitSet;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.hnsw.HnswGraph;
+import org.apache.lucene.util.hnsw.HnswGraphBuilder;
+import org.apache.lucene.util.hnsw.InitializedHnswGraphBuilder;
+import org.apache.lucene.util.hnsw.RandomVectorScorerSupplier;
+
+/**
+ * This selects the biggest Hnsw graph from the provided merge state and initializes a new
+ * HnswGraphBuilder with that graph as a starting point.
+ */
+public class IncrementalHnswGraphMerger {

Review Comment:
   Should we put it in `util/hnsw` package instead?



-- 
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-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Extract the hnsw graph merging from being part of the vector writer [lucene]

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12657:
URL: https://github.com/apache/lucene/pull/12657#discussion_r1356818110


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/IncrementalHnswGraphMerger.java:
##########
@@ -0,0 +1,226 @@
+/*
+ * 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.codecs.lucene95;
+
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.lucene.codecs.HnswGraphProvider;
+import org.apache.lucene.codecs.KnnVectorsReader;
+import org.apache.lucene.codecs.KnnVectorsWriter;
+import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
+import org.apache.lucene.index.ByteVectorValues;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FloatVectorValues;
+import org.apache.lucene.index.MergeState;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.hnsw.HnswGraph;
+import org.apache.lucene.util.hnsw.HnswGraphBuilder;
+import org.apache.lucene.util.hnsw.InitializedHnswGraphBuilder;
+import org.apache.lucene.util.hnsw.RandomVectorScorerSupplier;
+
+/**
+ * This selects the biggest Hnsw graph from the provided merge state and initializes a new
+ * HnswGraphBuilder with that graph as a starting point.
+ */
+public class IncrementalHnswGraphMerger {
+
+  private final MergeState mergeState;
+  private final FieldInfo fieldInfo;
+
+  /**
+   * @param mergeState MergeState containing the readers to merge
+   * @param fieldInfo FieldInfo for the field being merged
+   */
+  public IncrementalHnswGraphMerger(MergeState mergeState, FieldInfo fieldInfo) {
+    this.mergeState = mergeState;
+    this.fieldInfo = fieldInfo;
+  }
+
+  /**
+   * Selects the biggest Hnsw graph from the provided merge state and initializes a new
+   * HnswGraphBuilder with that graph as a starting point.
+   *
+   * @param scorerSupplier ScorerSupplier to use for scoring vectors
+   * @param M The number of connections to allow per node
+   * @param beamWidth The number of nodes to consider when searching for the nearest neighbor
+   * @return HnswGraphBuilder initialized with the biggest graph from the merge state
+   * @throws IOException If an error occurs while reading from the merge state
+   */
+  public HnswGraphBuilder build(RandomVectorScorerSupplier scorerSupplier, int M, int beamWidth)

Review Comment:
   I guess, " fully merged graph" isn't possible as we will always need to account for vectors that couldn't be read from a graph (i.e. previous Lucene 95). 
   
   So, accepting graphs and returning a builder seems like the best API. I will iterate on how it looks.
   
   I don't expect this to be perfect and we may have to change it when we actually add a new merger kind.



-- 
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-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Extract the hnsw graph merging from being part of the vector writer [lucene]

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent merged PR #12657:
URL: https://github.com/apache/lucene/pull/12657


-- 
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-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Extract the hnsw graph merging from being part of the vector writer [lucene]

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12657:
URL: https://github.com/apache/lucene/pull/12657#discussion_r1358143088


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/IncrementalHnswGraphMerger.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.codecs.lucene95;
+
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.lucene.codecs.HnswGraphProvider;
+import org.apache.lucene.codecs.KnnVectorsReader;
+import org.apache.lucene.codecs.KnnVectorsWriter;
+import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
+import org.apache.lucene.index.ByteVectorValues;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FloatVectorValues;
+import org.apache.lucene.index.MergeState;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.hnsw.HnswGraph;
+import org.apache.lucene.util.hnsw.HnswGraphBuilder;
+import org.apache.lucene.util.hnsw.InitializedHnswGraphBuilder;
+import org.apache.lucene.util.hnsw.RandomVectorScorerSupplier;
+
+/**
+ * This selects the biggest Hnsw graph from the provided merge state and initializes a new
+ * HnswGraphBuilder with that graph as a starting point.
+ */
+public class IncrementalHnswGraphMerger {
+
+  private KnnVectorsReader curReader;
+  private MergeState.DocMap curDocMap;
+  private int curGraphSize;
+  private final FieldInfo fieldInfo;
+  private final RandomVectorScorerSupplier scorerSupplier;
+  private final int M;
+  private final int beamWidth;
+
+  /**
+   * @param fieldInfo FieldInfo for the field being merged
+   */
+  public IncrementalHnswGraphMerger(
+      FieldInfo fieldInfo, RandomVectorScorerSupplier scorerSupplier, int M, int beamWidth) {
+    this.fieldInfo = fieldInfo;
+    this.scorerSupplier = scorerSupplier;
+    this.M = M;
+    this.beamWidth = beamWidth;
+  }
+
+  /**
+   * Adds a reader to the graph merger if it meets the following criteria: 1. Does not contain any
+   * deleted docs 2. Is a HnswGraphProvider/PerFieldKnnVectorReader 3. Has the most docs of any
+   * previous reader that met the above criteria
+   *
+   * @param reader KnnVectorsReader to add to the merger
+   * @param docMap MergeState.DocMap for the reader
+   * @param liveDocs Bits representing live docs, can be null
+   * @return this
+   * @throws IOException If an error occurs while reading from the merge state
+   */
+  IncrementalHnswGraphMerger addReader(
+      KnnVectorsReader reader, MergeState.DocMap docMap, Bits liveDocs) throws IOException {
+    KnnVectorsReader currKnnVectorsReader = reader;
+    if (reader instanceof PerFieldKnnVectorsFormat.FieldsReader candidateReader) {
+      currKnnVectorsReader = candidateReader.getFieldReader(fieldInfo.name);
+    }
+
+    if (!(currKnnVectorsReader instanceof HnswGraphProvider) || !allMatch(liveDocs)) {
+      return this;
+    }
+
+    int candidateVectorCount = 0;
+    switch (fieldInfo.getVectorEncoding()) {
+      case BYTE -> {
+        ByteVectorValues byteVectorValues =
+            currKnnVectorsReader.getByteVectorValues(fieldInfo.name);
+        if (byteVectorValues == null) {
+          return this;
+        }
+        candidateVectorCount = byteVectorValues.size();
+      }
+      case FLOAT32 -> {
+        FloatVectorValues vectorValues = currKnnVectorsReader.getFloatVectorValues(fieldInfo.name);
+        if (vectorValues == null) {
+          return this;
+        }
+        candidateVectorCount = vectorValues.size();
+      }
+    }
+    if (candidateVectorCount > curGraphSize) {
+      curReader = currKnnVectorsReader;
+      curDocMap = docMap;
+      curGraphSize = candidateVectorCount;
+    }
+    return this;
+  }
+
+  /**
+   * Builds a new HnswGraphBuilder using the biggest graph from the merge state as a starting point.
+   * If no valid readers were added to the merge state, a new graph is created.
+   *
+   * @param mergeState MergeState for the merge
+   * @return HnswGraphBuilder
+   * @throws IOException If an error occurs while reading from the merge state
+   */
+  public HnswGraphBuilder createBuilder(MergeState mergeState) throws IOException {
+    if (curReader == null) {
+      return HnswGraphBuilder.create(scorerSupplier, M, beamWidth, HnswGraphBuilder.randSeed);
+    }
+
+    HnswGraph initializerGraph = ((HnswGraphProvider) curReader).getGraph(fieldInfo.name);
+    Map<Integer, Integer> ordinalMapper = getOldToNewOrdinalMap(mergeState);
+    return new InitializedHnswGraphBuilder(
+        scorerSupplier, M, beamWidth, HnswGraphBuilder.randSeed, initializerGraph, ordinalMapper);
+  }
+
+  private Map<Integer, Integer> getOldToNewOrdinalMap(MergeState mergeState) throws IOException {
+
+    DocIdSetIterator initializerIterator = null;
+    switch (fieldInfo.getVectorEncoding()) {
+      case BYTE -> initializerIterator = curReader.getByteVectorValues(fieldInfo.name);
+      case FLOAT32 -> initializerIterator = curReader.getFloatVectorValues(fieldInfo.name);
+    }
+
+    Map<Integer, Integer> newIdToOldOrdinal = new HashMap<>();
+    int oldOrd = 0;
+    int maxNewDocID = -1;
+    for (int oldId = initializerIterator.nextDoc();
+        oldId != NO_MORE_DOCS;
+        oldId = initializerIterator.nextDoc()) {
+      if (isCurrentVectorNull(initializerIterator)) {
+        continue;
+      }

Review Comment:
   Ah, ok. The vector readers automatically handle this by skipping docs. I will remove these null checks and simplify this method.



-- 
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-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Extract the hnsw graph merging from being part of the vector writer [lucene]

Posted by "zhaih (via GitHub)" <gi...@apache.org>.
zhaih commented on code in PR #12657:
URL: https://github.com/apache/lucene/pull/12657#discussion_r1355784178


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/IncrementalHnswGraphMerger.java:
##########
@@ -0,0 +1,226 @@
+/*
+ * 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.codecs.lucene95;
+
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.lucene.codecs.HnswGraphProvider;
+import org.apache.lucene.codecs.KnnVectorsReader;
+import org.apache.lucene.codecs.KnnVectorsWriter;
+import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
+import org.apache.lucene.index.ByteVectorValues;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FloatVectorValues;
+import org.apache.lucene.index.MergeState;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.hnsw.HnswGraph;
+import org.apache.lucene.util.hnsw.HnswGraphBuilder;
+import org.apache.lucene.util.hnsw.InitializedHnswGraphBuilder;
+import org.apache.lucene.util.hnsw.RandomVectorScorerSupplier;
+
+/**
+ * This selects the biggest Hnsw graph from the provided merge state and initializes a new
+ * HnswGraphBuilder with that graph as a starting point.
+ */
+public class IncrementalHnswGraphMerger {
+
+  private final MergeState mergeState;
+  private final FieldInfo fieldInfo;
+
+  /**
+   * @param mergeState MergeState containing the readers to merge
+   * @param fieldInfo FieldInfo for the field being merged
+   */
+  public IncrementalHnswGraphMerger(MergeState mergeState, FieldInfo fieldInfo) {
+    this.mergeState = mergeState;
+    this.fieldInfo = fieldInfo;
+  }
+
+  /**
+   * Selects the biggest Hnsw graph from the provided merge state and initializes a new
+   * HnswGraphBuilder with that graph as a starting point.
+   *
+   * @param scorerSupplier ScorerSupplier to use for scoring vectors
+   * @param M The number of connections to allow per node
+   * @param beamWidth The number of nodes to consider when searching for the nearest neighbor
+   * @return HnswGraphBuilder initialized with the biggest graph from the merge state
+   * @throws IOException If an error occurs while reading from the merge state
+   */
+  public HnswGraphBuilder build(RandomVectorScorerSupplier scorerSupplier, int M, int beamWidth)
+      throws IOException {
+    // Find the KnnVectorReader with the most docs that meets the following criteria:

Review Comment:
   Can we abstract this part as a separate method as well?
   Then the overall flow will be more clear I guess:
   
    * find init graph
    * if not found, use default builder
    * if found use initializer



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/IncrementalHnswGraphMerger.java:
##########
@@ -0,0 +1,226 @@
+/*
+ * 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.codecs.lucene95;
+
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.lucene.codecs.HnswGraphProvider;
+import org.apache.lucene.codecs.KnnVectorsReader;
+import org.apache.lucene.codecs.KnnVectorsWriter;
+import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
+import org.apache.lucene.index.ByteVectorValues;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FloatVectorValues;
+import org.apache.lucene.index.MergeState;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.hnsw.HnswGraph;
+import org.apache.lucene.util.hnsw.HnswGraphBuilder;
+import org.apache.lucene.util.hnsw.InitializedHnswGraphBuilder;
+import org.apache.lucene.util.hnsw.RandomVectorScorerSupplier;
+
+/**
+ * This selects the biggest Hnsw graph from the provided merge state and initializes a new
+ * HnswGraphBuilder with that graph as a starting point.
+ */
+public class IncrementalHnswGraphMerger {
+
+  private final MergeState mergeState;
+  private final FieldInfo fieldInfo;
+
+  /**
+   * @param mergeState MergeState containing the readers to merge
+   * @param fieldInfo FieldInfo for the field being merged
+   */
+  public IncrementalHnswGraphMerger(MergeState mergeState, FieldInfo fieldInfo) {
+    this.mergeState = mergeState;
+    this.fieldInfo = fieldInfo;
+  }
+
+  /**
+   * Selects the biggest Hnsw graph from the provided merge state and initializes a new
+   * HnswGraphBuilder with that graph as a starting point.
+   *
+   * @param scorerSupplier ScorerSupplier to use for scoring vectors
+   * @param M The number of connections to allow per node
+   * @param beamWidth The number of nodes to consider when searching for the nearest neighbor
+   * @return HnswGraphBuilder initialized with the biggest graph from the merge state
+   * @throws IOException If an error occurs while reading from the merge state
+   */
+  public HnswGraphBuilder build(RandomVectorScorerSupplier scorerSupplier, int M, int beamWidth)

Review Comment:
   rename to `builder` or `getBuilder`?



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/IncrementalHnswGraphMerger.java:
##########
@@ -0,0 +1,226 @@
+/*
+ * 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.codecs.lucene95;
+
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.lucene.codecs.HnswGraphProvider;
+import org.apache.lucene.codecs.KnnVectorsReader;
+import org.apache.lucene.codecs.KnnVectorsWriter;
+import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
+import org.apache.lucene.index.ByteVectorValues;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FloatVectorValues;
+import org.apache.lucene.index.MergeState;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.hnsw.HnswGraph;
+import org.apache.lucene.util.hnsw.HnswGraphBuilder;
+import org.apache.lucene.util.hnsw.InitializedHnswGraphBuilder;
+import org.apache.lucene.util.hnsw.RandomVectorScorerSupplier;
+
+/**
+ * This selects the biggest Hnsw graph from the provided merge state and initializes a new
+ * HnswGraphBuilder with that graph as a starting point.
+ */
+public class IncrementalHnswGraphMerger {
+
+  private final MergeState mergeState;
+  private final FieldInfo fieldInfo;
+
+  /**
+   * @param mergeState MergeState containing the readers to merge
+   * @param fieldInfo FieldInfo for the field being merged
+   */
+  public IncrementalHnswGraphMerger(MergeState mergeState, FieldInfo fieldInfo) {
+    this.mergeState = mergeState;
+    this.fieldInfo = fieldInfo;
+  }
+
+  /**
+   * Selects the biggest Hnsw graph from the provided merge state and initializes a new
+   * HnswGraphBuilder with that graph as a starting point.
+   *
+   * @param scorerSupplier ScorerSupplier to use for scoring vectors
+   * @param M The number of connections to allow per node
+   * @param beamWidth The number of nodes to consider when searching for the nearest neighbor
+   * @return HnswGraphBuilder initialized with the biggest graph from the merge state
+   * @throws IOException If an error occurs while reading from the merge state
+   */
+  public HnswGraphBuilder build(RandomVectorScorerSupplier scorerSupplier, int M, int beamWidth)

Review Comment:
   This method seems can be a static method if we make the two member fields part of the parameter as this is the only public method?
   
   But I guess we're expecting more different mergers (like a concurrent one?) such that you make the class look like this?



-- 
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-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Extract the hnsw graph merging from being part of the vector writer [lucene]

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12657:
URL: https://github.com/apache/lucene/pull/12657#discussion_r1356733186


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/IncrementalHnswGraphMerger.java:
##########
@@ -0,0 +1,226 @@
+/*
+ * 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.codecs.lucene95;
+
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.lucene.codecs.HnswGraphProvider;
+import org.apache.lucene.codecs.KnnVectorsReader;
+import org.apache.lucene.codecs.KnnVectorsWriter;
+import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
+import org.apache.lucene.index.ByteVectorValues;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FloatVectorValues;
+import org.apache.lucene.index.MergeState;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.hnsw.HnswGraph;
+import org.apache.lucene.util.hnsw.HnswGraphBuilder;
+import org.apache.lucene.util.hnsw.InitializedHnswGraphBuilder;
+import org.apache.lucene.util.hnsw.RandomVectorScorerSupplier;
+
+/**
+ * This selects the biggest Hnsw graph from the provided merge state and initializes a new
+ * HnswGraphBuilder with that graph as a starting point.
+ */
+public class IncrementalHnswGraphMerger {
+
+  private final MergeState mergeState;
+  private final FieldInfo fieldInfo;
+
+  /**
+   * @param mergeState MergeState containing the readers to merge
+   * @param fieldInfo FieldInfo for the field being merged
+   */
+  public IncrementalHnswGraphMerger(MergeState mergeState, FieldInfo fieldInfo) {
+    this.mergeState = mergeState;
+    this.fieldInfo = fieldInfo;
+  }
+
+  /**
+   * Selects the biggest Hnsw graph from the provided merge state and initializes a new
+   * HnswGraphBuilder with that graph as a starting point.
+   *
+   * @param scorerSupplier ScorerSupplier to use for scoring vectors
+   * @param M The number of connections to allow per node
+   * @param beamWidth The number of nodes to consider when searching for the nearest neighbor
+   * @return HnswGraphBuilder initialized with the biggest graph from the merge state
+   * @throws IOException If an error occurs while reading from the merge state
+   */
+  public HnswGraphBuilder build(RandomVectorScorerSupplier scorerSupplier, int M, int beamWidth)

Review Comment:
   @zhaih I think so. But thinking about this more, it makes sense to me that a `HnswGraphMerger` accepts graphs, and then returns a fully merged graph. So, I am gonna reiterate on the API a bit.



-- 
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-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Extract the hnsw graph merging from being part of the vector writer [lucene]

Posted by "zhaih (via GitHub)" <gi...@apache.org>.
zhaih commented on code in PR #12657:
URL: https://github.com/apache/lucene/pull/12657#discussion_r1361341835


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/IncrementalHnswGraphMerger.java:
##########
@@ -0,0 +1,197 @@
+/*
+ * 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 static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+import java.io.IOException;
+import java.util.Map;
+import org.apache.lucene.codecs.HnswGraphProvider;
+import org.apache.lucene.codecs.KnnVectorsReader;
+import org.apache.lucene.codecs.KnnVectorsWriter;
+import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
+import org.apache.lucene.index.ByteVectorValues;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FloatVectorValues;
+import org.apache.lucene.index.MergeState;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.BitSet;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.CollectionUtil;
+import org.apache.lucene.util.FixedBitSet;
+
+/**
+ * This selects the biggest Hnsw graph from the provided merge state and initializes a new
+ * HnswGraphBuilder with that graph as a starting point.
+ *
+ * @lucene.experimental
+ */
+public class IncrementalHnswGraphMerger {
+
+  private KnnVectorsReader initReader;
+  private MergeState.DocMap initDocMap;
+  private int initGraphSize;
+  private final FieldInfo fieldInfo;
+  private final RandomVectorScorerSupplier scorerSupplier;
+  private final int M;
+  private final int beamWidth;
+
+  /**
+   * @param fieldInfo FieldInfo for the field being merged
+   */
+  public IncrementalHnswGraphMerger(
+      FieldInfo fieldInfo, RandomVectorScorerSupplier scorerSupplier, int M, int beamWidth) {
+    this.fieldInfo = fieldInfo;
+    this.scorerSupplier = scorerSupplier;
+    this.M = M;
+    this.beamWidth = beamWidth;
+  }
+
+  /**
+   * Adds a reader to the graph merger if it meets the following criteria: 1. Does not contain any
+   * deleted docs 2. Is a HnswGraphProvider/PerFieldKnnVectorReader 3. Has the most docs of any
+   * previous reader that met the above criteria
+   *
+   * @param reader KnnVectorsReader to add to the merger
+   * @param docMap MergeState.DocMap for the reader
+   * @param liveDocs Bits representing live docs, can be null
+   * @return this
+   * @throws IOException If an error occurs while reading from the merge state
+   */
+  public IncrementalHnswGraphMerger addReader(
+      KnnVectorsReader reader, MergeState.DocMap docMap, Bits liveDocs) throws IOException {
+    KnnVectorsReader currKnnVectorsReader = reader;
+    if (reader instanceof PerFieldKnnVectorsFormat.FieldsReader candidateReader) {
+      currKnnVectorsReader = candidateReader.getFieldReader(fieldInfo.name);
+    }
+
+    if (!(currKnnVectorsReader instanceof HnswGraphProvider) || !allMatch(liveDocs)) {
+      return this;
+    }
+
+    int candidateVectorCount = 0;
+    switch (fieldInfo.getVectorEncoding()) {
+      case BYTE -> {
+        ByteVectorValues byteVectorValues =
+            currKnnVectorsReader.getByteVectorValues(fieldInfo.name);
+        if (byteVectorValues == null) {
+          return this;
+        }
+        candidateVectorCount = byteVectorValues.size();
+      }
+      case FLOAT32 -> {
+        FloatVectorValues vectorValues = currKnnVectorsReader.getFloatVectorValues(fieldInfo.name);
+        if (vectorValues == null) {
+          return this;
+        }
+        candidateVectorCount = vectorValues.size();
+      }
+    }
+    if (candidateVectorCount > initGraphSize) {
+      initReader = currKnnVectorsReader;
+      initDocMap = docMap;
+      initGraphSize = candidateVectorCount;
+    }
+    return this;
+  }
+
+  /**
+   * Builds a new HnswGraphBuilder using the biggest graph from the merge state as a starting point.
+   * If no valid readers were added to the merge state, a new graph is created.
+   *
+   * @param mergeState MergeState for the merge
+   * @return HnswGraphBuilder
+   * @throws IOException If an error occurs while reading from the merge state
+   */
+  public HnswGraphBuilder createBuilder(MergeState mergeState) throws IOException {
+    if (initReader == null) {
+      return HnswGraphBuilder.create(scorerSupplier, M, beamWidth, HnswGraphBuilder.randSeed);
+    }
+
+    HnswGraph initializerGraph = ((HnswGraphProvider) initReader).getGraph(fieldInfo.name);
+    BitSet initializedNodes = new FixedBitSet(mergeState.segmentInfo.maxDoc() + 1);
+    int[] ordBaseline = getNewOrdOffset(mergeState, initializedNodes);
+    return InitializedHnswGraphBuilder.fromGraph(
+        scorerSupplier,
+        M,
+        beamWidth,
+        HnswGraphBuilder.randSeed,
+        initializerGraph,
+        ordBaseline,
+        initializedNodes);
+  }
+
+  private int[] getNewOrdOffset(MergeState mergeState, BitSet initializedNodes) throws IOException {
+    DocIdSetIterator initializerIterator = null;
+
+    switch (fieldInfo.getVectorEncoding()) {
+      case BYTE -> initializerIterator = initReader.getByteVectorValues(fieldInfo.name);
+      case FLOAT32 -> initializerIterator = initReader.getFloatVectorValues(fieldInfo.name);
+    }
+
+    Map<Integer, Integer> newIdToOldOrdinal = CollectionUtil.newHashMap(initGraphSize);
+    int oldOrd = 0;
+    int maxNewDocID = -1;
+    for (int oldId = initializerIterator.nextDoc();
+        oldId != NO_MORE_DOCS;
+        oldId = initializerIterator.nextDoc()) {
+      int newId = initDocMap.get(oldId);
+      maxNewDocID = Math.max(newId, maxNewDocID);
+      newIdToOldOrdinal.put(newId, oldOrd);
+      oldOrd++;
+    }
+
+    if (maxNewDocID == -1) {
+      return new int[0];
+    }
+
+    int[] oldToNewOrdinalMap = new int[initGraphSize];
+
+    DocIdSetIterator vectorIterator = null;
+    switch (fieldInfo.getVectorEncoding()) {
+      case BYTE -> vectorIterator =
+          KnnVectorsWriter.MergedVectorValues.mergeByteVectorValues(fieldInfo, mergeState);
+      case FLOAT32 -> vectorIterator =
+          KnnVectorsWriter.MergedVectorValues.mergeFloatVectorValues(fieldInfo, mergeState);
+    }
+
+    int newOrd = 0;
+    for (int newDocId = vectorIterator.nextDoc();
+        newDocId <= maxNewDocID;
+        newDocId = vectorIterator.nextDoc()) {
+      if (newIdToOldOrdinal.containsKey(newDocId)) {
+        initializedNodes.set(newOrd);
+        oldToNewOrdinalMap[newIdToOldOrdinal.get(newDocId)] = newOrd;
+      }
+      newOrd++;
+    }
+    return oldToNewOrdinalMap;
+  }
+
+  private static boolean allMatch(Bits bits) {

Review Comment:
   Let's just name it `noDeletion` and the parameter `livedocs`?



##########
lucene/core/src/java/org/apache/lucene/util/hnsw/IncrementalHnswGraphMerger.java:
##########
@@ -0,0 +1,197 @@
+/*
+ * 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 static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+import java.io.IOException;
+import java.util.Map;
+import org.apache.lucene.codecs.HnswGraphProvider;
+import org.apache.lucene.codecs.KnnVectorsReader;
+import org.apache.lucene.codecs.KnnVectorsWriter;
+import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
+import org.apache.lucene.index.ByteVectorValues;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FloatVectorValues;
+import org.apache.lucene.index.MergeState;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.BitSet;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.CollectionUtil;
+import org.apache.lucene.util.FixedBitSet;
+
+/**
+ * This selects the biggest Hnsw graph from the provided merge state and initializes a new
+ * HnswGraphBuilder with that graph as a starting point.
+ *
+ * @lucene.experimental
+ */
+public class IncrementalHnswGraphMerger {
+
+  private KnnVectorsReader initReader;
+  private MergeState.DocMap initDocMap;
+  private int initGraphSize;
+  private final FieldInfo fieldInfo;
+  private final RandomVectorScorerSupplier scorerSupplier;
+  private final int M;
+  private final int beamWidth;
+
+  /**
+   * @param fieldInfo FieldInfo for the field being merged
+   */
+  public IncrementalHnswGraphMerger(
+      FieldInfo fieldInfo, RandomVectorScorerSupplier scorerSupplier, int M, int beamWidth) {
+    this.fieldInfo = fieldInfo;
+    this.scorerSupplier = scorerSupplier;
+    this.M = M;
+    this.beamWidth = beamWidth;
+  }
+
+  /**
+   * Adds a reader to the graph merger if it meets the following criteria: 1. Does not contain any
+   * deleted docs 2. Is a HnswGraphProvider/PerFieldKnnVectorReader 3. Has the most docs of any
+   * previous reader that met the above criteria
+   *
+   * @param reader KnnVectorsReader to add to the merger
+   * @param docMap MergeState.DocMap for the reader
+   * @param liveDocs Bits representing live docs, can be null
+   * @return this
+   * @throws IOException If an error occurs while reading from the merge state
+   */
+  public IncrementalHnswGraphMerger addReader(
+      KnnVectorsReader reader, MergeState.DocMap docMap, Bits liveDocs) throws IOException {
+    KnnVectorsReader currKnnVectorsReader = reader;
+    if (reader instanceof PerFieldKnnVectorsFormat.FieldsReader candidateReader) {
+      currKnnVectorsReader = candidateReader.getFieldReader(fieldInfo.name);
+    }
+
+    if (!(currKnnVectorsReader instanceof HnswGraphProvider) || !allMatch(liveDocs)) {
+      return this;
+    }
+
+    int candidateVectorCount = 0;
+    switch (fieldInfo.getVectorEncoding()) {
+      case BYTE -> {
+        ByteVectorValues byteVectorValues =
+            currKnnVectorsReader.getByteVectorValues(fieldInfo.name);
+        if (byteVectorValues == null) {
+          return this;
+        }
+        candidateVectorCount = byteVectorValues.size();
+      }
+      case FLOAT32 -> {
+        FloatVectorValues vectorValues = currKnnVectorsReader.getFloatVectorValues(fieldInfo.name);
+        if (vectorValues == null) {
+          return this;
+        }
+        candidateVectorCount = vectorValues.size();
+      }
+    }
+    if (candidateVectorCount > initGraphSize) {
+      initReader = currKnnVectorsReader;
+      initDocMap = docMap;
+      initGraphSize = candidateVectorCount;
+    }
+    return this;
+  }
+
+  /**
+   * Builds a new HnswGraphBuilder using the biggest graph from the merge state as a starting point.
+   * If no valid readers were added to the merge state, a new graph is created.
+   *
+   * @param mergeState MergeState for the merge
+   * @return HnswGraphBuilder
+   * @throws IOException If an error occurs while reading from the merge state
+   */
+  public HnswGraphBuilder createBuilder(MergeState mergeState) throws IOException {
+    if (initReader == null) {
+      return HnswGraphBuilder.create(scorerSupplier, M, beamWidth, HnswGraphBuilder.randSeed);
+    }
+
+    HnswGraph initializerGraph = ((HnswGraphProvider) initReader).getGraph(fieldInfo.name);
+    BitSet initializedNodes = new FixedBitSet(mergeState.segmentInfo.maxDoc() + 1);
+    int[] ordBaseline = getNewOrdOffset(mergeState, initializedNodes);
+    return InitializedHnswGraphBuilder.fromGraph(
+        scorerSupplier,
+        M,
+        beamWidth,
+        HnswGraphBuilder.randSeed,
+        initializerGraph,
+        ordBaseline,
+        initializedNodes);
+  }
+
+  private int[] getNewOrdOffset(MergeState mergeState, BitSet initializedNodes) throws IOException {

Review Comment:
   rename to `getNewOrdMap` and add some explanation about the returned int array?



##########
lucene/core/src/java/org/apache/lucene/util/hnsw/IncrementalHnswGraphMerger.java:
##########
@@ -0,0 +1,197 @@
+/*
+ * 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 static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+import java.io.IOException;
+import java.util.Map;
+import org.apache.lucene.codecs.HnswGraphProvider;
+import org.apache.lucene.codecs.KnnVectorsReader;
+import org.apache.lucene.codecs.KnnVectorsWriter;
+import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
+import org.apache.lucene.index.ByteVectorValues;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FloatVectorValues;
+import org.apache.lucene.index.MergeState;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.BitSet;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.CollectionUtil;
+import org.apache.lucene.util.FixedBitSet;
+
+/**
+ * This selects the biggest Hnsw graph from the provided merge state and initializes a new
+ * HnswGraphBuilder with that graph as a starting point.
+ *
+ * @lucene.experimental
+ */
+public class IncrementalHnswGraphMerger {
+
+  private KnnVectorsReader initReader;
+  private MergeState.DocMap initDocMap;
+  private int initGraphSize;
+  private final FieldInfo fieldInfo;
+  private final RandomVectorScorerSupplier scorerSupplier;
+  private final int M;
+  private final int beamWidth;
+
+  /**
+   * @param fieldInfo FieldInfo for the field being merged
+   */
+  public IncrementalHnswGraphMerger(
+      FieldInfo fieldInfo, RandomVectorScorerSupplier scorerSupplier, int M, int beamWidth) {
+    this.fieldInfo = fieldInfo;
+    this.scorerSupplier = scorerSupplier;
+    this.M = M;
+    this.beamWidth = beamWidth;
+  }
+
+  /**
+   * Adds a reader to the graph merger if it meets the following criteria: 1. Does not contain any
+   * deleted docs 2. Is a HnswGraphProvider/PerFieldKnnVectorReader 3. Has the most docs of any
+   * previous reader that met the above criteria
+   *
+   * @param reader KnnVectorsReader to add to the merger
+   * @param docMap MergeState.DocMap for the reader
+   * @param liveDocs Bits representing live docs, can be null
+   * @return this
+   * @throws IOException If an error occurs while reading from the merge state
+   */
+  public IncrementalHnswGraphMerger addReader(
+      KnnVectorsReader reader, MergeState.DocMap docMap, Bits liveDocs) throws IOException {
+    KnnVectorsReader currKnnVectorsReader = reader;
+    if (reader instanceof PerFieldKnnVectorsFormat.FieldsReader candidateReader) {
+      currKnnVectorsReader = candidateReader.getFieldReader(fieldInfo.name);
+    }
+
+    if (!(currKnnVectorsReader instanceof HnswGraphProvider) || !allMatch(liveDocs)) {
+      return this;
+    }
+
+    int candidateVectorCount = 0;
+    switch (fieldInfo.getVectorEncoding()) {
+      case BYTE -> {
+        ByteVectorValues byteVectorValues =
+            currKnnVectorsReader.getByteVectorValues(fieldInfo.name);
+        if (byteVectorValues == null) {
+          return this;
+        }
+        candidateVectorCount = byteVectorValues.size();
+      }
+      case FLOAT32 -> {
+        FloatVectorValues vectorValues = currKnnVectorsReader.getFloatVectorValues(fieldInfo.name);
+        if (vectorValues == null) {
+          return this;
+        }
+        candidateVectorCount = vectorValues.size();
+      }
+    }
+    if (candidateVectorCount > initGraphSize) {
+      initReader = currKnnVectorsReader;
+      initDocMap = docMap;
+      initGraphSize = candidateVectorCount;
+    }
+    return this;
+  }
+
+  /**
+   * Builds a new HnswGraphBuilder using the biggest graph from the merge state as a starting point.
+   * If no valid readers were added to the merge state, a new graph is created.
+   *
+   * @param mergeState MergeState for the merge
+   * @return HnswGraphBuilder
+   * @throws IOException If an error occurs while reading from the merge state
+   */
+  public HnswGraphBuilder createBuilder(MergeState mergeState) throws IOException {
+    if (initReader == null) {
+      return HnswGraphBuilder.create(scorerSupplier, M, beamWidth, HnswGraphBuilder.randSeed);
+    }
+
+    HnswGraph initializerGraph = ((HnswGraphProvider) initReader).getGraph(fieldInfo.name);
+    BitSet initializedNodes = new FixedBitSet(mergeState.segmentInfo.maxDoc() + 1);

Review Comment:
   We probably don't need this much number of bits, but we can postpone it to another PR I think? Could you create an issue for this after the PR is merged? (And maybe attach "Good First Issue" label?)



##########
lucene/core/src/java/org/apache/lucene/util/hnsw/InitializedHnswGraphBuilder.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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 static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+import java.io.IOException;
+import org.apache.lucene.util.BitSet;
+
+/**
+ * This creates a graph builder that is initialized with the provided HnswGraph. This is useful for
+ * merging HnswGraphs from multiple segments.
+ *
+ * @lucene.experimental
+ */
+public final class InitializedHnswGraphBuilder extends HnswGraphBuilder {
+
+  /**
+   * Create a new HnswGraphBuilder that is initialized with the provided HnswGraph.
+   *
+   * @param scorerSupplier the scorer to use for vectors
+   * @param M the number of connections to keep per node
+   * @param beamWidth the number of nodes to explore in the search
+   * @param seed the seed for the random number generator
+   * @param initializerGraph the graph to initialize the new graph builder
+   * @param newOrdMap a mapping from the old node ordinal to the new node ordinal
+   * @param initializedNodes a bitset of nodes that are already initialized in the initializerGraph
+   * @return a new HnswGraphBuilder that is initialized with the provided HnswGraph
+   * @throws IOException when reading the graph fails
+   */
+  public static InitializedHnswGraphBuilder fromGraph(
+      RandomVectorScorerSupplier scorerSupplier,
+      int M,
+      int beamWidth,
+      long seed,
+      HnswGraph initializerGraph,
+      int[] newOrdMap,
+      BitSet initializedNodes)
+      throws IOException {
+    OnHeapHnswGraph hnsw = new OnHeapHnswGraph(M);
+    for (int level = 0; level < initializerGraph.numLevels(); level++) {

Review Comment:
   FYI I have just merged #12651 so insertion sequence here might need to be changed



##########
lucene/core/src/java/org/apache/lucene/util/hnsw/IncrementalHnswGraphMerger.java:
##########
@@ -0,0 +1,197 @@
+/*
+ * 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 static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+import java.io.IOException;
+import java.util.Map;
+import org.apache.lucene.codecs.HnswGraphProvider;
+import org.apache.lucene.codecs.KnnVectorsReader;
+import org.apache.lucene.codecs.KnnVectorsWriter;
+import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
+import org.apache.lucene.index.ByteVectorValues;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FloatVectorValues;
+import org.apache.lucene.index.MergeState;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.BitSet;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.CollectionUtil;
+import org.apache.lucene.util.FixedBitSet;
+
+/**
+ * This selects the biggest Hnsw graph from the provided merge state and initializes a new
+ * HnswGraphBuilder with that graph as a starting point.
+ *
+ * @lucene.experimental
+ */
+public class IncrementalHnswGraphMerger {
+
+  private KnnVectorsReader initReader;
+  private MergeState.DocMap initDocMap;
+  private int initGraphSize;
+  private final FieldInfo fieldInfo;
+  private final RandomVectorScorerSupplier scorerSupplier;
+  private final int M;
+  private final int beamWidth;
+
+  /**
+   * @param fieldInfo FieldInfo for the field being merged
+   */
+  public IncrementalHnswGraphMerger(
+      FieldInfo fieldInfo, RandomVectorScorerSupplier scorerSupplier, int M, int beamWidth) {
+    this.fieldInfo = fieldInfo;
+    this.scorerSupplier = scorerSupplier;
+    this.M = M;
+    this.beamWidth = beamWidth;
+  }
+
+  /**
+   * Adds a reader to the graph merger if it meets the following criteria: 1. Does not contain any
+   * deleted docs 2. Is a HnswGraphProvider/PerFieldKnnVectorReader 3. Has the most docs of any
+   * previous reader that met the above criteria
+   *
+   * @param reader KnnVectorsReader to add to the merger
+   * @param docMap MergeState.DocMap for the reader
+   * @param liveDocs Bits representing live docs, can be null
+   * @return this
+   * @throws IOException If an error occurs while reading from the merge state
+   */
+  public IncrementalHnswGraphMerger addReader(
+      KnnVectorsReader reader, MergeState.DocMap docMap, Bits liveDocs) throws IOException {
+    KnnVectorsReader currKnnVectorsReader = reader;
+    if (reader instanceof PerFieldKnnVectorsFormat.FieldsReader candidateReader) {
+      currKnnVectorsReader = candidateReader.getFieldReader(fieldInfo.name);
+    }
+
+    if (!(currKnnVectorsReader instanceof HnswGraphProvider) || !allMatch(liveDocs)) {
+      return this;
+    }
+
+    int candidateVectorCount = 0;
+    switch (fieldInfo.getVectorEncoding()) {
+      case BYTE -> {
+        ByteVectorValues byteVectorValues =
+            currKnnVectorsReader.getByteVectorValues(fieldInfo.name);
+        if (byteVectorValues == null) {
+          return this;
+        }
+        candidateVectorCount = byteVectorValues.size();
+      }
+      case FLOAT32 -> {
+        FloatVectorValues vectorValues = currKnnVectorsReader.getFloatVectorValues(fieldInfo.name);
+        if (vectorValues == null) {
+          return this;
+        }
+        candidateVectorCount = vectorValues.size();
+      }
+    }
+    if (candidateVectorCount > initGraphSize) {
+      initReader = currKnnVectorsReader;
+      initDocMap = docMap;
+      initGraphSize = candidateVectorCount;
+    }
+    return this;
+  }
+
+  /**
+   * Builds a new HnswGraphBuilder using the biggest graph from the merge state as a starting point.
+   * If no valid readers were added to the merge state, a new graph is created.
+   *
+   * @param mergeState MergeState for the merge
+   * @return HnswGraphBuilder
+   * @throws IOException If an error occurs while reading from the merge state
+   */
+  public HnswGraphBuilder createBuilder(MergeState mergeState) throws IOException {
+    if (initReader == null) {
+      return HnswGraphBuilder.create(scorerSupplier, M, beamWidth, HnswGraphBuilder.randSeed);
+    }
+
+    HnswGraph initializerGraph = ((HnswGraphProvider) initReader).getGraph(fieldInfo.name);
+    BitSet initializedNodes = new FixedBitSet(mergeState.segmentInfo.maxDoc() + 1);
+    int[] ordBaseline = getNewOrdOffset(mergeState, initializedNodes);

Review Comment:
   `ordBaseline` -> `newOrdMap`?



-- 
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-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Extract the hnsw graph merging from being part of the vector writer [lucene]

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12657:
URL: https://github.com/apache/lucene/pull/12657#discussion_r1362457390


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/IncrementalHnswGraphMerger.java:
##########
@@ -0,0 +1,197 @@
+/*
+ * 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 static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+import java.io.IOException;
+import java.util.Map;
+import org.apache.lucene.codecs.HnswGraphProvider;
+import org.apache.lucene.codecs.KnnVectorsReader;
+import org.apache.lucene.codecs.KnnVectorsWriter;
+import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
+import org.apache.lucene.index.ByteVectorValues;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FloatVectorValues;
+import org.apache.lucene.index.MergeState;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.BitSet;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.CollectionUtil;
+import org.apache.lucene.util.FixedBitSet;
+
+/**
+ * This selects the biggest Hnsw graph from the provided merge state and initializes a new
+ * HnswGraphBuilder with that graph as a starting point.
+ *
+ * @lucene.experimental
+ */
+public class IncrementalHnswGraphMerger {
+
+  private KnnVectorsReader initReader;
+  private MergeState.DocMap initDocMap;
+  private int initGraphSize;
+  private final FieldInfo fieldInfo;
+  private final RandomVectorScorerSupplier scorerSupplier;
+  private final int M;
+  private final int beamWidth;
+
+  /**
+   * @param fieldInfo FieldInfo for the field being merged
+   */
+  public IncrementalHnswGraphMerger(
+      FieldInfo fieldInfo, RandomVectorScorerSupplier scorerSupplier, int M, int beamWidth) {
+    this.fieldInfo = fieldInfo;
+    this.scorerSupplier = scorerSupplier;
+    this.M = M;
+    this.beamWidth = beamWidth;
+  }
+
+  /**
+   * Adds a reader to the graph merger if it meets the following criteria: 1. Does not contain any
+   * deleted docs 2. Is a HnswGraphProvider/PerFieldKnnVectorReader 3. Has the most docs of any
+   * previous reader that met the above criteria
+   *
+   * @param reader KnnVectorsReader to add to the merger
+   * @param docMap MergeState.DocMap for the reader
+   * @param liveDocs Bits representing live docs, can be null
+   * @return this
+   * @throws IOException If an error occurs while reading from the merge state
+   */
+  public IncrementalHnswGraphMerger addReader(
+      KnnVectorsReader reader, MergeState.DocMap docMap, Bits liveDocs) throws IOException {
+    KnnVectorsReader currKnnVectorsReader = reader;
+    if (reader instanceof PerFieldKnnVectorsFormat.FieldsReader candidateReader) {
+      currKnnVectorsReader = candidateReader.getFieldReader(fieldInfo.name);
+    }
+
+    if (!(currKnnVectorsReader instanceof HnswGraphProvider) || !allMatch(liveDocs)) {
+      return this;
+    }
+
+    int candidateVectorCount = 0;
+    switch (fieldInfo.getVectorEncoding()) {
+      case BYTE -> {
+        ByteVectorValues byteVectorValues =
+            currKnnVectorsReader.getByteVectorValues(fieldInfo.name);
+        if (byteVectorValues == null) {
+          return this;
+        }
+        candidateVectorCount = byteVectorValues.size();
+      }
+      case FLOAT32 -> {
+        FloatVectorValues vectorValues = currKnnVectorsReader.getFloatVectorValues(fieldInfo.name);
+        if (vectorValues == null) {
+          return this;
+        }
+        candidateVectorCount = vectorValues.size();
+      }
+    }
+    if (candidateVectorCount > initGraphSize) {
+      initReader = currKnnVectorsReader;
+      initDocMap = docMap;
+      initGraphSize = candidateVectorCount;
+    }
+    return this;
+  }
+
+  /**
+   * Builds a new HnswGraphBuilder using the biggest graph from the merge state as a starting point.
+   * If no valid readers were added to the merge state, a new graph is created.
+   *
+   * @param mergeState MergeState for the merge
+   * @return HnswGraphBuilder
+   * @throws IOException If an error occurs while reading from the merge state
+   */
+  public HnswGraphBuilder createBuilder(MergeState mergeState) throws IOException {
+    if (initReader == null) {
+      return HnswGraphBuilder.create(scorerSupplier, M, beamWidth, HnswGraphBuilder.randSeed);
+    }
+
+    HnswGraph initializerGraph = ((HnswGraphProvider) initReader).getGraph(fieldInfo.name);
+    BitSet initializedNodes = new FixedBitSet(mergeState.segmentInfo.maxDoc() + 1);

Review Comment:
   YES! I refactored and only used `mergedVectorReader.cost() + 1`, so we only size to the number of vectors, not the number of docs, which greatly reduces the number of bits in the sparse case.



-- 
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-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Extract the hnsw graph merging from being part of the vector writer [lucene]

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12657:
URL: https://github.com/apache/lucene/pull/12657#discussion_r1356732427


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/IncrementalHnswGraphMerger.java:
##########
@@ -0,0 +1,226 @@
+/*
+ * 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.codecs.lucene95;
+
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.lucene.codecs.HnswGraphProvider;
+import org.apache.lucene.codecs.KnnVectorsReader;
+import org.apache.lucene.codecs.KnnVectorsWriter;
+import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
+import org.apache.lucene.index.ByteVectorValues;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FloatVectorValues;
+import org.apache.lucene.index.MergeState;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.hnsw.HnswGraph;
+import org.apache.lucene.util.hnsw.HnswGraphBuilder;
+import org.apache.lucene.util.hnsw.InitializedHnswGraphBuilder;
+import org.apache.lucene.util.hnsw.RandomVectorScorerSupplier;
+
+/**
+ * This selects the biggest Hnsw graph from the provided merge state and initializes a new
+ * HnswGraphBuilder with that graph as a starting point.
+ */
+public class IncrementalHnswGraphMerger {
+
+  private final MergeState mergeState;
+  private final FieldInfo fieldInfo;
+
+  /**
+   * @param mergeState MergeState containing the readers to merge
+   * @param fieldInfo FieldInfo for the field being merged
+   */
+  public IncrementalHnswGraphMerger(MergeState mergeState, FieldInfo fieldInfo) {
+    this.mergeState = mergeState;
+    this.fieldInfo = fieldInfo;
+  }
+
+  /**
+   * Selects the biggest Hnsw graph from the provided merge state and initializes a new
+   * HnswGraphBuilder with that graph as a starting point.
+   *
+   * @param scorerSupplier ScorerSupplier to use for scoring vectors
+   * @param M The number of connections to allow per node
+   * @param beamWidth The number of nodes to consider when searching for the nearest neighbor
+   * @return HnswGraphBuilder initialized with the biggest graph from the merge state
+   * @throws IOException If an error occurs while reading from the merge state
+   */
+  public HnswGraphBuilder build(RandomVectorScorerSupplier scorerSupplier, int M, int beamWidth)
+      throws IOException {
+    // Find the KnnVectorReader with the most docs that meets the following criteria:

Review Comment:
   @zhaih I was also thinking of a "true" builder interface that is actually extensible. I will have another crack at it today.
   
   The tricky part is that we are not 100% sure what another merger kind would look like. 
   
   Something else that would be good could be
   
   `HnswGraphMerger#addGraph` and `HnswGraphMerger#merge`. I will iterate some more to see what I can come up with.



-- 
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-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Extract the hnsw graph merging from being part of the vector writer [lucene]

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12657:
URL: https://github.com/apache/lucene/pull/12657#discussion_r1356734118


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/IncrementalHnswGraphMerger.java:
##########
@@ -0,0 +1,226 @@
+/*
+ * 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.codecs.lucene95;
+
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.lucene.codecs.HnswGraphProvider;
+import org.apache.lucene.codecs.KnnVectorsReader;
+import org.apache.lucene.codecs.KnnVectorsWriter;
+import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
+import org.apache.lucene.index.ByteVectorValues;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FloatVectorValues;
+import org.apache.lucene.index.MergeState;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.hnsw.HnswGraph;
+import org.apache.lucene.util.hnsw.HnswGraphBuilder;
+import org.apache.lucene.util.hnsw.InitializedHnswGraphBuilder;
+import org.apache.lucene.util.hnsw.RandomVectorScorerSupplier;
+
+/**
+ * This selects the biggest Hnsw graph from the provided merge state and initializes a new
+ * HnswGraphBuilder with that graph as a starting point.
+ */
+public class IncrementalHnswGraphMerger {
+
+  private final MergeState mergeState;
+  private final FieldInfo fieldInfo;
+
+  /**
+   * @param mergeState MergeState containing the readers to merge
+   * @param fieldInfo FieldInfo for the field being merged
+   */
+  public IncrementalHnswGraphMerger(MergeState mergeState, FieldInfo fieldInfo) {
+    this.mergeState = mergeState;
+    this.fieldInfo = fieldInfo;
+  }
+
+  /**
+   * Selects the biggest Hnsw graph from the provided merge state and initializes a new
+   * HnswGraphBuilder with that graph as a starting point.
+   *
+   * @param scorerSupplier ScorerSupplier to use for scoring vectors
+   * @param M The number of connections to allow per node
+   * @param beamWidth The number of nodes to consider when searching for the nearest neighbor
+   * @return HnswGraphBuilder initialized with the biggest graph from the merge state
+   * @throws IOException If an error occurs while reading from the merge state
+   */
+  public HnswGraphBuilder build(RandomVectorScorerSupplier scorerSupplier, int M, int beamWidth)

Review Comment:
   @zhaih yes, I am thinking of a concurrent merger & a more advanced merger (not one that just initializes from the largest graph, but instead actually merges graphs...)



-- 
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-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Extract the hnsw graph merging from being part of the vector writer [lucene]

Posted by "jmazanec15 (via GitHub)" <gi...@apache.org>.
jmazanec15 commented on code in PR #12657:
URL: https://github.com/apache/lucene/pull/12657#discussion_r1357750648


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/IncrementalHnswGraphMerger.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.codecs.lucene95;
+
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.lucene.codecs.HnswGraphProvider;
+import org.apache.lucene.codecs.KnnVectorsReader;
+import org.apache.lucene.codecs.KnnVectorsWriter;
+import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
+import org.apache.lucene.index.ByteVectorValues;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FloatVectorValues;
+import org.apache.lucene.index.MergeState;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.hnsw.HnswGraph;
+import org.apache.lucene.util.hnsw.HnswGraphBuilder;
+import org.apache.lucene.util.hnsw.InitializedHnswGraphBuilder;
+import org.apache.lucene.util.hnsw.RandomVectorScorerSupplier;
+
+/**
+ * This selects the biggest Hnsw graph from the provided merge state and initializes a new
+ * HnswGraphBuilder with that graph as a starting point.
+ */
+public class IncrementalHnswGraphMerger {
+
+  private KnnVectorsReader curReader;
+  private MergeState.DocMap curDocMap;
+  private int curGraphSize;
+  private final FieldInfo fieldInfo;
+  private final RandomVectorScorerSupplier scorerSupplier;
+  private final int M;
+  private final int beamWidth;
+
+  /**
+   * @param fieldInfo FieldInfo for the field being merged
+   */
+  public IncrementalHnswGraphMerger(
+      FieldInfo fieldInfo, RandomVectorScorerSupplier scorerSupplier, int M, int beamWidth) {
+    this.fieldInfo = fieldInfo;
+    this.scorerSupplier = scorerSupplier;
+    this.M = M;
+    this.beamWidth = beamWidth;
+  }
+
+  /**
+   * Adds a reader to the graph merger if it meets the following criteria: 1. Does not contain any
+   * deleted docs 2. Is a HnswGraphProvider/PerFieldKnnVectorReader 3. Has the most docs of any
+   * previous reader that met the above criteria
+   *
+   * @param reader KnnVectorsReader to add to the merger
+   * @param docMap MergeState.DocMap for the reader
+   * @param liveDocs Bits representing live docs, can be null
+   * @return this
+   * @throws IOException If an error occurs while reading from the merge state
+   */
+  IncrementalHnswGraphMerger addReader(
+      KnnVectorsReader reader, MergeState.DocMap docMap, Bits liveDocs) throws IOException {
+    KnnVectorsReader currKnnVectorsReader = reader;
+    if (reader instanceof PerFieldKnnVectorsFormat.FieldsReader candidateReader) {
+      currKnnVectorsReader = candidateReader.getFieldReader(fieldInfo.name);
+    }
+
+    if (!(currKnnVectorsReader instanceof HnswGraphProvider) || !allMatch(liveDocs)) {
+      return this;
+    }
+
+    int candidateVectorCount = 0;
+    switch (fieldInfo.getVectorEncoding()) {
+      case BYTE -> {
+        ByteVectorValues byteVectorValues =
+            currKnnVectorsReader.getByteVectorValues(fieldInfo.name);
+        if (byteVectorValues == null) {
+          return this;
+        }
+        candidateVectorCount = byteVectorValues.size();
+      }
+      case FLOAT32 -> {
+        FloatVectorValues vectorValues = currKnnVectorsReader.getFloatVectorValues(fieldInfo.name);
+        if (vectorValues == null) {
+          return this;
+        }
+        candidateVectorCount = vectorValues.size();
+      }
+    }
+    if (candidateVectorCount > curGraphSize) {
+      curReader = currKnnVectorsReader;
+      curDocMap = docMap;
+      curGraphSize = candidateVectorCount;
+    }
+    return this;
+  }
+
+  /**
+   * Builds a new HnswGraphBuilder using the biggest graph from the merge state as a starting point.
+   * If no valid readers were added to the merge state, a new graph is created.
+   *
+   * @param mergeState MergeState for the merge
+   * @return HnswGraphBuilder
+   * @throws IOException If an error occurs while reading from the merge state
+   */
+  public HnswGraphBuilder createBuilder(MergeState mergeState) throws IOException {
+    if (curReader == null) {
+      return HnswGraphBuilder.create(scorerSupplier, M, beamWidth, HnswGraphBuilder.randSeed);
+    }
+
+    HnswGraph initializerGraph = ((HnswGraphProvider) curReader).getGraph(fieldInfo.name);
+    Map<Integer, Integer> ordinalMapper = getOldToNewOrdinalMap(mergeState);
+    return new InitializedHnswGraphBuilder(
+        scorerSupplier, M, beamWidth, HnswGraphBuilder.randSeed, initializerGraph, ordinalMapper);
+  }
+
+  private Map<Integer, Integer> getOldToNewOrdinalMap(MergeState mergeState) throws IOException {
+
+    DocIdSetIterator initializerIterator = null;
+    switch (fieldInfo.getVectorEncoding()) {
+      case BYTE -> initializerIterator = curReader.getByteVectorValues(fieldInfo.name);
+      case FLOAT32 -> initializerIterator = curReader.getFloatVectorValues(fieldInfo.name);
+    }
+
+    Map<Integer, Integer> newIdToOldOrdinal = new HashMap<>();
+    int oldOrd = 0;
+    int maxNewDocID = -1;
+    for (int oldId = initializerIterator.nextDoc();
+        oldId != NO_MORE_DOCS;
+        oldId = initializerIterator.nextDoc()) {
+      if (isCurrentVectorNull(initializerIterator)) {
+        continue;
+      }

Review Comment:
   I am not sure I ran into it explicitly. I think my concern was in the case of sparse vector values (which I am not sure existed when I made this change) where a particular document does not have the vector field set.



-- 
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-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Extract the hnsw graph merging from being part of the vector writer [lucene]

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12657:
URL: https://github.com/apache/lucene/pull/12657#discussion_r1358208212


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/InitializedHnswGraphBuilder.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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 static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.Set;
+import org.apache.lucene.util.CollectionUtil;
+
+/**
+ * This creates a graph builder that is initialized with the provided HnswGraph. This is useful for
+ * merging HnswGraphs from multiple segments.
+ *
+ * @lucene.internal
+ */
+public final class InitializedHnswGraphBuilder extends HnswGraphBuilder {

Review Comment:
   @zhaih I think we can do that. What I also want to do is remove all this ` Map<Integer, Integer> oldToNewOrdinalMap`. The caller should handle all that, not the constructor, especially since there is probably a way faster and simpler way to do it.



-- 
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-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Extract the hnsw graph merging from being part of the vector writer [lucene]

Posted by "zhaih (via GitHub)" <gi...@apache.org>.
zhaih commented on code in PR #12657:
URL: https://github.com/apache/lucene/pull/12657#discussion_r1359590587


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/IncrementalHnswGraphMerger.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.codecs.lucene95;
+
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+import java.io.IOException;
+import org.apache.lucene.codecs.HnswGraphProvider;
+import org.apache.lucene.codecs.KnnVectorsReader;
+import org.apache.lucene.codecs.KnnVectorsWriter;
+import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
+import org.apache.lucene.index.ByteVectorValues;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FloatVectorValues;
+import org.apache.lucene.index.MergeState;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.BitSet;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.hnsw.HnswGraph;
+import org.apache.lucene.util.hnsw.HnswGraphBuilder;
+import org.apache.lucene.util.hnsw.InitializedHnswGraphBuilder;
+import org.apache.lucene.util.hnsw.RandomVectorScorerSupplier;
+
+/**
+ * This selects the biggest Hnsw graph from the provided merge state and initializes a new
+ * HnswGraphBuilder with that graph as a starting point.
+ */
+public class IncrementalHnswGraphMerger {
+
+  private KnnVectorsReader initReader;
+  private MergeState.DocMap initDocMap;
+  private int initGraphSize;
+  private final FieldInfo fieldInfo;
+  private final RandomVectorScorerSupplier scorerSupplier;
+  private final int M;
+  private final int beamWidth;
+
+  /**
+   * @param fieldInfo FieldInfo for the field being merged
+   */
+  public IncrementalHnswGraphMerger(
+      FieldInfo fieldInfo, RandomVectorScorerSupplier scorerSupplier, int M, int beamWidth) {
+    this.fieldInfo = fieldInfo;
+    this.scorerSupplier = scorerSupplier;
+    this.M = M;
+    this.beamWidth = beamWidth;
+  }
+
+  /**
+   * Adds a reader to the graph merger if it meets the following criteria: 1. Does not contain any
+   * deleted docs 2. Is a HnswGraphProvider/PerFieldKnnVectorReader 3. Has the most docs of any
+   * previous reader that met the above criteria
+   *
+   * @param reader KnnVectorsReader to add to the merger
+   * @param docMap MergeState.DocMap for the reader
+   * @param liveDocs Bits representing live docs, can be null
+   * @return this
+   * @throws IOException If an error occurs while reading from the merge state
+   */
+  IncrementalHnswGraphMerger addReader(
+      KnnVectorsReader reader, MergeState.DocMap docMap, Bits liveDocs) throws IOException {
+    KnnVectorsReader currKnnVectorsReader = reader;
+    if (reader instanceof PerFieldKnnVectorsFormat.FieldsReader candidateReader) {
+      currKnnVectorsReader = candidateReader.getFieldReader(fieldInfo.name);
+    }
+
+    if (!(currKnnVectorsReader instanceof HnswGraphProvider) || !allMatch(liveDocs)) {
+      return this;
+    }
+
+    int candidateVectorCount = 0;
+    switch (fieldInfo.getVectorEncoding()) {
+      case BYTE -> {
+        ByteVectorValues byteVectorValues =
+            currKnnVectorsReader.getByteVectorValues(fieldInfo.name);
+        if (byteVectorValues == null) {
+          return this;
+        }
+        candidateVectorCount = byteVectorValues.size();
+      }
+      case FLOAT32 -> {
+        FloatVectorValues vectorValues = currKnnVectorsReader.getFloatVectorValues(fieldInfo.name);
+        if (vectorValues == null) {
+          return this;
+        }
+        candidateVectorCount = vectorValues.size();
+      }
+    }
+    if (candidateVectorCount > initGraphSize) {
+      initReader = currKnnVectorsReader;
+      initDocMap = docMap;
+      initGraphSize = candidateVectorCount;
+    }
+    return this;
+  }
+
+  /**
+   * Builds a new HnswGraphBuilder using the biggest graph from the merge state as a starting point.
+   * If no valid readers were added to the merge state, a new graph is created.
+   *
+   * @param mergeState MergeState for the merge
+   * @return HnswGraphBuilder
+   * @throws IOException If an error occurs while reading from the merge state
+   */
+  public HnswGraphBuilder createBuilder(MergeState mergeState) throws IOException {
+    if (initReader == null) {
+      return HnswGraphBuilder.create(scorerSupplier, M, beamWidth, HnswGraphBuilder.randSeed);
+    }
+
+    HnswGraph initializerGraph = ((HnswGraphProvider) initReader).getGraph(fieldInfo.name);
+    int ordBaseline = getNewOrdOffset(mergeState);
+    BitSet initializedNodes =
+        BitSet.of(
+            DocIdSetIterator.range(ordBaseline, initGraphSize + ordBaseline),
+            mergeState.segmentInfo.maxDoc() + 1);
+    return InitializedHnswGraphBuilder.fromGraph(
+        scorerSupplier,
+        M,
+        beamWidth,
+        HnswGraphBuilder.randSeed,
+        initializerGraph,
+        ordBaseline,
+        initializedNodes);
+  }
+
+  private int getNewOrdOffset(MergeState mergeState) throws IOException {
+    final DocIdSetIterator initializerIterator =
+        switch (fieldInfo.getVectorEncoding()) {
+          case BYTE -> initReader.getByteVectorValues(fieldInfo.name);
+          case FLOAT32 -> initReader.getFloatVectorValues(fieldInfo.name);
+        };
+
+    // minDoc is just the first doc in the main segment containing our graph
+    // we don't need to worry about deleted documents as deletions are not allowed when selecting
+    // the best graph
+    int minDoc = initializerIterator.nextDoc();
+    if (minDoc == NO_MORE_DOCS) {
+      return -1;
+    }
+    minDoc = initDocMap.get(minDoc);
+
+    DocIdSetIterator vectorIterator = null;
+    switch (fieldInfo.getVectorEncoding()) {
+      case BYTE -> vectorIterator =
+          KnnVectorsWriter.MergedVectorValues.mergeByteVectorValues(fieldInfo, mergeState);
+      case FLOAT32 -> vectorIterator =
+          KnnVectorsWriter.MergedVectorValues.mergeFloatVectorValues(fieldInfo, mergeState);
+    }
+
+    // Since there are no deleted documents in our chosen segment, we can assume that the ordinals
+    // are unchanged
+    // meaning we only need to know what ordinal offset to select and apply it

Review Comment:
   This can be wrong when index sort is configured? E.g. if we have 2 segments each have 2 docs
   ```
   seg0
       doc0: rank=0
       doc1: rank=2
   seg1
       doc0: rank=1
       doc1: rank=3
   ```
   So after merge those doc will be in an interleaved order but not simply old order + base?



-- 
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-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Extract the hnsw graph merging from being part of the vector writer [lucene]

Posted by "msokolov (via GitHub)" <gi...@apache.org>.
msokolov commented on code in PR #12657:
URL: https://github.com/apache/lucene/pull/12657#discussion_r1359929577


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/InitializedHnswGraphBuilder.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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 static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.Set;
+import org.apache.lucene.util.CollectionUtil;
+
+/**
+ * This creates a graph builder that is initialized with the provided HnswGraph. This is useful for
+ * merging HnswGraphs from multiple segments.
+ *
+ * @lucene.internal
+ */
+public final class InitializedHnswGraphBuilder extends HnswGraphBuilder {

Review Comment:
   I didn't follow this closely, but do we really have a map like that? If so, let's at least switch to IntIntHashMap to avoid boxing (if we're unable to remove it)



-- 
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-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Extract the hnsw graph merging from being part of the vector writer [lucene]

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on PR #12657:
URL: https://github.com/apache/lucene/pull/12657#issuecomment-1760088558

   @zhaih I updated the API a bit. This is more like I was thinking. Having a builder that accepts readers, doc maps, etc. And then can build with the final merge state. 


-- 
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-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Extract the hnsw graph merging from being part of the vector writer [lucene]

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12657:
URL: https://github.com/apache/lucene/pull/12657#discussion_r1357376455


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/IncrementalHnswGraphMerger.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.codecs.lucene95;
+
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.lucene.codecs.HnswGraphProvider;
+import org.apache.lucene.codecs.KnnVectorsReader;
+import org.apache.lucene.codecs.KnnVectorsWriter;
+import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
+import org.apache.lucene.index.ByteVectorValues;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FloatVectorValues;
+import org.apache.lucene.index.MergeState;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.hnsw.HnswGraph;
+import org.apache.lucene.util.hnsw.HnswGraphBuilder;
+import org.apache.lucene.util.hnsw.InitializedHnswGraphBuilder;
+import org.apache.lucene.util.hnsw.RandomVectorScorerSupplier;
+
+/**
+ * This selects the biggest Hnsw graph from the provided merge state and initializes a new
+ * HnswGraphBuilder with that graph as a starting point.
+ */
+public class IncrementalHnswGraphMerger {
+
+  private KnnVectorsReader curReader;
+  private MergeState.DocMap curDocMap;
+  private int curGraphSize;
+  private final FieldInfo fieldInfo;
+  private final RandomVectorScorerSupplier scorerSupplier;
+  private final int M;
+  private final int beamWidth;
+
+  /**
+   * @param fieldInfo FieldInfo for the field being merged
+   */
+  public IncrementalHnswGraphMerger(
+      FieldInfo fieldInfo, RandomVectorScorerSupplier scorerSupplier, int M, int beamWidth) {
+    this.fieldInfo = fieldInfo;
+    this.scorerSupplier = scorerSupplier;
+    this.M = M;
+    this.beamWidth = beamWidth;
+  }
+
+  /**
+   * Adds a reader to the graph merger if it meets the following criteria: 1. Does not contain any
+   * deleted docs 2. Is a HnswGraphProvider/PerFieldKnnVectorReader 3. Has the most docs of any
+   * previous reader that met the above criteria
+   *
+   * @param reader KnnVectorsReader to add to the merger
+   * @param docMap MergeState.DocMap for the reader
+   * @param liveDocs Bits representing live docs, can be null
+   * @return this
+   * @throws IOException If an error occurs while reading from the merge state
+   */
+  IncrementalHnswGraphMerger addReader(
+      KnnVectorsReader reader, MergeState.DocMap docMap, Bits liveDocs) throws IOException {
+    KnnVectorsReader currKnnVectorsReader = reader;
+    if (reader instanceof PerFieldKnnVectorsFormat.FieldsReader candidateReader) {
+      currKnnVectorsReader = candidateReader.getFieldReader(fieldInfo.name);
+    }
+
+    if (!(currKnnVectorsReader instanceof HnswGraphProvider) || !allMatch(liveDocs)) {
+      return this;
+    }
+
+    int candidateVectorCount = 0;
+    switch (fieldInfo.getVectorEncoding()) {
+      case BYTE -> {
+        ByteVectorValues byteVectorValues =
+            currKnnVectorsReader.getByteVectorValues(fieldInfo.name);
+        if (byteVectorValues == null) {
+          return this;
+        }
+        candidateVectorCount = byteVectorValues.size();
+      }
+      case FLOAT32 -> {
+        FloatVectorValues vectorValues = currKnnVectorsReader.getFloatVectorValues(fieldInfo.name);
+        if (vectorValues == null) {
+          return this;
+        }
+        candidateVectorCount = vectorValues.size();
+      }
+    }
+    if (candidateVectorCount > curGraphSize) {
+      curReader = currKnnVectorsReader;
+      curDocMap = docMap;
+      curGraphSize = candidateVectorCount;
+    }
+    return this;
+  }
+
+  /**
+   * Builds a new HnswGraphBuilder using the biggest graph from the merge state as a starting point.
+   * If no valid readers were added to the merge state, a new graph is created.
+   *
+   * @param mergeState MergeState for the merge
+   * @return HnswGraphBuilder
+   * @throws IOException If an error occurs while reading from the merge state
+   */
+  public HnswGraphBuilder createBuilder(MergeState mergeState) throws IOException {
+    if (curReader == null) {
+      return HnswGraphBuilder.create(scorerSupplier, M, beamWidth, HnswGraphBuilder.randSeed);
+    }
+
+    HnswGraph initializerGraph = ((HnswGraphProvider) curReader).getGraph(fieldInfo.name);
+    Map<Integer, Integer> ordinalMapper = getOldToNewOrdinalMap(mergeState);
+    return new InitializedHnswGraphBuilder(
+        scorerSupplier, M, beamWidth, HnswGraphBuilder.randSeed, initializerGraph, ordinalMapper);
+  }
+
+  private Map<Integer, Integer> getOldToNewOrdinalMap(MergeState mergeState) throws IOException {
+
+    DocIdSetIterator initializerIterator = null;
+    switch (fieldInfo.getVectorEncoding()) {
+      case BYTE -> initializerIterator = curReader.getByteVectorValues(fieldInfo.name);
+      case FLOAT32 -> initializerIterator = curReader.getFloatVectorValues(fieldInfo.name);
+    }
+
+    Map<Integer, Integer> newIdToOldOrdinal = new HashMap<>();
+    int oldOrd = 0;
+    int maxNewDocID = -1;
+    for (int oldId = initializerIterator.nextDoc();
+        oldId != NO_MORE_DOCS;
+        oldId = initializerIterator.nextDoc()) {
+      if (isCurrentVectorNull(initializerIterator)) {
+        continue;
+      }

Review Comment:
   While digging into this refactor more, it seems weird to me that we check if a vector is null. I don't think any vector reader will return a `null` vector.
   
   @jmazanec15 did you run into `null` vectors before? I am trying to figure out the scenario where we would run into a `null` vector.



-- 
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-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Extract the hnsw graph merging from being part of the vector writer [lucene]

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12657:
URL: https://github.com/apache/lucene/pull/12657#discussion_r1361111233


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/IncrementalHnswGraphMerger.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.codecs.lucene95;
+
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+import java.io.IOException;
+import org.apache.lucene.codecs.HnswGraphProvider;
+import org.apache.lucene.codecs.KnnVectorsReader;
+import org.apache.lucene.codecs.KnnVectorsWriter;
+import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
+import org.apache.lucene.index.ByteVectorValues;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FloatVectorValues;
+import org.apache.lucene.index.MergeState;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.BitSet;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.hnsw.HnswGraph;
+import org.apache.lucene.util.hnsw.HnswGraphBuilder;
+import org.apache.lucene.util.hnsw.InitializedHnswGraphBuilder;
+import org.apache.lucene.util.hnsw.RandomVectorScorerSupplier;
+
+/**
+ * This selects the biggest Hnsw graph from the provided merge state and initializes a new
+ * HnswGraphBuilder with that graph as a starting point.
+ */
+public class IncrementalHnswGraphMerger {

Review Comment:
   ++
   
   This would make sense for a ConcurrentMerger & "BetterFutureHnswGraphMerger" or whatever it will be.



-- 
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-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Extract the hnsw graph merging from being part of the vector writer [lucene]

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12657:
URL: https://github.com/apache/lucene/pull/12657#discussion_r1358794152


##########
lucene/core/src/test/org/apache/lucene/util/hnsw/HnswGraphTestCase.java:
##########
@@ -1147,15 +1139,15 @@ void assertVectorsEqual(AbstractMockVectorValues<T> u, AbstractMockVectorValues<
 
   static float[][] createRandomFloatVectors(int size, int dimension, Random random) {
     float[][] vectors = new float[size][];
-    for (int offset = 0; offset < size; offset += random.nextInt(3) + 1) {
+    for (int offset = 0; offset < size; offset++) {

Review Comment:
   I think this was to simulate sparse vectors. But the sparse vector iterator never returns `null` for the vectors. Instead it just skips to the next non-null vector from what I can tell.



-- 
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-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Extract the hnsw graph merging from being part of the vector writer [lucene]

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12657:
URL: https://github.com/apache/lucene/pull/12657#discussion_r1360502250


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/InitializedHnswGraphBuilder.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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 static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.Set;
+import org.apache.lucene.util.CollectionUtil;
+
+/**
+ * This creates a graph builder that is initialized with the provided HnswGraph. This is useful for
+ * merging HnswGraphs from multiple segments.
+ *
+ * @lucene.internal
+ */
+public final class InitializedHnswGraphBuilder extends HnswGraphBuilder {

Review Comment:
   @msokolov I think we can just have `int[]`, where `int[oldOrd]=newOrd` since old vector ordinals are continuous.



-- 
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-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Extract the hnsw graph merging from being part of the vector writer [lucene]

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12657:
URL: https://github.com/apache/lucene/pull/12657#discussion_r1361112199


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/IncrementalHnswGraphMerger.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.codecs.lucene95;
+
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+import java.io.IOException;
+import org.apache.lucene.codecs.HnswGraphProvider;
+import org.apache.lucene.codecs.KnnVectorsReader;
+import org.apache.lucene.codecs.KnnVectorsWriter;
+import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
+import org.apache.lucene.index.ByteVectorValues;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FloatVectorValues;
+import org.apache.lucene.index.MergeState;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.BitSet;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.hnsw.HnswGraph;
+import org.apache.lucene.util.hnsw.HnswGraphBuilder;
+import org.apache.lucene.util.hnsw.InitializedHnswGraphBuilder;
+import org.apache.lucene.util.hnsw.RandomVectorScorerSupplier;
+
+/**
+ * This selects the biggest Hnsw graph from the provided merge state and initializes a new
+ * HnswGraphBuilder with that graph as a starting point.
+ */
+public class IncrementalHnswGraphMerger {
+
+  private KnnVectorsReader initReader;
+  private MergeState.DocMap initDocMap;
+  private int initGraphSize;
+  private final FieldInfo fieldInfo;
+  private final RandomVectorScorerSupplier scorerSupplier;
+  private final int M;
+  private final int beamWidth;
+
+  /**
+   * @param fieldInfo FieldInfo for the field being merged
+   */
+  public IncrementalHnswGraphMerger(
+      FieldInfo fieldInfo, RandomVectorScorerSupplier scorerSupplier, int M, int beamWidth) {
+    this.fieldInfo = fieldInfo;
+    this.scorerSupplier = scorerSupplier;
+    this.M = M;
+    this.beamWidth = beamWidth;
+  }
+
+  /**
+   * Adds a reader to the graph merger if it meets the following criteria: 1. Does not contain any
+   * deleted docs 2. Is a HnswGraphProvider/PerFieldKnnVectorReader 3. Has the most docs of any
+   * previous reader that met the above criteria
+   *
+   * @param reader KnnVectorsReader to add to the merger
+   * @param docMap MergeState.DocMap for the reader
+   * @param liveDocs Bits representing live docs, can be null
+   * @return this
+   * @throws IOException If an error occurs while reading from the merge state
+   */
+  IncrementalHnswGraphMerger addReader(
+      KnnVectorsReader reader, MergeState.DocMap docMap, Bits liveDocs) throws IOException {
+    KnnVectorsReader currKnnVectorsReader = reader;
+    if (reader instanceof PerFieldKnnVectorsFormat.FieldsReader candidateReader) {
+      currKnnVectorsReader = candidateReader.getFieldReader(fieldInfo.name);
+    }
+
+    if (!(currKnnVectorsReader instanceof HnswGraphProvider) || !allMatch(liveDocs)) {
+      return this;
+    }
+
+    int candidateVectorCount = 0;
+    switch (fieldInfo.getVectorEncoding()) {
+      case BYTE -> {
+        ByteVectorValues byteVectorValues =
+            currKnnVectorsReader.getByteVectorValues(fieldInfo.name);
+        if (byteVectorValues == null) {
+          return this;
+        }
+        candidateVectorCount = byteVectorValues.size();
+      }
+      case FLOAT32 -> {
+        FloatVectorValues vectorValues = currKnnVectorsReader.getFloatVectorValues(fieldInfo.name);
+        if (vectorValues == null) {
+          return this;
+        }
+        candidateVectorCount = vectorValues.size();
+      }
+    }
+    if (candidateVectorCount > initGraphSize) {
+      initReader = currKnnVectorsReader;
+      initDocMap = docMap;
+      initGraphSize = candidateVectorCount;
+    }
+    return this;
+  }
+
+  /**
+   * Builds a new HnswGraphBuilder using the biggest graph from the merge state as a starting point.
+   * If no valid readers were added to the merge state, a new graph is created.
+   *
+   * @param mergeState MergeState for the merge
+   * @return HnswGraphBuilder
+   * @throws IOException If an error occurs while reading from the merge state
+   */
+  public HnswGraphBuilder createBuilder(MergeState mergeState) throws IOException {
+    if (initReader == null) {
+      return HnswGraphBuilder.create(scorerSupplier, M, beamWidth, HnswGraphBuilder.randSeed);
+    }
+
+    HnswGraph initializerGraph = ((HnswGraphProvider) initReader).getGraph(fieldInfo.name);
+    int ordBaseline = getNewOrdOffset(mergeState);
+    BitSet initializedNodes =
+        BitSet.of(
+            DocIdSetIterator.range(ordBaseline, initGraphSize + ordBaseline),
+            mergeState.segmentInfo.maxDoc() + 1);
+    return InitializedHnswGraphBuilder.fromGraph(
+        scorerSupplier,
+        M,
+        beamWidth,
+        HnswGraphBuilder.randSeed,
+        initializerGraph,
+        ordBaseline,
+        initializedNodes);
+  }
+
+  private int getNewOrdOffset(MergeState mergeState) throws IOException {
+    final DocIdSetIterator initializerIterator =
+        switch (fieldInfo.getVectorEncoding()) {
+          case BYTE -> initReader.getByteVectorValues(fieldInfo.name);
+          case FLOAT32 -> initReader.getFloatVectorValues(fieldInfo.name);
+        };
+
+    // minDoc is just the first doc in the main segment containing our graph
+    // we don't need to worry about deleted documents as deletions are not allowed when selecting
+    // the best graph
+    int minDoc = initializerIterator.nextDoc();
+    if (minDoc == NO_MORE_DOCS) {
+      return -1;
+    }
+    minDoc = initDocMap.get(minDoc);
+
+    DocIdSetIterator vectorIterator = null;
+    switch (fieldInfo.getVectorEncoding()) {
+      case BYTE -> vectorIterator =
+          KnnVectorsWriter.MergedVectorValues.mergeByteVectorValues(fieldInfo, mergeState);
+      case FLOAT32 -> vectorIterator =
+          KnnVectorsWriter.MergedVectorValues.mergeFloatVectorValues(fieldInfo, mergeState);
+    }
+
+    // Since there are no deleted documents in our chosen segment, we can assume that the ordinals
+    // are unchanged
+    // meaning we only need to know what ordinal offset to select and apply it

Review Comment:
   @zhaih I switched to an `int[]` where the indices are the old ordinals (since these are contiguous values as we don't allow deleted docs...for now).
   
   The values in the indices are the `newOrdinal` calculated in a similar loop as before with the `Map<Integer, Integer>`



-- 
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-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Extract the hnsw graph merging from being part of the vector writer [lucene]

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on PR #12657:
URL: https://github.com/apache/lucene/pull/12657#issuecomment-1762131240

   Just being paranoid, I tested and verified that recall is absolutely unchanged between these changes.
   
   baseline:
   ```
   0.500	 0.10	100000	10	4	50	20	11496	1.00	post-filter
   0.533	 0.10	100000	10	4	100	20	18504	1.00	post-filter
   0.844	 0.21	100000	10	16	50	20	22211	1.00	post-filter
   0.875	 0.24	100000	10	16	100	20	44031	1.00	post-filter
   ```
   
   candidate:
   ```
   0.500	 0.10	100000	10	4	50	20	11778	1.00	post-filter
   0.533	 0.10	100000	10	4	100	20	18439	1.00	post-filter
   0.844	 0.20	100000	10	16	50	20	24012	1.00	post-filter
   0.875	 0.25	100000	10	16	100	20	46249	1.00	post-filter
   ```
   
   The performance isn't reliable, this was running on my laptop while it was doing lots of other work.


-- 
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-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Extract the hnsw graph merging from being part of the vector writer [lucene]

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12657:
URL: https://github.com/apache/lucene/pull/12657#discussion_r1362456237


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/InitializedHnswGraphBuilder.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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 static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+import java.io.IOException;
+import org.apache.lucene.util.BitSet;
+
+/**
+ * This creates a graph builder that is initialized with the provided HnswGraph. This is useful for
+ * merging HnswGraphs from multiple segments.
+ *
+ * @lucene.experimental
+ */
+public final class InitializedHnswGraphBuilder extends HnswGraphBuilder {
+
+  /**
+   * Create a new HnswGraphBuilder that is initialized with the provided HnswGraph.
+   *
+   * @param scorerSupplier the scorer to use for vectors
+   * @param M the number of connections to keep per node
+   * @param beamWidth the number of nodes to explore in the search
+   * @param seed the seed for the random number generator
+   * @param initializerGraph the graph to initialize the new graph builder
+   * @param newOrdMap a mapping from the old node ordinal to the new node ordinal
+   * @param initializedNodes a bitset of nodes that are already initialized in the initializerGraph
+   * @return a new HnswGraphBuilder that is initialized with the provided HnswGraph
+   * @throws IOException when reading the graph fails
+   */
+  public static InitializedHnswGraphBuilder fromGraph(
+      RandomVectorScorerSupplier scorerSupplier,
+      int M,
+      int beamWidth,
+      long seed,
+      HnswGraph initializerGraph,
+      int[] newOrdMap,
+      BitSet initializedNodes)
+      throws IOException {
+    OnHeapHnswGraph hnsw = new OnHeapHnswGraph(M);
+    for (int level = 0; level < initializerGraph.numLevels(); level++) {

Review Comment:
   updated! Thanks for the callout!



-- 
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-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org