You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ma...@apache.org on 2022/01/28 11:56:09 UTC

[lucene] branch branch_9x updated: LUCENE-10054 Make HnswGraph hierarchical (#608) (#629)

This is an automated email from the ASF dual-hosted git repository.

mayya pushed a commit to branch branch_9x
in repository https://gitbox.apache.org/repos/asf/lucene.git


The following commit(s) were added to refs/heads/branch_9x by this push:
     new 68beb1a  LUCENE-10054 Make HnswGraph hierarchical (#608) (#629)
68beb1a is described below

commit 68beb1acb499b5759c9471dcc815e46071558371
Author: Mayya Sharipova <ma...@elastic.co>
AuthorDate: Fri Jan 28 06:56:04 2022 -0500

    LUCENE-10054 Make HnswGraph hierarchical (#608) (#629)
    
    Currently HNSW has only a single layer.
    This patch makes HNSW graph multi-layered.
    
    This PR is based on the following PRs:
     #250, #267, #287, #315, #536, #416
    
    Main changes:
    - Multi layers are introduced into HnswGraph and HnswGraphBuilder
    - A new Lucene91HnswVectorsFormat with new Lucene91HnswVectorsReader
    and Lucene91HnswVectorsWriter are introduced to encode graph
    layers' information
    - Lucene90Codec, Lucene90HnswVectorsFormat, and the reading logic of
    Lucene90HnswVectorsReader and Lucene90HnswGraph are moved to
    backward_codecs to support reading and searching of graphs built
    in pre 9.1 version. Lucene90HnswVectorsWriter is deleted.
    - For backwards compatible tests, previous Lucene90 graph reading and
    writing logic was copied into test files of
    Lucene90RWHnswVectorsFormat, Lucene90HnswVectorsWriter,
    Lucene90HnswGraphBuilder and Lucene90HnswRWGraph.
    
    TODO: tests for KNN search for graphs built in pre 9.1 version;
    tests for merge of indices of pre 9.1 + current versions.
---
 lucene/CHANGES.txt                                 |   4 +
 lucene/backward-codecs/src/java/module-info.java   |   6 +-
 .../backward_codecs}/lucene90/Lucene90Codec.java   |  12 +-
 .../lucene90/Lucene90HnswGraph.java}               |  44 ++-
 .../lucene90/Lucene90HnswGraphBuilder.java}        |  53 +--
 .../lucene90/Lucene90HnswVectorsFormat.java        |  17 +-
 .../lucene90/Lucene90HnswVectorsReader.java        |  23 +-
 .../backward_codecs/lucene90/package-info.java     |  19 +
 .../services/org.apache.lucene.codecs.Codec        |   1 +
 .../org.apache.lucene.codecs.KnnVectorsFormat      |   2 +-
 .../lucene90/Lucene90HnswVectorsWriter.java        |  16 +-
 .../lucene90/Lucene90RWHnswVectorsFormat.java      |  43 +++
 .../lucene90/TestLucene90HnswVectorsFormat.java    |  10 +-
 .../benchmark/byTask/tasks/CreateIndexTask.java    |   4 +-
 lucene/core/src/java/module-info.java              |   5 +-
 .../src/java/org/apache/lucene/codecs/Codec.java   |   2 +-
 .../org/apache/lucene/codecs/KnnVectorsFormat.java |   2 +-
 .../lucene/codecs/lucene90/package-info.java       | 402 +--------------------
 .../Lucene91Codec.java}                            |  34 +-
 .../Lucene91HnswVectorsFormat.java}                |  59 ++-
 .../Lucene91HnswVectorsReader.java}                | 154 +++++---
 .../Lucene91HnswVectorsWriter.java}                | 140 +++----
 .../{lucene90 => lucene91}/package-info.java       |   9 +-
 .../org/apache/lucene/index/KnnGraphValues.java    |  90 ++++-
 .../org/apache/lucene/util/hnsw/BoundsChecker.java |  34 +-
 .../org/apache/lucene/util/hnsw/HnswGraph.java     | 217 ++++++++---
 .../apache/lucene/util/hnsw/HnswGraphBuilder.java  |  95 +++--
 .../org/apache/lucene/util/hnsw/NeighborArray.java |   8 +-
 .../org/apache/lucene/util/hnsw/NeighborQueue.java |   4 +-
 .../services/org.apache.lucene.codecs.Codec        |   2 +-
 .../org.apache.lucene.codecs.KnnVectorsFormat      |   2 +-
 ...tLucene90StoredFieldsFormatHighCompression.java |   9 +-
 .../TestLucene91HnswVectorsFormat.java}            |  21 +-
 .../test/org/apache/lucene/index/TestKnnGraph.java | 364 ++++++++++++-------
 .../apache/lucene/util/hnsw/KnnGraphTester.java    |  18 +-
 .../org/apache/lucene/util/hnsw/TestHnswGraph.java | 140 +++----
 .../search/suggest/document/TestSuggestField.java  |   4 +-
 .../util/TestRuleSetupAndRestoreClassEnv.java      |   8 +-
 .../org/apache/lucene/tests/util/TestUtil.java     |   8 +-
 39 files changed, 1133 insertions(+), 952 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index f933144..cd02a13 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -108,6 +108,10 @@ Improvements
 * LUCENE-10229: Unify behaviour of match offsets for interval queries on fields
   with or without offsets enabled. (Patrick Zhai)
 
+* LUCENE-10054 Make HnswGraph hierarchical (Mayya Sharipova, Julie Tibshirani, Mike Sokolov,
+  Adrien Grand)
+
+
 Optimizations
 ---------------------
 
diff --git a/lucene/backward-codecs/src/java/module-info.java b/lucene/backward-codecs/src/java/module-info.java
index 8e01546..a2a0c2f 100644
--- a/lucene/backward-codecs/src/java/module-info.java
+++ b/lucene/backward-codecs/src/java/module-info.java
@@ -29,6 +29,7 @@ module org.apache.lucene.backward_codecs {
   exports org.apache.lucene.backward_codecs.lucene84;
   exports org.apache.lucene.backward_codecs.lucene86;
   exports org.apache.lucene.backward_codecs.lucene87;
+  exports org.apache.lucene.backward_codecs.lucene90;
   exports org.apache.lucene.backward_codecs.packed;
   exports org.apache.lucene.backward_codecs.store;
 
@@ -38,10 +39,13 @@ module org.apache.lucene.backward_codecs {
   provides org.apache.lucene.codecs.PostingsFormat with
       org.apache.lucene.backward_codecs.lucene50.Lucene50PostingsFormat,
       org.apache.lucene.backward_codecs.lucene84.Lucene84PostingsFormat;
+  provides org.apache.lucene.codecs.KnnVectorsFormat with
+      org.apache.lucene.backward_codecs.lucene90.Lucene90HnswVectorsFormat;
   provides org.apache.lucene.codecs.Codec with
       org.apache.lucene.backward_codecs.lucene70.Lucene70Codec,
       org.apache.lucene.backward_codecs.lucene80.Lucene80Codec,
       org.apache.lucene.backward_codecs.lucene84.Lucene84Codec,
       org.apache.lucene.backward_codecs.lucene86.Lucene86Codec,
-      org.apache.lucene.backward_codecs.lucene87.Lucene87Codec;
+      org.apache.lucene.backward_codecs.lucene87.Lucene87Codec,
+      org.apache.lucene.backward_codecs.lucene90.Lucene90Codec;
 }
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90Codec.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90Codec.java
similarity index 90%
copy from lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90Codec.java
copy to lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90Codec.java
index 97b8c1b..40ba066 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90Codec.java
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90Codec.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.lucene.codecs.lucene90;
+package org.apache.lucene.backward_codecs.lucene90;
 
 import java.util.Objects;
 import org.apache.lucene.codecs.Codec;
@@ -30,6 +30,16 @@ import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.SegmentInfoFormat;
 import org.apache.lucene.codecs.StoredFieldsFormat;
 import org.apache.lucene.codecs.TermVectorsFormat;
+import org.apache.lucene.codecs.lucene90.Lucene90CompoundFormat;
+import org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat;
+import org.apache.lucene.codecs.lucene90.Lucene90FieldInfosFormat;
+import org.apache.lucene.codecs.lucene90.Lucene90LiveDocsFormat;
+import org.apache.lucene.codecs.lucene90.Lucene90NormsFormat;
+import org.apache.lucene.codecs.lucene90.Lucene90PointsFormat;
+import org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat;
+import org.apache.lucene.codecs.lucene90.Lucene90SegmentInfoFormat;
+import org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat;
+import org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat;
 import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
 import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
 import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
diff --git a/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraph.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswGraph.java
similarity index 84%
copy from lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraph.java
copy to lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswGraph.java
index 9af1017..d8d28ec 100644
--- a/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraph.java
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswGraph.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.lucene.util.hnsw;
+package org.apache.lucene.backward_codecs.lucene90;
 
 import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
 
@@ -28,6 +28,9 @@ import org.apache.lucene.index.RandomAccessVectorValues;
 import org.apache.lucene.index.VectorSimilarityFunction;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.SparseFixedBitSet;
+import org.apache.lucene.util.hnsw.BoundsChecker;
+import org.apache.lucene.util.hnsw.NeighborArray;
+import org.apache.lucene.util.hnsw.NeighborQueue;
 
 /**
  * Navigable Small-world graph. Provides efficient approximate nearest neighbor search for high
@@ -40,20 +43,22 @@ import org.apache.lucene.util.SparseFixedBitSet;
  * <h2>Hyperparameters</h2>
  *
  * <ul>
- *   <li><code>numSeed</code> is the equivalent of <code>m</code> in the 2012 paper; it controls the
+ *   <li><code>numSeed</code> is the equivalent of <code>m</code> in the 2014 paper; it controls the
  *       number of random entry points to sample.
- *   <li><code>beamWidth</code> in {@link HnswGraphBuilder} has the same meaning as <code>efConst
- *       </code> in the 2016 paper. It is the number of nearest neighbor candidates to track while
- *       searching the graph for each newly inserted node.
+ *   <li><code>beamWidth</code> in {@link Lucene90HnswGraphBuilder} has the same meaning as <code>
+ *       efConst </code> in the 2018 paper. It is the number of nearest neighbor candidates to track
+ *       while searching the graph for each newly inserted node.
  *   <li><code>maxConn</code> has the same meaning as <code>M</code> in the later paper; it controls
  *       how many of the <code>efConst</code> neighbors are connected to the new node
  * </ul>
  *
  * <p>Note: The graph may be searched by multiple threads concurrently, but updates are not
- * thread-safe. The search method optionally takes a set of "accepted nodes", which can be used to
- * exclude deleted documents.
+ * thread-safe. Also note: there is no notion of deletions. Document searching built on top of this
+ * must do its own deletion-filtering.
+ *
+ * <p>Graph building logic is preserved here only for tests.
  */
-public final class HnswGraph extends KnnGraphValues {
+public final class Lucene90HnswGraph extends KnnGraphValues {
 
   private final int maxConn;
 
@@ -66,7 +71,7 @@ public final class HnswGraph extends KnnGraphValues {
   private int upto;
   private NeighborArray cur;
 
-  HnswGraph(int maxConn) {
+  Lucene90HnswGraph(int maxConn) {
     graph = new ArrayList<>();
     // Typically with diversity criteria we see nodes not fully occupied; average fanout seems to be
     // about 1/2 maxConn. There is some indexing time penalty for under-allocating, but saves RAM
@@ -136,7 +141,7 @@ public final class HnswGraph extends KnnGraphValues {
         }
       }
       int topCandidateNode = candidates.pop();
-      graphValues.seek(topCandidateNode);
+      graphValues.seek(0, topCandidateNode);
       int friendOrd;
       while ((friendOrd = graphValues.nextNeighbor()) != NO_MORE_DOCS) {
         assert friendOrd < size : "friendOrd=" + friendOrd + "; size=" + size;
@@ -181,7 +186,7 @@ public final class HnswGraph extends KnnGraphValues {
   }
 
   @Override
-  public void seek(int targetNode) {
+  public void seek(int level, int targetNode) {
     cur = getNeighbors(targetNode);
     upto = -1;
   }
@@ -189,8 +194,23 @@ public final class HnswGraph extends KnnGraphValues {
   @Override
   public int nextNeighbor() {
     if (++upto < cur.size()) {
-      return cur.node[upto];
+      return cur.node()[upto];
     }
     return NO_MORE_DOCS;
   }
+
+  @Override
+  public int numLevels() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int entryNode() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public NodesIterator getNodesOnLevel(int level) {
+    throw new UnsupportedOperationException();
+  }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswGraphBuilder.java
similarity index 87%
copy from lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java
copy to lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswGraphBuilder.java
index f5cfc6a..7fda65e 100644
--- a/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswGraphBuilder.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.lucene.util.hnsw;
+package org.apache.lucene.backward_codecs.lucene90;
 
 import java.io.IOException;
 import java.util.Locale;
@@ -25,18 +25,24 @@ import org.apache.lucene.index.RandomAccessVectorValues;
 import org.apache.lucene.index.RandomAccessVectorValuesProducer;
 import org.apache.lucene.index.VectorSimilarityFunction;
 import org.apache.lucene.util.InfoStream;
+import org.apache.lucene.util.hnsw.BoundsChecker;
+import org.apache.lucene.util.hnsw.NeighborArray;
+import org.apache.lucene.util.hnsw.NeighborQueue;
 
 /**
- * Builder for HNSW graph. See {@link HnswGraph} for a gloss on the algorithm and the meaning of the
- * hyperparameters.
+ * Builder for HNSW graph. See {@link Lucene90HnswGraph} for a gloss on the algorithm and the
+ * meaning of the hyperparameters.
+ *
+ * <p>This class is preserved here only for tests.
  */
-public final class HnswGraphBuilder {
+public final class Lucene90HnswGraphBuilder {
 
-  // default random seed for level generation
+  /** Default random seed for level generation * */
   private static final long DEFAULT_RAND_SEED = System.currentTimeMillis();
+  /** A name for the HNSW component for the info-stream * */
   public static final String HNSW_COMPONENT = "HNSW";
 
-  // expose for testing.
+  /** Random seed for level generation; public to expose for testing * */
   public static long randSeed = DEFAULT_RAND_SEED;
 
   private final int maxConn;
@@ -47,7 +53,7 @@ public final class HnswGraphBuilder {
   private final RandomAccessVectorValues vectorValues;
   private final SplittableRandom random;
   private final BoundsChecker bound;
-  final HnswGraph hnsw;
+  final Lucene90HnswGraph hnsw;
 
   private InfoStream infoStream = InfoStream.getDefault();
 
@@ -67,7 +73,7 @@ public final class HnswGraphBuilder {
    * @param seed the seed for a random number generator used during graph construction. Provide this
    *     to ensure repeatable construction.
    */
-  public HnswGraphBuilder(
+  public Lucene90HnswGraphBuilder(
       RandomAccessVectorValuesProducer vectors,
       VectorSimilarityFunction similarityFunction,
       int maxConn,
@@ -84,7 +90,7 @@ public final class HnswGraphBuilder {
     }
     this.maxConn = maxConn;
     this.beamWidth = beamWidth;
-    this.hnsw = new HnswGraph(maxConn);
+    this.hnsw = new Lucene90HnswGraph(maxConn);
     bound = BoundsChecker.create(similarityFunction.reversed);
     random = new SplittableRandom(seed);
     scratch = new NeighborArray(Math.max(beamWidth, maxConn + 1));
@@ -98,7 +104,7 @@ public final class HnswGraphBuilder {
    * @param vectors the vectors for which to build a nearest neighbors graph. Must be an independet
    *     accessor for the vectors
    */
-  public HnswGraph build(RandomAccessVectorValues vectors) throws IOException {
+  public Lucene90HnswGraph build(RandomAccessVectorValues vectors) throws IOException {
     if (vectors == vectorValues) {
       throw new IllegalArgumentException(
           "Vectors to build must be independent of the source of vectors provided to HnswGraphBuilder()");
@@ -128,6 +134,7 @@ public final class HnswGraphBuilder {
     return hnsw;
   }
 
+  /** Set info-stream to output debugging information * */
   public void setInfoStream(InfoStream infoStream) {
     this.infoStream = infoStream;
   }
@@ -136,7 +143,7 @@ public final class HnswGraphBuilder {
   void addGraphNode(float[] value) throws IOException {
     // We pass 'null' for acceptOrds because there are no deletions while building the graph
     NeighborQueue candidates =
-        HnswGraph.search(
+        Lucene90HnswGraph.search(
             value, beamWidth, beamWidth, vectorValues, similarityFunction, hnsw, null, random);
 
     int node = hnsw.addNode();
@@ -167,9 +174,9 @@ public final class HnswGraphBuilder {
     // applying diversity heuristic)
     int size = neighbors.size();
     for (int i = 0; i < size; i++) {
-      int nbr = neighbors.node[i];
+      int nbr = neighbors.node()[i];
       NeighborArray nbrNbr = hnsw.getNeighbors(nbr);
-      nbrNbr.add(node, neighbors.score[i]);
+      nbrNbr.add(node, neighbors.score()[i]);
       if (nbrNbr.size() > maxConn) {
         diversityUpdate(nbrNbr);
       }
@@ -181,8 +188,8 @@ public final class HnswGraphBuilder {
     for (int i = candidates.size() - 1; neighbors.size() < maxConn && i >= 0; i--) {
       // compare each neighbor (in distance order) against the closer neighbors selected so far,
       // only adding it if it is closer to the target than to any of the other selected neighbors
-      int cNode = candidates.node[i];
-      float cScore = candidates.score[i];
+      int cNode = candidates.node()[i];
+      float cScore = candidates.score()[i];
       assert cNode < hnsw.size();
       if (diversityCheck(vectorValues.vectorValue(cNode), cScore, neighbors, buildVectors)) {
         neighbors.add(cNode, cScore);
@@ -219,7 +226,7 @@ public final class HnswGraphBuilder {
     bound.set(score);
     for (int i = 0; i < neighbors.size(); i++) {
       float diversityCheck =
-          similarityFunction.compare(candidate, vectorValues.vectorValue(neighbors.node[i]));
+          similarityFunction.compare(candidate, vectorValues.vectorValue(neighbors.node()[i]));
       if (bound.check(diversityCheck) == false) {
         return false;
       }
@@ -232,8 +239,8 @@ public final class HnswGraphBuilder {
     int replacePoint = findNonDiverse(neighbors);
     if (replacePoint == -1) {
       // none found; check score against worst existing neighbor
-      bound.set(neighbors.score[0]);
-      if (bound.check(neighbors.score[maxConn])) {
+      bound.set(neighbors.score()[0]);
+      if (bound.check(neighbors.score()[maxConn])) {
         // drop the new neighbor; it is not competitive and there were no diversity failures
         neighbors.removeLast();
         return;
@@ -241,8 +248,8 @@ public final class HnswGraphBuilder {
         replacePoint = 0;
       }
     }
-    neighbors.node[replacePoint] = neighbors.node[maxConn];
-    neighbors.score[replacePoint] = neighbors.score[maxConn];
+    neighbors.node()[replacePoint] = neighbors.node()[maxConn];
+    neighbors.score()[replacePoint] = neighbors.score()[maxConn];
     neighbors.removeLast();
   }
 
@@ -251,12 +258,12 @@ public final class HnswGraphBuilder {
     for (int i = neighbors.size() - 1; i >= 0; i--) {
       // check each neighbor against its better-scoring neighbors. If it fails diversity check with
       // them, drop it
-      int nbrNode = neighbors.node[i];
-      bound.set(neighbors.score[i]);
+      int nbrNode = neighbors.node()[i];
+      bound.set(neighbors.score()[i]);
       float[] nbrVector = vectorValues.vectorValue(nbrNode);
       for (int j = maxConn; j > i; j--) {
         float diversityCheck =
-            similarityFunction.compare(nbrVector, buildVectors.vectorValue(neighbors.node[j]));
+            similarityFunction.compare(nbrVector, buildVectors.vectorValue(neighbors.node()[j]));
         if (bound.check(diversityCheck) == false) {
           // node j is too similar to node i given its score relative to the base node
           // replace it with the new node, which is at [maxConn]
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90HnswVectorsFormat.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswVectorsFormat.java
similarity index 88%
copy from lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90HnswVectorsFormat.java
copy to lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswVectorsFormat.java
index 90875d9..c622d9c 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90HnswVectorsFormat.java
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswVectorsFormat.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.lucene.codecs.lucene90;
+package org.apache.lucene.backward_codecs.lucene90;
 
 import java.io.IOException;
 import org.apache.lucene.codecs.KnnVectorsFormat;
@@ -65,7 +65,7 @@ import org.apache.lucene.util.hnsw.HnswGraph;
  *
  * @lucene.experimental
  */
-public final class Lucene90HnswVectorsFormat extends KnnVectorsFormat {
+public class Lucene90HnswVectorsFormat extends KnnVectorsFormat {
 
   static final String META_CODEC_NAME = "Lucene90HnswVectorsFormatMeta";
   static final String VECTOR_DATA_CODEC_NAME = "Lucene90HnswVectorsFormatData";
@@ -77,26 +77,33 @@ public final class Lucene90HnswVectorsFormat extends KnnVectorsFormat {
   static final int VERSION_START = 0;
   static final int VERSION_CURRENT = VERSION_START;
 
+  /** Default number of maximum connections per node */
   public static final int DEFAULT_MAX_CONN = 16;
+  /**
+   * Default number of the size of the queue maintained while searching and the number of random
+   * entry points to sample during a graph construction.
+   */
   public static final int DEFAULT_BEAM_WIDTH = 100;
 
   /**
    * Controls how many of the nearest neighbor candidates are connected to the new node. Defaults to
    * {@link Lucene90HnswVectorsFormat#DEFAULT_MAX_CONN}. See {@link HnswGraph} for more details.
    */
-  private final int maxConn;
+  final int maxConn;
 
   /**
    * The number of candidate neighbors to track while searching the graph for each newly inserted
    * node. Defaults to to {@link Lucene90HnswVectorsFormat#DEFAULT_BEAM_WIDTH}. See {@link
    * HnswGraph} for details.
    */
-  private final int beamWidth;
+  final int beamWidth;
 
+  /** A constructor for vectors format with default parameters */
   public Lucene90HnswVectorsFormat() {
     this(DEFAULT_MAX_CONN, DEFAULT_BEAM_WIDTH);
   }
 
+  /** A constructor for vectors format */
   public Lucene90HnswVectorsFormat(int maxConn, int beamWidth) {
     super("Lucene90HnswVectorsFormat");
     this.maxConn = maxConn;
@@ -105,7 +112,7 @@ public final class Lucene90HnswVectorsFormat extends KnnVectorsFormat {
 
   @Override
   public KnnVectorsWriter fieldsWriter(SegmentWriteState state) throws IOException {
-    return new Lucene90HnswVectorsWriter(state, maxConn, beamWidth);
+    throw new UnsupportedOperationException("Old codecs may only be used for reading");
   }
 
   @Override
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90HnswVectorsReader.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswVectorsReader.java
similarity index 96%
copy from lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90HnswVectorsReader.java
copy to lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswVectorsReader.java
index bb62ab9..7669a8d 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90HnswVectorsReader.java
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswVectorsReader.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.lucene.codecs.lucene90;
+package org.apache.lucene.backward_codecs.lucene90;
 
 import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
 
@@ -47,7 +47,6 @@ import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.RamUsageEstimator;
-import org.apache.lucene.util.hnsw.HnswGraph;
 import org.apache.lucene.util.hnsw.NeighborQueue;
 
 /**
@@ -244,7 +243,7 @@ public final class Lucene90HnswVectorsReader extends KnnVectorsReader {
     // use a seed that is fixed for the index so we get reproducible results for the same query
     final SplittableRandom random = new SplittableRandom(checksumSeed);
     NeighborQueue results =
-        HnswGraph.search(
+        Lucene90HnswGraph.search(
             target,
             k,
             k,
@@ -291,6 +290,7 @@ public final class Lucene90HnswVectorsReader extends KnnVectorsReader {
     };
   }
 
+  /** Get knn graph values; used for testing */
   public KnnGraphValues getGraphValues(String field) throws IOException {
     FieldInfo info = fieldInfos.fieldInfo(field);
     if (info == null) {
@@ -480,7 +480,7 @@ public final class Lucene90HnswVectorsReader extends KnnVectorsReader {
     }
 
     @Override
-    public void seek(int targetOrd) throws IOException {
+    public void seek(int level, int targetOrd) throws IOException {
       // unsafe; no bounds checking
       dataIn.seek(entry.ordOffsets[targetOrd]);
       arcCount = dataIn.readInt();
@@ -502,5 +502,20 @@ public final class Lucene90HnswVectorsReader extends KnnVectorsReader {
       arc += dataIn.readVInt();
       return arc;
     }
+
+    @Override
+    public int numLevels() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public int entryNode() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public NodesIterator getNodesOnLevel(int level) {
+      throw new UnsupportedOperationException();
+    }
   }
 }
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/package-info.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/package-info.java
new file mode 100644
index 0000000..5ad2dbc
--- /dev/null
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/package-info.java
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+
+/** Lucene 9.0 file format. */
+package org.apache.lucene.backward_codecs.lucene90;
diff --git a/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
index 5ff3ebf..6cf7b7a 100644
--- a/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
+++ b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
@@ -18,3 +18,4 @@ org.apache.lucene.backward_codecs.lucene84.Lucene84Codec
 org.apache.lucene.backward_codecs.lucene86.Lucene86Codec
 org.apache.lucene.backward_codecs.lucene87.Lucene87Codec
 org.apache.lucene.backward_codecs.lucene70.Lucene70Codec
+org.apache.lucene.backward_codecs.lucene90.Lucene90Codec
diff --git a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.KnnVectorsFormat b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.KnnVectorsFormat
similarity index 92%
copy from lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.KnnVectorsFormat
copy to lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.KnnVectorsFormat
index e7158f1..17d89f3 100644
--- a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.KnnVectorsFormat
+++ b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.KnnVectorsFormat
@@ -13,4 +13,4 @@
 #  See the License for the specific language governing permissions and
 #  limitations under the License.
 
-org.apache.lucene.codecs.lucene90.Lucene90HnswVectorsFormat
\ No newline at end of file
+org.apache.lucene.backward_codecs.lucene90.Lucene90HnswVectorsFormat
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90HnswVectorsWriter.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswVectorsWriter.java
similarity index 96%
copy from lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90HnswVectorsWriter.java
copy to lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswVectorsWriter.java
index 0fbd1be..d76e5ef 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90HnswVectorsWriter.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswVectorsWriter.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.lucene.codecs.lucene90;
+package org.apache.lucene.backward_codecs.lucene90;
 
 import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
 
@@ -35,8 +35,6 @@ import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.hnsw.HnswGraph;
-import org.apache.lucene.util.hnsw.HnswGraphBuilder;
 import org.apache.lucene.util.hnsw.NeighborArray;
 
 /**
@@ -235,11 +233,15 @@ public final class Lucene90HnswVectorsWriter extends KnnVectorsWriter {
       int maxConn,
       int beamWidth)
       throws IOException {
-    HnswGraphBuilder hnswGraphBuilder =
-        new HnswGraphBuilder(
-            vectorValues, similarityFunction, maxConn, beamWidth, HnswGraphBuilder.randSeed);
+    Lucene90HnswGraphBuilder hnswGraphBuilder =
+        new Lucene90HnswGraphBuilder(
+            vectorValues,
+            similarityFunction,
+            maxConn,
+            beamWidth,
+            Lucene90HnswGraphBuilder.randSeed);
     hnswGraphBuilder.setInfoStream(segmentWriteState.infoStream);
-    HnswGraph graph = hnswGraphBuilder.build(vectorValues.randomAccess());
+    Lucene90HnswGraph graph = hnswGraphBuilder.build(vectorValues.randomAccess());
 
     for (int ord = 0; ord < offsets.length; ord++) {
       // write graph
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene90/Lucene90RWHnswVectorsFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene90/Lucene90RWHnswVectorsFormat.java
new file mode 100644
index 0000000..dd720c1
--- /dev/null
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene90/Lucene90RWHnswVectorsFormat.java
@@ -0,0 +1,43 @@
+/*
+ * 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.backward_codecs.lucene90;
+
+import java.io.IOException;
+import org.apache.lucene.codecs.KnnVectorsWriter;
+import org.apache.lucene.index.SegmentWriteState;
+
+public class Lucene90RWHnswVectorsFormat extends Lucene90HnswVectorsFormat {
+
+  public Lucene90RWHnswVectorsFormat(int maxConn, int beamWidth) {
+    super(maxConn, beamWidth);
+  }
+
+  @Override
+  public KnnVectorsWriter fieldsWriter(SegmentWriteState state) throws IOException {
+    return new Lucene90HnswVectorsWriter(state, maxConn, beamWidth);
+  }
+
+  @Override
+  public String toString() {
+    return "Lucene90RWHnswVectorsFormat(name = Lucene90RWHnswVectorsFormat, maxConn = "
+        + maxConn
+        + ", beamWidth="
+        + beamWidth
+        + ")";
+  }
+}
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90HnswVectorsFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene90/TestLucene90HnswVectorsFormat.java
similarity index 82%
copy from lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90HnswVectorsFormat.java
copy to lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene90/TestLucene90HnswVectorsFormat.java
index 7baca8c..8fe6dc2 100644
--- a/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90HnswVectorsFormat.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene90/TestLucene90HnswVectorsFormat.java
@@ -14,11 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.lucene.codecs.lucene90;
+package org.apache.lucene.backward_codecs.lucene90;
 
 import static com.carrotsearch.randomizedtesting.RandomizedTest.randomIntBetween;
-import static org.apache.lucene.codecs.lucene90.Lucene90HnswVectorsFormat.DEFAULT_BEAM_WIDTH;
-import static org.apache.lucene.codecs.lucene90.Lucene90HnswVectorsFormat.DEFAULT_MAX_CONN;
+import static org.apache.lucene.backward_codecs.lucene90.Lucene90HnswVectorsFormat.DEFAULT_BEAM_WIDTH;
+import static org.apache.lucene.backward_codecs.lucene90.Lucene90HnswVectorsFormat.DEFAULT_MAX_CONN;
 
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.KnnVectorsFormat;
@@ -38,11 +38,11 @@ public class TestLucene90HnswVectorsFormat extends BaseKnnVectorsFormatTestCase
         new Lucene90Codec() {
           @Override
           public KnnVectorsFormat getKnnVectorsFormatForField(String field) {
-            return new Lucene90HnswVectorsFormat(maxConn, beamWidth);
+            return new Lucene90RWHnswVectorsFormat(maxConn, beamWidth);
           }
         };
     String expectedString =
-        "Lucene90HnswVectorsFormat(name = Lucene90HnswVectorsFormat, maxConn = "
+        "Lucene90RWHnswVectorsFormat(name = Lucene90RWHnswVectorsFormat, maxConn = "
             + maxConn
             + ", beamWidth="
             + beamWidth
diff --git a/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java b/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java
index d7ae013..1f76e82 100644
--- a/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java
+++ b/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java
@@ -27,7 +27,7 @@ import org.apache.lucene.benchmark.byTask.PerfRunData;
 import org.apache.lucene.benchmark.byTask.utils.Config;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene90.Lucene90Codec;
+import org.apache.lucene.codecs.lucene91.Lucene91Codec;
 import org.apache.lucene.index.ConcurrentMergeScheduler;
 import org.apache.lucene.index.IndexCommit;
 import org.apache.lucene.index.IndexDeletionPolicy;
@@ -152,7 +152,7 @@ public class CreateIndexTask extends PerfTask {
       try {
         final PostingsFormat postingsFormatChosen = PostingsFormat.forName(postingsFormat);
         iwConf.setCodec(
-            new Lucene90Codec() {
+            new Lucene91Codec() {
               @Override
               public PostingsFormat getPostingsFormatForField(String field) {
                 return postingsFormatChosen;
diff --git a/lucene/core/src/java/module-info.java b/lucene/core/src/java/module-info.java
index 02fc3ff..f4139e7 100644
--- a/lucene/core/src/java/module-info.java
+++ b/lucene/core/src/java/module-info.java
@@ -27,6 +27,7 @@ module org.apache.lucene.core {
   exports org.apache.lucene.analysis.tokenattributes;
   exports org.apache.lucene.codecs;
   exports org.apache.lucene.codecs.compressing;
+  exports org.apache.lucene.codecs.lucene91;
   exports org.apache.lucene.codecs.lucene90;
   exports org.apache.lucene.codecs.lucene90.blocktree;
   exports org.apache.lucene.codecs.lucene90.compressing;
@@ -59,11 +60,11 @@ module org.apache.lucene.core {
   provides org.apache.lucene.analysis.TokenizerFactory with
       org.apache.lucene.analysis.standard.StandardTokenizerFactory;
   provides org.apache.lucene.codecs.Codec with
-      org.apache.lucene.codecs.lucene90.Lucene90Codec;
+      org.apache.lucene.codecs.lucene91.Lucene91Codec;
   provides org.apache.lucene.codecs.DocValuesFormat with
       org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat;
   provides org.apache.lucene.codecs.KnnVectorsFormat with
-      org.apache.lucene.codecs.lucene90.Lucene90HnswVectorsFormat;
+      org.apache.lucene.codecs.lucene91.Lucene91HnswVectorsFormat;
   provides org.apache.lucene.codecs.PostingsFormat with
       org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat;
   provides org.apache.lucene.index.SortFieldProvider with
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/Codec.java b/lucene/core/src/java/org/apache/lucene/codecs/Codec.java
index 69ecde2..176cc57 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/Codec.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/Codec.java
@@ -55,7 +55,7 @@ public abstract class Codec implements NamedSPILoader.NamedSPI {
       return LOADER;
     }
 
-    static Codec defaultCodec = LOADER.lookup("Lucene90");
+    static Codec defaultCodec = LOADER.lookup("Lucene91");
   }
 
   private final String name;
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsFormat.java
index 4b58f2d..69b1a42 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsFormat.java
@@ -85,7 +85,7 @@ public abstract class KnnVectorsFormat implements NamedSPILoader.NamedSPI {
         @Override
         public KnnVectorsWriter fieldsWriter(SegmentWriteState state) {
           throw new UnsupportedOperationException(
-              "Attempt to write EMPTY VectorValues: maybe you forgot to use codec=Lucene90");
+              "Attempt to write EMPTY VectorValues: maybe you forgot to use codec=Lucene91");
         }
 
         @Override
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/package-info.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/package-info.java
index d8c6828..e8fdeb4 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/package-info.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/package-info.java
@@ -15,405 +15,5 @@
  * limitations under the License.
  */
 
-/**
- * Lucene 9.0 file format.
- *
- * <h2>Apache Lucene - Index File Formats</h2>
- *
- * <div>
- *
- * <ul>
- *   <li><a href="#Introduction">Introduction</a>
- *   <li><a href="#Definitions">Definitions</a>
- *       <ul>
- *         <li><a href="#Inverted_Indexing">Inverted Indexing</a>
- *         <li><a href="#Types_of_Fields">Types of Fields</a>
- *         <li><a href="#Segments">Segments</a>
- *         <li><a href="#Document_Numbers">Document Numbers</a>
- *       </ul>
- *   <li><a href="#Overview">Index Structure Overview</a>
- *   <li><a href="#File_Naming">File Naming</a>
- *   <li><a href="#file-names">Summary of File Extensions</a>
- *       <ul>
- *         <li><a href="#Lock_File">Lock File</a>
- *         <li><a href="#History">History</a>
- *         <li><a href="#Limitations">Limitations</a>
- *       </ul>
- * </ul>
- *
- * </div> <a id="Introduction"></a>
- *
- * <h3>Introduction</h3>
- *
- * <div>
- *
- * <p>This document defines the index file formats used in this version of Lucene. If you are using
- * a different version of Lucene, please consult the copy of <code>docs/</code> that was distributed
- * with the version you are using.
- *
- * <p>This document attempts to provide a high-level definition of the Apache Lucene file formats.
- * </div> <a id="Definitions"></a>
- *
- * <h3>Definitions</h3>
- *
- * <div>
- *
- * <p>The fundamental concepts in Lucene are index, document, field and term.
- *
- * <p>An index contains a sequence of documents.
- *
- * <ul>
- *   <li>A document is a sequence of fields.
- *   <li>A field is a named sequence of terms.
- *   <li>A term is a sequence of bytes.
- * </ul>
- *
- * <p>The same sequence of bytes in two different fields is considered a different term. Thus terms
- * are represented as a pair: the string naming the field, and the bytes within the field. <a
- * id="Inverted_Indexing"></a>
- *
- * <h4>Inverted Indexing</h4>
- *
- * <p>Lucene's index stores terms and statistics about those terms in order to make term-based
- * search more efficient. Lucene's terms index falls into the family of indexes known as an
- * <i>inverted index.</i> This is because it can list, for a term, the documents that contain it.
- * This is the inverse of the natural relationship, in which documents list terms. <a
- * id="Types_of_Fields"></a>
- *
- * <h4>Types of Fields</h4>
- *
- * <p>In Lucene, fields may be <i>stored</i>, in which case their text is stored in the index
- * literally, in a non-inverted manner. Fields that are inverted are called <i>indexed</i>. A field
- * may be both stored and indexed.
- *
- * <p>The text of a field may be <i>tokenized</i> into terms to be indexed, or the text of a field
- * may be used literally as a term to be indexed. Most fields are tokenized, but sometimes it is
- * useful for certain identifier fields to be indexed literally.
- *
- * <p>See the {@link org.apache.lucene.document.Field Field} java docs for more information on
- * Fields. <a id="Segments"></a>
- *
- * <h4>Segments</h4>
- *
- * <p>Lucene indexes may be composed of multiple sub-indexes, or <i>segments</i>. Each segment is a
- * fully independent index, which could be searched separately. Indexes evolve by:
- *
- * <ol>
- *   <li>Creating new segments for newly added documents.
- *   <li>Merging existing segments.
- * </ol>
- *
- * <p>Searches may involve multiple segments and/or multiple indexes, each index potentially
- * composed of a set of segments. <a id="Document_Numbers"></a>
- *
- * <h4>Document Numbers</h4>
- *
- * <p>Internally, Lucene refers to documents by an integer <i>document number</i>. The first
- * document added to an index is numbered zero, and each subsequent document added gets a number one
- * greater than the previous.
- *
- * <p>Note that a document's number may change, so caution should be taken when storing these
- * numbers outside of Lucene. In particular, numbers may change in the following situations:
- *
- * <ul>
- *   <li>
- *       <p>The numbers stored in each segment are unique only within the segment, and must be
- *       converted before they can be used in a larger context. The standard technique is to
- *       allocate each segment a range of values, based on the range of numbers used in that
- *       segment. To convert a document number from a segment to an external value, the segment's
- *       <i>base</i> document number is added. To convert an external value back to a
- *       segment-specific value, the segment is identified by the range that the external value is
- *       in, and the segment's base value is subtracted. For example two five document segments
- *       might be combined, so that the first segment has a base value of zero, and the second of
- *       five. Document three from the second segment would have an external value of eight.
- *   <li>
- *       <p>When documents are deleted, gaps are created in the numbering. These are eventually
- *       removed as the index evolves through merging. Deleted documents are dropped when segments
- *       are merged. A freshly-merged segment thus has no gaps in its numbering.
- * </ul>
- *
- * </div> <a id="Overview"></a>
- *
- * <h3>Index Structure Overview</h3>
- *
- * <div>
- *
- * <p>Each segment index maintains the following:
- *
- * <ul>
- *   <li>{@link org.apache.lucene.codecs.lucene90.Lucene90SegmentInfoFormat Segment info}. This
- *       contains metadata about a segment, such as the number of documents, what files it uses, and
- *       information about how the segment is sorted
- *   <li>{@link org.apache.lucene.codecs.lucene90.Lucene90FieldInfosFormat Field names}. This
- *       contains metadata about the set of named fields used in the index.
- *   <li>{@link org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat Stored Field values}.
- *       This contains, for each document, a list of attribute-value pairs, where the attributes are
- *       field names. These are used to store auxiliary information about the document, such as its
- *       title, url, or an identifier to access a database. The set of stored fields are what is
- *       returned for each hit when searching. This is keyed by document number.
- *   <li>{@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat Term dictionary}. A
- *       dictionary containing all of the terms used in all of the indexed fields of all of the
- *       documents. The dictionary also contains the number of documents which contain the term, and
- *       pointers to the term's frequency and proximity data.
- *   <li>{@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat Term Frequency data}. For
- *       each term in the dictionary, the numbers of all the documents that contain that term, and
- *       the frequency of the term in that document, unless frequencies are omitted ({@link
- *       org.apache.lucene.index.IndexOptions#DOCS IndexOptions.DOCS})
- *   <li>{@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat Term Proximity data}. For
- *       each term in the dictionary, the positions that the term occurs in each document. Note that
- *       this will not exist if all fields in all documents omit position data.
- *   <li>{@link org.apache.lucene.codecs.lucene90.Lucene90NormsFormat Normalization factors}. For
- *       each field in each document, a value is stored that is multiplied into the score for hits
- *       on that field.
- *   <li>{@link org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat Term Vectors}. For each
- *       field in each document, the term vector (sometimes called document vector) may be stored. A
- *       term vector consists of term text and term frequency. To add Term Vectors to your index see
- *       the {@link org.apache.lucene.document.Field Field} constructors
- *   <li>{@link org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat Per-document values}. Like
- *       stored values, these are also keyed by document number, but are generally intended to be
- *       loaded into main memory for fast access. Whereas stored values are generally intended for
- *       summary results from searches, per-document values are useful for things like scoring
- *       factors.
- *   <li>{@link org.apache.lucene.codecs.lucene90.Lucene90LiveDocsFormat Live documents}. An
- *       optional file indicating which documents are live.
- *   <li>{@link org.apache.lucene.codecs.lucene90.Lucene90PointsFormat Point values}. Optional pair
- *       of files, recording dimensionally indexed fields, to enable fast numeric range filtering
- *       and large numeric values like BigInteger and BigDecimal (1D) and geographic shape
- *       intersection (2D, 3D).
- *   <li>{@link org.apache.lucene.codecs.lucene90.Lucene90HnswVectorsFormat Vector values}. The
- *       vector format stores numeric vectors in a format optimized for random access and
- *       computation, supporting high-dimensional nearest-neighbor search.
- * </ul>
- *
- * <p>Details on each of these are provided in their linked pages. </div> <a id="File_Naming"></a>
- *
- * <h3>File Naming</h3>
- *
- * <div>
- *
- * <p>All files belonging to a segment have the same name with varying extensions. The extensions
- * correspond to the different file formats described below. When using the Compound File format
- * (default for small segments) these files (except for the Segment info file, the Lock file, and
- * Deleted documents file) are collapsed into a single .cfs file (see below for details)
- *
- * <p>Typically, all segments in an index are stored in a single directory, although this is not
- * required.
- *
- * <p>File names are never re-used. That is, when any file is saved to the Directory it is given a
- * never before used filename. This is achieved using a simple generations approach. For example,
- * the first segments file is segments_1, then segments_2, etc. The generation is a sequential long
- * integer represented in alpha-numeric (base 36) form. </div> <a id="file-names"></a>
- *
- * <h3>Summary of File Extensions</h3>
- *
- * <div>
- *
- * <p>The following table summarizes the names and extensions of the files in Lucene:
- *
- * <table class="padding4" style="border-spacing: 1px; border-collapse: separate">
- * <caption>lucene filenames by extension</caption>
- * <tr>
- * <th>Name</th>
- * <th>Extension</th>
- * <th>Brief Description</th>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.index.SegmentInfos Segments File}</td>
- * <td>segments_N</td>
- * <td>Stores information about a commit point</td>
- * </tr>
- * <tr>
- * <td><a href="#Lock_File">Lock File</a></td>
- * <td>write.lock</td>
- * <td>The Write lock prevents multiple IndexWriters from writing to the same
- * file.</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene90.Lucene90SegmentInfoFormat Segment Info}</td>
- * <td>.si</td>
- * <td>Stores metadata about a segment</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene90.Lucene90CompoundFormat Compound File}</td>
- * <td>.cfs, .cfe</td>
- * <td>An optional "virtual" file consisting of all the other index files for
- * systems that frequently run out of file handles.</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene90.Lucene90FieldInfosFormat Fields}</td>
- * <td>.fnm</td>
- * <td>Stores information about the fields</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat Field Index}</td>
- * <td>.fdx</td>
- * <td>Contains pointers to field data</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat Field Data}</td>
- * <td>.fdt</td>
- * <td>The stored fields for documents</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat Term Dictionary}</td>
- * <td>.tim</td>
- * <td>The term dictionary, stores term info</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat Term Index}</td>
- * <td>.tip</td>
- * <td>The index into the Term Dictionary</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat Frequencies}</td>
- * <td>.doc</td>
- * <td>Contains the list of docs which contain each term along with frequency</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat Positions}</td>
- * <td>.pos</td>
- * <td>Stores position information about where a term occurs in the index</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat Payloads}</td>
- * <td>.pay</td>
- * <td>Stores additional per-position metadata information such as character offsets and user payloads</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene90.Lucene90NormsFormat Norms}</td>
- * <td>.nvd, .nvm</td>
- * <td>Encodes length and boost factors for docs and fields</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat Per-Document Values}</td>
- * <td>.dvd, .dvm</td>
- * <td>Encodes additional scoring factors or other per-document information.</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat Term Vector Index}</td>
- * <td>.tvx</td>
- * <td>Stores offset into the document data file</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat Term Vector Data}</td>
- * <td>.tvd</td>
- * <td>Contains term vector data.</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene90.Lucene90LiveDocsFormat Live Documents}</td>
- * <td>.liv</td>
- * <td>Info about what documents are live</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene90.Lucene90PointsFormat Point values}</td>
- * <td>.dii, .dim</td>
- * <td>Holds indexed points</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene90.Lucene90HnswVectorsFormat Vector values}</td>
- * <td>.vec, .vem</td>
- * <td>Holds indexed vectors; <code>.vec</code> files contain the raw vector data, and
- * <code>.vem</code> the vector metadata</td>
- * </tr>
- * </table>
- *
- * </div> <a id="Lock_File"></a>
- *
- * <h3>Lock File</h3>
- *
- * The write lock, which is stored in the index directory by default, is named "write.lock". If the
- * lock directory is different from the index directory then the write lock will be named
- * "XXXX-write.lock" where XXXX is a unique prefix derived from the full path to the index
- * directory. When this file is present, a writer is currently modifying the index (adding or
- * removing documents). This lock file ensures that only one writer is modifying the index at a
- * time. <a id="History"></a>
- *
- * <h3>History</h3>
- *
- * <p>Compatibility notes are provided in this document, describing how file formats have changed
- * from prior versions:
- *
- * <ul>
- *   <li>In version 2.1, the file format was changed to allow lock-less commits (ie, no more commit
- *       lock). The change is fully backwards compatible: you can open a pre-2.1 index for searching
- *       or adding/deleting of docs. When the new segments file is saved (committed), it will be
- *       written in the new file format (meaning no specific "upgrade" process is needed). But note
- *       that once a commit has occurred, pre-2.1 Lucene will not be able to read the index.
- *   <li>In version 2.3, the file format was changed to allow segments to share a single set of doc
- *       store (vectors &amp; stored fields) files. This allows for faster indexing in certain
- *       cases. The change is fully backwards compatible (in the same way as the lock-less commits
- *       change in 2.1).
- *   <li>In version 2.4, Strings are now written as true UTF-8 byte sequence, not Java's modified
- *       UTF-8. See <a href="http://issues.apache.org/jira/browse/LUCENE-510">LUCENE-510</a> for
- *       details.
- *   <li>In version 2.9, an optional opaque Map&lt;String,String&gt; CommitUserData may be passed to
- *       IndexWriter's commit methods (and later retrieved), which is recorded in the segments_N
- *       file. See <a href="http://issues.apache.org/jira/browse/LUCENE-1382">LUCENE-1382</a> for
- *       details. Also, diagnostics were added to each segment written recording details about why
- *       it was written (due to flush, merge; which OS/JRE was used; etc.). See issue <a
- *       href="http://issues.apache.org/jira/browse/LUCENE-1654">LUCENE-1654</a> for details.
- *   <li>In version 3.0, compressed fields are no longer written to the index (they can still be
- *       read, but on merge the new segment will write them, uncompressed). See issue <a
- *       href="http://issues.apache.org/jira/browse/LUCENE-1960">LUCENE-1960</a> for details.
- *   <li>In version 3.1, segments records the code version that created them. See <a
- *       href="http://issues.apache.org/jira/browse/LUCENE-2720">LUCENE-2720</a> for details.
- *       Additionally segments track explicitly whether or not they have term vectors. See <a
- *       href="http://issues.apache.org/jira/browse/LUCENE-2811">LUCENE-2811</a> for details.
- *   <li>In version 3.2, numeric fields are written as natively to stored fields file, previously
- *       they were stored in text format only.
- *   <li>In version 3.4, fields can omit position data while still indexing term frequencies.
- *   <li>In version 4.0, the format of the inverted index became extensible via the {@link
- *       org.apache.lucene.codecs.Codec Codec} api. Fast per-document storage ({@code DocValues})
- *       was introduced. Normalization factors need no longer be a single byte, they can be any
- *       {@link org.apache.lucene.index.NumericDocValues NumericDocValues}. Terms need not be
- *       unicode strings, they can be any byte sequence. Term offsets can optionally be indexed into
- *       the postings lists. Payloads can be stored in the term vectors.
- *   <li>In version 4.1, the format of the postings list changed to use either of FOR compression or
- *       variable-byte encoding, depending upon the frequency of the term. Terms appearing only once
- *       were changed to inline directly into the term dictionary. Stored fields are compressed by
- *       default.
- *   <li>In version 4.2, term vectors are compressed by default. DocValues has a new multi-valued
- *       type (SortedSet), that can be used for faceting/grouping/joining on multi-valued fields.
- *   <li>In version 4.5, DocValues were extended to explicitly represent missing values.
- *   <li>In version 4.6, FieldInfos were extended to support per-field DocValues generation, to
- *       allow updating NumericDocValues fields.
- *   <li>In version 4.8, checksum footers were added to the end of each index file for improved data
- *       integrity. Specifically, the last 8 bytes of every index file contain the zlib-crc32
- *       checksum of the file.
- *   <li>In version 4.9, DocValues has a new multi-valued numeric type (SortedNumeric) that is
- *       suitable for faceting/sorting/analytics.
- *   <li>In version 5.4, DocValues have been improved to store more information on disk: addresses
- *       for binary fields and ord indexes for multi-valued fields.
- *   <li>In version 6.0, Points were added, for multi-dimensional range/distance search.
- *   <li>In version 6.2, new Segment info format that reads/writes the index sort, to support index
- *       sorting.
- *   <li>In version 7.0, DocValues have been improved to better support sparse doc values thanks to
- *       an iterator API.
- *   <li>In version 8.0, postings have been enhanced to record, for each block of doc ids, the (term
- *       freq, normalization factor) pairs that may trigger the maximum score of the block. This
- *       information is recorded alongside skip data in order to be able to skip blocks of doc ids
- *       if they may not produce high enough scores. Additionally doc values and norms has been
- *       extended with jump-tables to make access O(1) instead of O(n), where n is the number of
- *       elements to skip when advancing in the data.
- *   <li>In version 8.4, postings, positions, offsets and payload lengths have move to a more
- *       performant encoding that is vectorized.
- *   <li>In version 8.6, index sort serialization is delegated to the sorts themselves, to allow
- *       user-defined sorts to be used
- *   <li>In version 8.7, stored fields compression became adaptive to better handle documents with
- *       smaller stored fields.
- *   <li>In version 9.0, vector-valued fields were added.
- * </ul>
- *
- * <a id="Limitations"></a>
- *
- * <h3>Limitations</h3>
- *
- * <div>
- *
- * <p>Lucene uses a Java <code>int</code> to refer to document numbers, and the index file format
- * uses an <code>Int32</code> on-disk to store document numbers. This is a limitation of both the
- * index file format and the current implementation. Eventually these should be replaced with either
- * <code>UInt64</code> values, or better yet, {@link org.apache.lucene.store.DataOutput#writeVInt
- * VInt} values which have no limit. </div>
- */
+/** Lucene 9.0 file format. */
 package org.apache.lucene.codecs.lucene90;
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90Codec.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene91/Lucene91Codec.java
similarity index 84%
rename from lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90Codec.java
rename to lucene/core/src/java/org/apache/lucene/codecs/lucene91/Lucene91Codec.java
index 97b8c1b..2c289a8 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90Codec.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene91/Lucene91Codec.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.lucene.codecs.lucene90;
+package org.apache.lucene.codecs.lucene91;
 
 import java.util.Objects;
 import org.apache.lucene.codecs.Codec;
@@ -30,19 +30,29 @@ import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.SegmentInfoFormat;
 import org.apache.lucene.codecs.StoredFieldsFormat;
 import org.apache.lucene.codecs.TermVectorsFormat;
+import org.apache.lucene.codecs.lucene90.Lucene90CompoundFormat;
+import org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat;
+import org.apache.lucene.codecs.lucene90.Lucene90FieldInfosFormat;
+import org.apache.lucene.codecs.lucene90.Lucene90LiveDocsFormat;
+import org.apache.lucene.codecs.lucene90.Lucene90NormsFormat;
+import org.apache.lucene.codecs.lucene90.Lucene90PointsFormat;
+import org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat;
+import org.apache.lucene.codecs.lucene90.Lucene90SegmentInfoFormat;
+import org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat;
+import org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat;
 import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
 import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
 import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
 
 /**
- * Implements the Lucene 9.0 index format
+ * Implements the Lucene 9.1 index format
  *
  * <p>If you want to reuse functionality of this codec in another codec, extend {@link FilterCodec}.
  *
- * @see org.apache.lucene.codecs.lucene90 package documentation for file format details.
+ * @see org.apache.lucene.codecs.lucene91 package documentation for file format details.
  * @lucene.experimental
  */
-public class Lucene90Codec extends Codec {
+public class Lucene91Codec extends Codec {
 
   /** Configuration option for the codec. */
   public enum Mode {
@@ -70,7 +80,7 @@ public class Lucene90Codec extends Codec {
       new PerFieldPostingsFormat() {
         @Override
         public PostingsFormat getPostingsFormatForField(String field) {
-          return Lucene90Codec.this.getPostingsFormatForField(field);
+          return Lucene91Codec.this.getPostingsFormatForField(field);
         }
       };
 
@@ -79,7 +89,7 @@ public class Lucene90Codec extends Codec {
       new PerFieldDocValuesFormat() {
         @Override
         public DocValuesFormat getDocValuesFormatForField(String field) {
-          return Lucene90Codec.this.getDocValuesFormatForField(field);
+          return Lucene91Codec.this.getDocValuesFormatForField(field);
         }
       };
 
@@ -88,14 +98,14 @@ public class Lucene90Codec extends Codec {
       new PerFieldKnnVectorsFormat() {
         @Override
         public KnnVectorsFormat getKnnVectorsFormatForField(String field) {
-          return Lucene90Codec.this.getKnnVectorsFormatForField(field);
+          return Lucene91Codec.this.getKnnVectorsFormatForField(field);
         }
       };
 
   private final StoredFieldsFormat storedFieldsFormat;
 
   /** Instantiates a new codec. */
-  public Lucene90Codec() {
+  public Lucene91Codec() {
     this(Mode.BEST_SPEED);
   }
 
@@ -104,13 +114,13 @@ public class Lucene90Codec extends Codec {
    *
    * @param mode stored fields compression mode to use for newly flushed/merged segments.
    */
-  public Lucene90Codec(Mode mode) {
-    super("Lucene90");
+  public Lucene91Codec(Mode mode) {
+    super("Lucene91");
     this.storedFieldsFormat =
         new Lucene90StoredFieldsFormat(Objects.requireNonNull(mode).storedMode);
     this.defaultPostingsFormat = new Lucene90PostingsFormat();
     this.defaultDVFormat = new Lucene90DocValuesFormat();
-    this.defaultKnnVectorsFormat = new Lucene90HnswVectorsFormat();
+    this.defaultKnnVectorsFormat = new Lucene91HnswVectorsFormat();
   }
 
   @Override
@@ -186,7 +196,7 @@ public class Lucene90Codec extends Codec {
   /**
    * Returns the vectors format that should be used for writing new segments of <code>field</code>
    *
-   * <p>The default implementation always returns "Lucene90".
+   * <p>The default implementation always returns "Lucene91".
    *
    * <p><b>WARNING:</b> if you subclass, you are responsible for index backwards compatibility:
    * future version of Lucene are only guaranteed to be able to read the default implementation.
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90HnswVectorsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene91/Lucene91HnswVectorsFormat.java
similarity index 63%
rename from lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90HnswVectorsFormat.java
rename to lucene/core/src/java/org/apache/lucene/codecs/lucene91/Lucene91HnswVectorsFormat.java
index 90875d9..2e3fc6c 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90HnswVectorsFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene91/Lucene91HnswVectorsFormat.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.lucene.codecs.lucene90;
+package org.apache.lucene.codecs.lucene91;
 
 import java.io.IOException;
 import org.apache.lucene.codecs.KnnVectorsFormat;
@@ -35,14 +35,23 @@ import org.apache.lucene.util.hnsw.HnswGraph;
  * <p>This file stores all the floating-point vector data ordered by field, document ordinal, and
  * vector dimension. The floats are stored in little-endian byte order.
  *
- * <h2>.vex (vector index) file</h2>
+ * <h2>.vex (vector index)</h2>
  *
- * <p>Stores graphs connecting the documents for each field. For each document having a vector for a
- * given field, this is stored as:
+ * <p>Stores graphs connecting the documents for each field organized as a list of nodes' neighbours
+ * as following:
  *
  * <ul>
- *   <li><b>[int32]</b> the number of neighbor nodes
- *   <li><b>array[vint]</b> the neighbor ordinals, delta-encoded (initially subtracting -1)
+ *   <li>For each level:
+ *       <ul>
+ *         <li>For each node:
+ *             <ul>
+ *               <li><b>[int32]</b> the number of neighbor nodes
+ *               <li><b>array[int32]</b> the neighbor ordinals
+ *               <li><b>array[int32]</b> padding from empty integers if the number of neighbors less
+ *                   than the maximum number of connections (maxConn). Padding is equal to
+ *                   ((maxConn-the number of neighbours) * 4) bytes.
+ *             </ul>
+ *       </ul>
  * </ul>
  *
  * <h2>.vem (vector metadata) file</h2>
@@ -59,17 +68,23 @@ import org.apache.lucene.util.hnsw.HnswGraph;
  *   <li><b>[int]</b> dimension of this field's vectors
  *   <li><b>[int]</b> the number of documents having values for this field
  *   <li><b>array[vint]</b> the docids of documents having vectors, in order
- *   <li><b>array[vlong]</b> for each document having a vector, the offset (delta-encoded relative
- *       to the previous document) of its entry in the .vex file
+ *   <li><b>[int]</b> the maximum number of connections (neigbours) that each node can have
+ *   <li><b>[int]</b> number of levels in the graph
+ *   <li>Graph nodes by level. For each level
+ *       <ul>
+ *         <li><b>[int]</b> the number of nodes on this level
+ *         <li><b>array[vint]</b> for levels greater than 0 list of nodes on this level, stored as
+ *             the the level 0th nodes ordinals.
+ *       </ul>
  * </ul>
  *
  * @lucene.experimental
  */
-public final class Lucene90HnswVectorsFormat extends KnnVectorsFormat {
+public final class Lucene91HnswVectorsFormat extends KnnVectorsFormat {
 
-  static final String META_CODEC_NAME = "Lucene90HnswVectorsFormatMeta";
-  static final String VECTOR_DATA_CODEC_NAME = "Lucene90HnswVectorsFormatData";
-  static final String VECTOR_INDEX_CODEC_NAME = "Lucene90HnswVectorsFormatIndex";
+  static final String META_CODEC_NAME = "Lucene91HnswVectorsFormatMeta";
+  static final String VECTOR_DATA_CODEC_NAME = "Lucene91HnswVectorsFormatData";
+  static final String VECTOR_INDEX_CODEC_NAME = "Lucene91HnswVectorsFormatIndex";
   static final String META_EXTENSION = "vem";
   static final String VECTOR_DATA_EXTENSION = "vec";
   static final String VECTOR_INDEX_EXTENSION = "vex";
@@ -77,45 +92,49 @@ public final class Lucene90HnswVectorsFormat extends KnnVectorsFormat {
   static final int VERSION_START = 0;
   static final int VERSION_CURRENT = VERSION_START;
 
+  /** Default number of maximum connections per node */
   public static final int DEFAULT_MAX_CONN = 16;
+  /**
+   * Default number of the size of the queue maintained while searching during a graph construction.
+   */
   public static final int DEFAULT_BEAM_WIDTH = 100;
 
   /**
    * Controls how many of the nearest neighbor candidates are connected to the new node. Defaults to
-   * {@link Lucene90HnswVectorsFormat#DEFAULT_MAX_CONN}. See {@link HnswGraph} for more details.
+   * {@link Lucene91HnswVectorsFormat#DEFAULT_MAX_CONN}. See {@link HnswGraph} for more details.
    */
   private final int maxConn;
 
   /**
    * The number of candidate neighbors to track while searching the graph for each newly inserted
-   * node. Defaults to to {@link Lucene90HnswVectorsFormat#DEFAULT_BEAM_WIDTH}. See {@link
+   * node. Defaults to to {@link Lucene91HnswVectorsFormat#DEFAULT_BEAM_WIDTH}. See {@link
    * HnswGraph} for details.
    */
   private final int beamWidth;
 
-  public Lucene90HnswVectorsFormat() {
+  public Lucene91HnswVectorsFormat() {
     this(DEFAULT_MAX_CONN, DEFAULT_BEAM_WIDTH);
   }
 
-  public Lucene90HnswVectorsFormat(int maxConn, int beamWidth) {
-    super("Lucene90HnswVectorsFormat");
+  public Lucene91HnswVectorsFormat(int maxConn, int beamWidth) {
+    super("Lucene91HnswVectorsFormat");
     this.maxConn = maxConn;
     this.beamWidth = beamWidth;
   }
 
   @Override
   public KnnVectorsWriter fieldsWriter(SegmentWriteState state) throws IOException {
-    return new Lucene90HnswVectorsWriter(state, maxConn, beamWidth);
+    return new Lucene91HnswVectorsWriter(state, maxConn, beamWidth);
   }
 
   @Override
   public KnnVectorsReader fieldsReader(SegmentReadState state) throws IOException {
-    return new Lucene90HnswVectorsReader(state);
+    return new Lucene91HnswVectorsReader(state);
   }
 
   @Override
   public String toString() {
-    return "Lucene90HnswVectorsFormat(name = Lucene90HnswVectorsFormat, maxConn = "
+    return "Lucene91HnswVectorsFormat(name = Lucene91HnswVectorsFormat, maxConn = "
         + maxConn
         + ", beamWidth="
         + beamWidth
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90HnswVectorsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene91/Lucene91HnswVectorsReader.java
similarity index 77%
rename from lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90HnswVectorsReader.java
rename to lucene/core/src/java/org/apache/lucene/codecs/lucene91/Lucene91HnswVectorsReader.java
index bb62ab9..92135f4 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90HnswVectorsReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene91/Lucene91HnswVectorsReader.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.lucene.codecs.lucene90;
+package org.apache.lucene.codecs.lucene91;
 
 import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
 
@@ -24,7 +24,6 @@ import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
-import java.util.SplittableRandom;
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.KnnVectorsReader;
 import org.apache.lucene.index.CorruptIndexException;
@@ -55,48 +54,42 @@ import org.apache.lucene.util.hnsw.NeighborQueue;
  *
  * @lucene.experimental
  */
-public final class Lucene90HnswVectorsReader extends KnnVectorsReader {
+public final class Lucene91HnswVectorsReader extends KnnVectorsReader {
 
   private final FieldInfos fieldInfos;
   private final Map<String, FieldEntry> fields = new HashMap<>();
   private final IndexInput vectorData;
   private final IndexInput vectorIndex;
-  private final long checksumSeed;
 
-  Lucene90HnswVectorsReader(SegmentReadState state) throws IOException {
+  Lucene91HnswVectorsReader(SegmentReadState state) throws IOException {
     this.fieldInfos = state.fieldInfos;
-
     int versionMeta = readMetadata(state);
-    long[] checksumRef = new long[1];
     boolean success = false;
     try {
       vectorData =
           openDataInput(
               state,
               versionMeta,
-              Lucene90HnswVectorsFormat.VECTOR_DATA_EXTENSION,
-              Lucene90HnswVectorsFormat.VECTOR_DATA_CODEC_NAME,
-              checksumRef);
+              Lucene91HnswVectorsFormat.VECTOR_DATA_EXTENSION,
+              Lucene91HnswVectorsFormat.VECTOR_DATA_CODEC_NAME);
       vectorIndex =
           openDataInput(
               state,
               versionMeta,
-              Lucene90HnswVectorsFormat.VECTOR_INDEX_EXTENSION,
-              Lucene90HnswVectorsFormat.VECTOR_INDEX_CODEC_NAME,
-              checksumRef);
+              Lucene91HnswVectorsFormat.VECTOR_INDEX_EXTENSION,
+              Lucene91HnswVectorsFormat.VECTOR_INDEX_CODEC_NAME);
       success = true;
     } finally {
       if (success == false) {
         IOUtils.closeWhileHandlingException(this);
       }
     }
-    checksumSeed = checksumRef[0];
   }
 
   private int readMetadata(SegmentReadState state) throws IOException {
     String metaFileName =
         IndexFileNames.segmentFileName(
-            state.segmentInfo.name, state.segmentSuffix, Lucene90HnswVectorsFormat.META_EXTENSION);
+            state.segmentInfo.name, state.segmentSuffix, Lucene91HnswVectorsFormat.META_EXTENSION);
     int versionMeta = -1;
     try (ChecksumIndexInput meta = state.directory.openChecksumInput(metaFileName, state.context)) {
       Throwable priorE = null;
@@ -104,9 +97,9 @@ public final class Lucene90HnswVectorsReader extends KnnVectorsReader {
         versionMeta =
             CodecUtil.checkIndexHeader(
                 meta,
-                Lucene90HnswVectorsFormat.META_CODEC_NAME,
-                Lucene90HnswVectorsFormat.VERSION_START,
-                Lucene90HnswVectorsFormat.VERSION_CURRENT,
+                Lucene91HnswVectorsFormat.META_CODEC_NAME,
+                Lucene91HnswVectorsFormat.VERSION_START,
+                Lucene91HnswVectorsFormat.VERSION_CURRENT,
                 state.segmentInfo.getId(),
                 state.segmentSuffix);
         readFields(meta, state.fieldInfos);
@@ -120,11 +113,7 @@ public final class Lucene90HnswVectorsReader extends KnnVectorsReader {
   }
 
   private static IndexInput openDataInput(
-      SegmentReadState state,
-      int versionMeta,
-      String fileExtension,
-      String codecName,
-      long[] checksumRef)
+      SegmentReadState state, int versionMeta, String fileExtension, String codecName)
       throws IOException {
     String fileName =
         IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, fileExtension);
@@ -133,8 +122,8 @@ public final class Lucene90HnswVectorsReader extends KnnVectorsReader {
         CodecUtil.checkIndexHeader(
             in,
             codecName,
-            Lucene90HnswVectorsFormat.VERSION_START,
-            Lucene90HnswVectorsFormat.VERSION_CURRENT,
+            Lucene91HnswVectorsFormat.VERSION_START,
+            Lucene91HnswVectorsFormat.VERSION_CURRENT,
             state.segmentInfo.getId(),
             state.segmentSuffix);
     if (versionMeta != versionVectorData) {
@@ -147,7 +136,7 @@ public final class Lucene90HnswVectorsReader extends KnnVectorsReader {
               + versionVectorData,
           in);
     }
-    checksumRef[0] = CodecUtil.retrieveChecksum(in);
+    CodecUtil.retrieveChecksum(in);
     return in;
   }
 
@@ -157,7 +146,6 @@ public final class Lucene90HnswVectorsReader extends KnnVectorsReader {
       if (info == null) {
         throw new CorruptIndexException("Invalid field number: " + fieldNumber, meta);
       }
-
       FieldEntry fieldEntry = readField(meta);
       validateFieldEntry(info, fieldEntry);
       fields.put(info.name, fieldEntry);
@@ -207,7 +195,7 @@ public final class Lucene90HnswVectorsReader extends KnnVectorsReader {
 
   @Override
   public long ramBytesUsed() {
-    long totalBytes = RamUsageEstimator.shallowSizeOfInstance(Lucene90HnswVectorsReader.class);
+    long totalBytes = RamUsageEstimator.shallowSizeOfInstance(Lucene91HnswVectorsFormat.class);
     totalBytes +=
         RamUsageEstimator.sizeOfMap(
             fields, RamUsageEstimator.shallowSizeOfInstance(FieldEntry.class));
@@ -242,17 +230,14 @@ public final class Lucene90HnswVectorsReader extends KnnVectorsReader {
 
     OffHeapVectorValues vectorValues = getOffHeapVectorValues(fieldEntry);
     // use a seed that is fixed for the index so we get reproducible results for the same query
-    final SplittableRandom random = new SplittableRandom(checksumSeed);
     NeighborQueue results =
         HnswGraph.search(
             target,
             k,
-            k,
             vectorValues,
             fieldEntry.similarityFunction,
             getGraphValues(fieldEntry),
-            getAcceptOrds(acceptDocs, fieldEntry),
-            random);
+            getAcceptOrds(acceptDocs, fieldEntry));
     int i = 0;
     ScoreDoc[] scoreDocs = new ScoreDoc[Math.min(results.size(), k)];
     while (results.size() > 0) {
@@ -291,13 +276,14 @@ public final class Lucene90HnswVectorsReader extends KnnVectorsReader {
     };
   }
 
+  /** Get knn graph values; used for testing */
   public KnnGraphValues getGraphValues(String field) throws IOException {
     FieldInfo info = fieldInfos.fieldInfo(field);
     if (info == null) {
       throw new IllegalArgumentException("No such field '" + field + "'");
     }
     FieldEntry entry = fields.get(field);
-    if (entry != null && entry.indexDataLength > 0) {
+    if (entry != null && entry.vectorIndexLength > 0) {
       return getGraphValues(entry);
     } else {
       return KnnGraphValues.EMPTY;
@@ -306,7 +292,7 @@ public final class Lucene90HnswVectorsReader extends KnnVectorsReader {
 
   private KnnGraphValues getGraphValues(FieldEntry entry) throws IOException {
     IndexInput bytesSlice =
-        vectorIndex.slice("graph-data", entry.indexDataOffset, entry.indexDataLength);
+        vectorIndex.slice("graph-data", entry.vectorIndexOffset, entry.vectorIndexLength);
     return new IndexedKnnGraphReader(entry, bytesSlice);
   }
 
@@ -317,22 +303,25 @@ public final class Lucene90HnswVectorsReader extends KnnVectorsReader {
 
   private static class FieldEntry {
 
-    final int dimension;
     final VectorSimilarityFunction similarityFunction;
-
     final long vectorDataOffset;
     final long vectorDataLength;
-    final long indexDataOffset;
-    final long indexDataLength;
+    final long vectorIndexOffset;
+    final long vectorIndexLength;
+    final int maxConn;
+    final int numLevels;
+    final int dimension;
     final int[] ordToDoc;
-    final long[] ordOffsets;
+    final int[][] nodesByLevel;
+    // for each level the start offsets in vectorIndex file from where to read neighbours
+    final long[] graphOffsetsByLevel;
 
     FieldEntry(DataInput input, VectorSimilarityFunction similarityFunction) throws IOException {
       this.similarityFunction = similarityFunction;
       vectorDataOffset = input.readVLong();
       vectorDataLength = input.readVLong();
-      indexDataOffset = input.readVLong();
-      indexDataLength = input.readVLong();
+      vectorIndexOffset = input.readVLong();
+      vectorIndexLength = input.readVLong();
       dimension = input.readInt();
       int size = input.readInt();
       ordToDoc = new int[size];
@@ -340,11 +329,36 @@ public final class Lucene90HnswVectorsReader extends KnnVectorsReader {
         int doc = input.readVInt();
         ordToDoc[i] = doc;
       }
-      ordOffsets = new long[size()];
-      long offset = 0;
-      for (int i = 0; i < ordOffsets.length; i++) {
-        offset += input.readVLong();
-        ordOffsets[i] = offset;
+
+      // read nodes by level
+      maxConn = input.readInt();
+      numLevels = input.readInt();
+      nodesByLevel = new int[numLevels][];
+      for (int level = 0; level < numLevels; level++) {
+        int numNodesOnLevel = input.readInt();
+        if (level == 0) {
+          // we don't store nodes for level 0th, as this level contains all nodes
+          assert numNodesOnLevel == size;
+          nodesByLevel[0] = null;
+        } else {
+          nodesByLevel[level] = new int[numNodesOnLevel];
+          for (int i = 0; i < numNodesOnLevel; i++) {
+            nodesByLevel[level][i] = input.readVInt();
+          }
+        }
+      }
+
+      // calculate for each level the start offsets in vectorIndex file from where to read
+      // neighbours
+      graphOffsetsByLevel = new long[numLevels];
+      for (int level = 0; level < numLevels; level++) {
+        if (level == 0) {
+          graphOffsetsByLevel[level] = 0;
+        } else {
+          int numNodesOnPrevLevel = level == 1 ? size : nodesByLevel[level - 1].length;
+          graphOffsetsByLevel[level] =
+              graphOffsetsByLevel[level - 1] + (1 + maxConn) * Integer.BYTES * numNodesOnPrevLevel;
+        }
       }
     }
 
@@ -373,7 +387,6 @@ public final class Lucene90HnswVectorsReader extends KnnVectorsReader {
       this.dimension = dimension;
       this.ordToDoc = ordToDoc;
       this.dataIn = dataIn;
-
       byteSize = Float.BYTES * dimension;
       byteBuffer = ByteBuffer.allocate(byteSize);
       value = new float[dimension];
@@ -467,22 +480,38 @@ public final class Lucene90HnswVectorsReader extends KnnVectorsReader {
   /** Read the nearest-neighbors graph from the index input */
   private static final class IndexedKnnGraphReader extends KnnGraphValues {
 
-    final FieldEntry entry;
     final IndexInput dataIn;
+    final int[][] nodesByLevel;
+    final long[] graphOffsetsByLevel;
+    final int numLevels;
+    final int entryNode;
+    final int size;
+    final long bytesForConns;
 
     int arcCount;
     int arcUpTo;
     int arc;
 
     IndexedKnnGraphReader(FieldEntry entry, IndexInput dataIn) {
-      this.entry = entry;
       this.dataIn = dataIn;
+      this.nodesByLevel = entry.nodesByLevel;
+      this.numLevels = entry.numLevels;
+      this.entryNode = numLevels > 1 ? nodesByLevel[numLevels - 1][0] : 0;
+      this.size = entry.size();
+      this.graphOffsetsByLevel = entry.graphOffsetsByLevel;
+      this.bytesForConns = ((long) entry.maxConn + 1) * Integer.BYTES;
     }
 
     @Override
-    public void seek(int targetOrd) throws IOException {
+    public void seek(int level, int targetOrd) throws IOException {
+      int targetIndex =
+          level == 0
+              ? targetOrd
+              : Arrays.binarySearch(nodesByLevel[level], 0, nodesByLevel[level].length, targetOrd);
+      assert targetIndex >= 0;
+      long graphDataOffset = graphOffsetsByLevel[level] + targetIndex * bytesForConns;
       // unsafe; no bounds checking
-      dataIn.seek(entry.ordOffsets[targetOrd]);
+      dataIn.seek(graphDataOffset);
       arcCount = dataIn.readInt();
       arc = -1;
       arcUpTo = 0;
@@ -490,7 +519,7 @@ public final class Lucene90HnswVectorsReader extends KnnVectorsReader {
 
     @Override
     public int size() {
-      return entry.size();
+      return size;
     }
 
     @Override
@@ -499,8 +528,27 @@ public final class Lucene90HnswVectorsReader extends KnnVectorsReader {
         return NO_MORE_DOCS;
       }
       ++arcUpTo;
-      arc += dataIn.readVInt();
+      arc = dataIn.readInt();
       return arc;
     }
+
+    @Override
+    public int numLevels() throws IOException {
+      return numLevels;
+    }
+
+    @Override
+    public int entryNode() throws IOException {
+      return entryNode;
+    }
+
+    @Override
+    public NodesIterator getNodesOnLevel(int level) {
+      if (level == 0) {
+        return new NodesIterator(size());
+      } else {
+        return new NodesIterator(nodesByLevel[level], nodesByLevel[level].length);
+      }
+    }
   }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90HnswVectorsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene91/Lucene91HnswVectorsWriter.java
similarity index 72%
rename from lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90HnswVectorsWriter.java
rename to lucene/core/src/java/org/apache/lucene/codecs/lucene91/Lucene91HnswVectorsWriter.java
index 0fbd1be..f56f1d7 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90HnswVectorsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene91/Lucene91HnswVectorsWriter.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.lucene.codecs.lucene90;
+package org.apache.lucene.codecs.lucene91;
 
 import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
 
@@ -26,6 +26,7 @@ import org.apache.lucene.codecs.KnnVectorsReader;
 import org.apache.lucene.codecs.KnnVectorsWriter;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.KnnGraphValues.NodesIterator;
 import org.apache.lucene.index.RandomAccessVectorValuesProducer;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.VectorSimilarityFunction;
@@ -44,7 +45,7 @@ import org.apache.lucene.util.hnsw.NeighborArray;
  *
  * @lucene.experimental
  */
-public final class Lucene90HnswVectorsWriter extends KnnVectorsWriter {
+public final class Lucene91HnswVectorsWriter extends KnnVectorsWriter {
 
   private final SegmentWriteState segmentWriteState;
   private final IndexOutput meta, vectorData, vectorIndex;
@@ -53,7 +54,7 @@ public final class Lucene90HnswVectorsWriter extends KnnVectorsWriter {
   private final int beamWidth;
   private boolean finished;
 
-  Lucene90HnswVectorsWriter(SegmentWriteState state, int maxConn, int beamWidth)
+  Lucene91HnswVectorsWriter(SegmentWriteState state, int maxConn, int beamWidth)
       throws IOException {
     this.maxConn = maxConn;
     this.beamWidth = beamWidth;
@@ -63,19 +64,19 @@ public final class Lucene90HnswVectorsWriter extends KnnVectorsWriter {
 
     String metaFileName =
         IndexFileNames.segmentFileName(
-            state.segmentInfo.name, state.segmentSuffix, Lucene90HnswVectorsFormat.META_EXTENSION);
+            state.segmentInfo.name, state.segmentSuffix, Lucene91HnswVectorsFormat.META_EXTENSION);
 
     String vectorDataFileName =
         IndexFileNames.segmentFileName(
             state.segmentInfo.name,
             state.segmentSuffix,
-            Lucene90HnswVectorsFormat.VECTOR_DATA_EXTENSION);
+            Lucene91HnswVectorsFormat.VECTOR_DATA_EXTENSION);
 
     String indexDataFileName =
         IndexFileNames.segmentFileName(
             state.segmentInfo.name,
             state.segmentSuffix,
-            Lucene90HnswVectorsFormat.VECTOR_INDEX_EXTENSION);
+            Lucene91HnswVectorsFormat.VECTOR_INDEX_EXTENSION);
 
     boolean success = false;
     try {
@@ -85,20 +86,20 @@ public final class Lucene90HnswVectorsWriter extends KnnVectorsWriter {
 
       CodecUtil.writeIndexHeader(
           meta,
-          Lucene90HnswVectorsFormat.META_CODEC_NAME,
-          Lucene90HnswVectorsFormat.VERSION_CURRENT,
+          Lucene91HnswVectorsFormat.META_CODEC_NAME,
+          Lucene91HnswVectorsFormat.VERSION_CURRENT,
           state.segmentInfo.getId(),
           state.segmentSuffix);
       CodecUtil.writeIndexHeader(
           vectorData,
-          Lucene90HnswVectorsFormat.VECTOR_DATA_CODEC_NAME,
-          Lucene90HnswVectorsFormat.VERSION_CURRENT,
+          Lucene91HnswVectorsFormat.VECTOR_DATA_CODEC_NAME,
+          Lucene91HnswVectorsFormat.VERSION_CURRENT,
           state.segmentInfo.getId(),
           state.segmentSuffix);
       CodecUtil.writeIndexHeader(
           vectorIndex,
-          Lucene90HnswVectorsFormat.VECTOR_INDEX_CODEC_NAME,
-          Lucene90HnswVectorsFormat.VERSION_CURRENT,
+          Lucene91HnswVectorsFormat.VECTOR_INDEX_CODEC_NAME,
+          Lucene91HnswVectorsFormat.VERSION_CURRENT,
           state.segmentInfo.getId(),
           state.segmentSuffix);
       success = true;
@@ -133,24 +134,17 @@ public final class Lucene90HnswVectorsWriter extends KnnVectorsWriter {
               tempVectorData.getName(), segmentWriteState.context);
       vectorData.copyBytes(vectorDataInput, vectorDataInput.length() - CodecUtil.footerLength());
       CodecUtil.retrieveChecksum(vectorDataInput);
+      long vectorDataLength = vectorData.getFilePointer() - vectorDataOffset;
 
+      long vectorIndexOffset = vectorIndex.getFilePointer();
       // build the graph using the temporary vector data
-      Lucene90HnswVectorsReader.OffHeapVectorValues offHeapVectors =
-          new Lucene90HnswVectorsReader.OffHeapVectorValues(
+      Lucene91HnswVectorsReader.OffHeapVectorValues offHeapVectors =
+          new Lucene91HnswVectorsReader.OffHeapVectorValues(
               vectors.dimension(), docIds, vectorDataInput);
-
-      long[] offsets = new long[docIds.length];
-      long vectorIndexOffset = vectorIndex.getFilePointer();
-      writeGraph(
-          vectorIndex,
-          offHeapVectors,
-          fieldInfo.getVectorSimilarityFunction(),
-          vectorIndexOffset,
-          offsets,
-          maxConn,
-          beamWidth);
-
-      long vectorDataLength = vectorData.getFilePointer() - vectorDataOffset;
+      HnswGraph graph =
+          offHeapVectors.size() == 0
+              ? null
+              : writeGraph(offHeapVectors, fieldInfo.getVectorSimilarityFunction());
       long vectorIndexLength = vectorIndex.getFilePointer() - vectorIndexOffset;
       writeMeta(
           fieldInfo,
@@ -158,8 +152,8 @@ public final class Lucene90HnswVectorsWriter extends KnnVectorsWriter {
           vectorDataLength,
           vectorIndexOffset,
           vectorIndexLength,
-          docIds);
-      writeGraphOffsets(meta, offsets);
+          docIds,
+          graph);
       success = true;
     } finally {
       IOUtils.close(vectorDataInput);
@@ -200,68 +194,80 @@ public final class Lucene90HnswVectorsWriter extends KnnVectorsWriter {
       FieldInfo field,
       long vectorDataOffset,
       long vectorDataLength,
-      long indexDataOffset,
-      long indexDataLength,
-      int[] docIds)
+      long vectorIndexOffset,
+      long vectorIndexLength,
+      int[] docIds,
+      HnswGraph graph)
       throws IOException {
     meta.writeInt(field.number);
     meta.writeInt(field.getVectorSimilarityFunction().ordinal());
     meta.writeVLong(vectorDataOffset);
     meta.writeVLong(vectorDataLength);
-    meta.writeVLong(indexDataOffset);
-    meta.writeVLong(indexDataLength);
+    meta.writeVLong(vectorIndexOffset);
+    meta.writeVLong(vectorIndexLength);
     meta.writeInt(field.getVectorDimension());
     meta.writeInt(docIds.length);
     for (int docId : docIds) {
       // TODO: delta-encode, or write as bitset
       meta.writeVInt(docId);
     }
-  }
 
-  private void writeGraphOffsets(IndexOutput out, long[] offsets) throws IOException {
-    long last = 0;
-    for (long offset : offsets) {
-      out.writeVLong(offset - last);
-      last = offset;
+    meta.writeInt(maxConn);
+
+    // write graph nodes on each level
+    if (graph == null) {
+      meta.writeInt(0);
+    } else {
+      meta.writeInt(graph.numLevels());
+      for (int level = 0; level < graph.numLevels(); level++) {
+        NodesIterator nodesOnLevel = graph.getNodesOnLevel(level);
+        meta.writeInt(nodesOnLevel.size()); // number of nodes on a level
+        if (level > 0) {
+          while (nodesOnLevel.hasNext()) {
+            int node = nodesOnLevel.nextInt();
+            meta.writeVInt(node); // list of nodes on a level
+          }
+        }
+      }
     }
   }
 
-  private void writeGraph(
-      IndexOutput graphData,
-      RandomAccessVectorValuesProducer vectorValues,
-      VectorSimilarityFunction similarityFunction,
-      long graphDataOffset,
-      long[] offsets,
-      int maxConn,
-      int beamWidth)
+  private HnswGraph writeGraph(
+      RandomAccessVectorValuesProducer vectorValues, VectorSimilarityFunction similarityFunction)
       throws IOException {
+
+    // build graph
     HnswGraphBuilder hnswGraphBuilder =
         new HnswGraphBuilder(
             vectorValues, similarityFunction, maxConn, beamWidth, HnswGraphBuilder.randSeed);
     hnswGraphBuilder.setInfoStream(segmentWriteState.infoStream);
     HnswGraph graph = hnswGraphBuilder.build(vectorValues.randomAccess());
 
-    for (int ord = 0; ord < offsets.length; ord++) {
-      // write graph
-      offsets[ord] = graphData.getFilePointer() - graphDataOffset;
-
-      NeighborArray neighbors = graph.getNeighbors(ord);
-      int size = neighbors.size();
-
-      // Destructively modify; it's ok we are discarding it after this
-      int[] nodes = neighbors.node();
-      Arrays.sort(nodes, 0, size);
-      graphData.writeInt(size);
-
-      int lastNode = -1; // to make the assertion work?
-      for (int i = 0; i < size; i++) {
-        int node = nodes[i];
-        assert node > lastNode : "nodes out of order: " + lastNode + "," + node;
-        assert node < offsets.length : "node too large: " + node + ">=" + offsets.length;
-        graphData.writeVInt(node - lastNode);
-        lastNode = node;
+    // write vectors' neighbours on each level into the vectorIndex file
+    int countOnLevel0 = graph.size();
+    for (int level = 0; level < graph.numLevels(); level++) {
+      NodesIterator nodesOnLevel = graph.getNodesOnLevel(level);
+      while (nodesOnLevel.hasNext()) {
+        int node = nodesOnLevel.nextInt();
+        NeighborArray neighbors = graph.getNeighbors(level, node);
+        int size = neighbors.size();
+        vectorIndex.writeInt(size);
+        // Destructively modify; it's ok we are discarding it after this
+        int[] nnodes = neighbors.node();
+        Arrays.sort(nnodes, 0, size);
+        for (int i = 0; i < size; i++) {
+          int nnode = nnodes[i];
+          assert nnode < countOnLevel0 : "node too large: " + nnode + ">=" + countOnLevel0;
+          vectorIndex.writeInt(nnode);
+        }
+        // if number of connections < maxConn, add bogus values up to maxConn to have predictable
+        // offsets
+        for (int i = size; i < maxConn; i++) {
+          vectorIndex.writeInt(0);
+        }
       }
     }
+    return graph;
   }
 
   @Override
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/package-info.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene91/package-info.java
similarity index 98%
copy from lucene/core/src/java/org/apache/lucene/codecs/lucene90/package-info.java
copy to lucene/core/src/java/org/apache/lucene/codecs/lucene91/package-info.java
index d8c6828..0460332 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/package-info.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene91/package-info.java
@@ -16,7 +16,7 @@
  */
 
 /**
- * Lucene 9.0 file format.
+ * Lucene 9.1 file format.
  *
  * <h2>Apache Lucene - Index File Formats</h2>
  *
@@ -180,7 +180,7 @@
  *       of files, recording dimensionally indexed fields, to enable fast numeric range filtering
  *       and large numeric values like BigInteger and BigDecimal (1D) and geographic shape
  *       intersection (2D, 3D).
- *   <li>{@link org.apache.lucene.codecs.lucene90.Lucene90HnswVectorsFormat Vector values}. The
+ *   <li>{@link org.apache.lucene.codecs.lucene91.Lucene91HnswVectorsFormat Vector values}. The
  *       vector format stores numeric vectors in a format optimized for random access and
  *       computation, supporting high-dimensional nearest-neighbor search.
  * </ul>
@@ -310,7 +310,7 @@
  * <td>Holds indexed points</td>
  * </tr>
  * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene90.Lucene90HnswVectorsFormat Vector values}</td>
+ * <td>{@link org.apache.lucene.codecs.lucene91.Lucene91HnswVectorsFormat Vector values}</td>
  * <td>.vec, .vem</td>
  * <td>Holds indexed vectors; <code>.vec</code> files contain the raw vector data, and
  * <code>.vem</code> the vector metadata</td>
@@ -402,6 +402,7 @@
  *   <li>In version 8.7, stored fields compression became adaptive to better handle documents with
  *       smaller stored fields.
  *   <li>In version 9.0, vector-valued fields were added.
+ *   <li>In version 9.1, vector-valued fields were modified to add a graph hierarchy.
  * </ul>
  *
  * <a id="Limitations"></a>
@@ -416,4 +417,4 @@
  * <code>UInt64</code> values, or better yet, {@link org.apache.lucene.store.DataOutput#writeVInt
  * VInt} values which have no limit. </div>
  */
-package org.apache.lucene.codecs.lucene90;
+package org.apache.lucene.codecs.lucene91;
diff --git a/lucene/core/src/java/org/apache/lucene/index/KnnGraphValues.java b/lucene/core/src/java/org/apache/lucene/index/KnnGraphValues.java
index f8f175a..07feb2a 100644
--- a/lucene/core/src/java/org/apache/lucene/index/KnnGraphValues.java
+++ b/lucene/core/src/java/org/apache/lucene/index/KnnGraphValues.java
@@ -20,6 +20,8 @@ package org.apache.lucene.index;
 import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
 
 import java.io.IOException;
+import java.util.NoSuchElementException;
+import java.util.PrimitiveIterator;
 
 /**
  * Access to per-document neighbor lists in a (hierarchical) knn search graph.
@@ -32,25 +34,40 @@ public abstract class KnnGraphValues {
   protected KnnGraphValues() {}
 
   /**
-   * Move the pointer to exactly {@code target}, the id of a node in the graph. After this method
+   * Move the pointer to exactly the given {@code level}'s {@code target}. After this method
    * returns, call {@link #nextNeighbor()} to return successive (ordered) connected node ordinals.
    *
-   * @param target must be a valid node in the graph, ie. &ge; 0 and &lt; {@link
+   * @param level level of the graph
+   * @param target ordinal of a node in the graph, must be &ge; 0 and &lt; {@link
    *     VectorValues#size()}.
    */
-  public abstract void seek(int target) throws IOException;
+  public abstract void seek(int level, int target) throws IOException;
 
   /** Returns the number of nodes in the graph */
   public abstract int size();
 
   /**
    * Iterates over the neighbor list. It is illegal to call this method after it returns
-   * NO_MORE_DOCS without calling {@link #seek(int)}, which resets the iterator.
+   * NO_MORE_DOCS without calling {@link #seek(int, int)}, which resets the iterator.
    *
    * @return a node ordinal in the graph, or NO_MORE_DOCS if the iteration is complete.
    */
   public abstract int nextNeighbor() throws IOException;
 
+  /** Returns the number of levels of the graph */
+  public abstract int numLevels() throws IOException;
+
+  /** Returns graph's entry point on the top level * */
+  public abstract int entryNode() throws IOException;
+
+  /**
+   * Get all nodes on a given level as node 0th ordinals
+   *
+   * @param level level for which to get all nodes
+   * @return an iterator over nodes where {@code nextInt} returns a next node on the level
+   */
+  public abstract NodesIterator getNodesOnLevel(int level) throws IOException;
+
   /** Empty graph value */
   public static KnnGraphValues EMPTY =
       new KnnGraphValues() {
@@ -61,11 +78,74 @@ public abstract class KnnGraphValues {
         }
 
         @Override
-        public void seek(int target) {}
+        public void seek(int level, int target) {}
 
         @Override
         public int size() {
           return 0;
         }
+
+        @Override
+        public int numLevels() {
+          return 0;
+        }
+
+        @Override
+        public int entryNode() {
+          return 0;
+        }
+
+        @Override
+        public NodesIterator getNodesOnLevel(int level) {
+          return NodesIterator.EMPTY;
+        }
       };
+
+  /**
+   * Iterator over the graph nodes on a certain level, Iterator also provides the size – the total
+   * number of nodes to be iterated over.
+   */
+  public static final class NodesIterator implements PrimitiveIterator.OfInt {
+    static NodesIterator EMPTY = new NodesIterator(0);
+
+    private final int[] nodes;
+    private final int size;
+    int cur = 0;
+
+    /** Constructor for iterator based on the nodes array up to the size */
+    public NodesIterator(int[] nodes, int size) {
+      assert nodes != null;
+      assert size <= nodes.length;
+      this.nodes = nodes;
+      this.size = size;
+    }
+
+    /** Constructor for iterator based on the size */
+    public NodesIterator(int size) {
+      this.nodes = null;
+      this.size = size;
+    }
+
+    @Override
+    public int nextInt() {
+      if (hasNext() == false) {
+        throw new NoSuchElementException();
+      }
+      if (nodes == null) {
+        return cur++;
+      } else {
+        return nodes[cur++];
+      }
+    }
+
+    @Override
+    public boolean hasNext() {
+      return cur < size;
+    }
+
+    /** The number of elements in this iterator * */
+    public int size() {
+      return size;
+    }
+  }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/util/hnsw/BoundsChecker.java b/lucene/core/src/java/org/apache/lucene/util/hnsw/BoundsChecker.java
index ce15204..9cde17d 100644
--- a/lucene/core/src/java/org/apache/lucene/util/hnsw/BoundsChecker.java
+++ b/lucene/core/src/java/org/apache/lucene/util/hnsw/BoundsChecker.java
@@ -17,22 +17,26 @@
 
 package org.apache.lucene.util.hnsw;
 
-abstract class BoundsChecker {
+/**
+ * A helper class for an hnsw graph that serves as a comparator of the currently set bound value
+ * with a new value.
+ */
+public abstract class BoundsChecker {
 
   float bound;
 
   /** Update the bound if sample is better */
-  abstract void update(float sample);
+  public abstract void update(float sample);
 
   /** Update the bound unconditionally */
-  void set(float sample) {
+  public void set(float sample) {
     bound = sample;
   }
 
   /** @return whether the sample exceeds (is worse than) the bound */
-  abstract boolean check(float sample);
+  public abstract boolean check(float sample);
 
-  static BoundsChecker create(boolean reversed) {
+  public static BoundsChecker create(boolean reversed) {
     if (reversed) {
       return new Min();
     } else {
@@ -40,39 +44,47 @@ abstract class BoundsChecker {
     }
   }
 
-  static class Max extends BoundsChecker {
+  /**
+   * A helper class for an hnsw graph that serves as a comparator of the currently set maximum value
+   * with a new value.
+   */
+  public static class Max extends BoundsChecker {
     Max() {
       bound = Float.NEGATIVE_INFINITY;
     }
 
     @Override
-    void update(float sample) {
+    public void update(float sample) {
       if (sample > bound) {
         bound = sample;
       }
     }
 
     @Override
-    boolean check(float sample) {
+    public boolean check(float sample) {
       return sample < bound;
     }
   }
 
-  static class Min extends BoundsChecker {
+  /**
+   * A helper class for an hnsw graph that serves as a comparator of the currently set minimum value
+   * with a new value.
+   */
+  public static class Min extends BoundsChecker {
 
     Min() {
       bound = Float.POSITIVE_INFINITY;
     }
 
     @Override
-    void update(float sample) {
+    public void update(float sample) {
       if (sample < bound) {
         bound = sample;
       }
     }
 
     @Override
-    boolean check(float sample) {
+    public boolean check(float sample) {
       return sample > bound;
     }
   }
diff --git a/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraph.java b/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraph.java
index 9af1017..6678bb4 100644
--- a/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraph.java
+++ b/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraph.java
@@ -21,32 +21,31 @@ import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
-import java.util.SplittableRandom;
 import org.apache.lucene.index.KnnGraphValues;
 import org.apache.lucene.index.RandomAccessVectorValues;
 import org.apache.lucene.index.VectorSimilarityFunction;
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.SparseFixedBitSet;
 
 /**
- * Navigable Small-world graph. Provides efficient approximate nearest neighbor search for high
- * dimensional vectors. See <a href="https://doi.org/10.1016/j.is.2013.10.006">Approximate nearest
- * neighbor algorithm based on navigable small world graphs [2014]</a> and <a
- * href="https://arxiv.org/abs/1603.09320">this paper [2018]</a> for details.
+ * Hierarchical Navigable Small World graph. Provides efficient approximate nearest neighbor search
+ * for high dimensional vectors. See <a href="https://arxiv.org/abs/1603.09320">Efficient and robust
+ * approximate nearest neighbor search using Hierarchical Navigable Small World graphs [2018]</a>
+ * paper for details.
  *
- * <p>The nomenclature is a bit different here from what's used in those papers:
+ * <p>The nomenclature is a bit different here from what's used in the paper:
  *
  * <h2>Hyperparameters</h2>
  *
  * <ul>
- *   <li><code>numSeed</code> is the equivalent of <code>m</code> in the 2012 paper; it controls the
- *       number of random entry points to sample.
  *   <li><code>beamWidth</code> in {@link HnswGraphBuilder} has the same meaning as <code>efConst
- *       </code> in the 2016 paper. It is the number of nearest neighbor candidates to track while
+ *       </code> in the paper. It is the number of nearest neighbor candidates to track while
  *       searching the graph for each newly inserted node.
- *   <li><code>maxConn</code> has the same meaning as <code>M</code> in the later paper; it controls
- *       how many of the <code>efConst</code> neighbors are connected to the new node
+ *   <li><code>maxConn</code> has the same meaning as <code>M</code> in the paper; it controls how
+ *       many of the <code>efConst</code> neighbors are connected to the new node
  * </ul>
  *
  * <p>Note: The graph may be searched by multiple threads concurrently, but updates are not
@@ -56,75 +55,120 @@ import org.apache.lucene.util.SparseFixedBitSet;
 public final class HnswGraph extends KnnGraphValues {
 
   private final int maxConn;
+  private int numLevels; // the current number of levels in the graph
+  private int entryNode; // the current graph entry node on the top level
 
-  // Each entry lists the top maxConn neighbors of a node. The nodes correspond to vectors added to
-  // HnswBuilder, and the
-  // node values are the ordinals of those vectors.
-  private final List<NeighborArray> graph;
+  // Nodes by level expressed as the level 0's nodes' ordinals.
+  // As level 0 contains all nodes, nodesByLevel.get(0) is null.
+  private final List<int[]> nodesByLevel;
+
+  // graph is a list of graph levels.
+  // Each level is represented as List<NeighborArray> – nodes' connections on this level.
+  // Each entry in the list has the top maxConn neighbors of a node. The nodes correspond to vectors
+  // added to HnswBuilder, and the node values are the ordinals of those vectors.
+  // Thus, on all levels, neighbors expressed as the level 0's nodes' ordinals.
+  private final List<List<NeighborArray>> graph;
 
   // KnnGraphValues iterator members
   private int upto;
   private NeighborArray cur;
 
-  HnswGraph(int maxConn) {
-    graph = new ArrayList<>();
-    // Typically with diversity criteria we see nodes not fully occupied; average fanout seems to be
-    // about 1/2 maxConn. There is some indexing time penalty for under-allocating, but saves RAM
-    graph.add(new NeighborArray(Math.max(32, maxConn / 4)));
+  HnswGraph(int maxConn, int levelOfFirstNode) {
     this.maxConn = maxConn;
+    this.numLevels = levelOfFirstNode + 1;
+    this.graph = new ArrayList<>(numLevels);
+    this.entryNode = 0;
+    for (int i = 0; i < numLevels; i++) {
+      graph.add(new ArrayList<>());
+      // Typically with diversity criteria we see nodes not fully occupied;
+      // average fanout seems to be about 1/2 maxConn.
+      // There is some indexing time penalty for under-allocating, but saves RAM
+      graph.get(i).add(new NeighborArray(Math.max(32, maxConn / 4)));
+    }
+
+    this.nodesByLevel = new ArrayList<>(numLevels);
+    nodesByLevel.add(null); // we don't need this for 0th level, as it contains all nodes
+    for (int l = 1; l < numLevels; l++) {
+      nodesByLevel.add(new int[] {0});
+    }
   }
 
   /**
-   * Searches for the nearest neighbors of a query vector.
+   * Searches HNSW graph for the nearest neighbors of a query vector.
    *
    * @param query search query vector
    * @param topK the number of nodes to be returned
-   * @param numSeed the size of the queue maintained while searching, and controls the number of
-   *     random entry points to sample
    * @param vectors vector values
    * @param graphValues the graph values. May represent the entire graph, or a level in a
    *     hierarchical graph.
    * @param acceptOrds {@link Bits} that represents the allowed document ordinals to match, or
    *     {@code null} if they are all allowed to match.
-   * @param random a source of randomness, used for generating entry points to the graph
    * @return a priority queue holding the closest neighbors found
    */
   public static NeighborQueue search(
       float[] query,
       int topK,
-      int numSeed,
       RandomAccessVectorValues vectors,
       VectorSimilarityFunction similarityFunction,
       KnnGraphValues graphValues,
-      Bits acceptOrds,
-      SplittableRandom random)
+      Bits acceptOrds)
+      throws IOException {
+
+    NeighborQueue results;
+    int[] eps = new int[] {graphValues.entryNode()};
+    for (int level = graphValues.numLevels() - 1; level >= 1; level--) {
+      results = searchLevel(query, 1, level, eps, vectors, similarityFunction, graphValues, null);
+      eps[0] = results.pop();
+    }
+    results =
+        searchLevel(query, topK, 0, eps, vectors, similarityFunction, graphValues, acceptOrds);
+    return results;
+  }
+
+  /**
+   * Searches for the nearest neighbors of a query vector in a given level
+   *
+   * @param query search query vector
+   * @param topK the number of nearest to query results to return
+   * @param level level to search
+   * @param eps the entry points for search at this level expressed as level 0th ordinals
+   * @param vectors vector values
+   * @param similarityFunction similarity function
+   * @param graphValues the graph values
+   * @param acceptOrds {@link Bits} that represents the allowed document ordinals to match, or
+   *     {@code null} if they are all allowed to match.
+   * @return a priority queue holding the closest neighbors found
+   */
+  static NeighborQueue searchLevel(
+      float[] query,
+      int topK,
+      int level,
+      final int[] eps,
+      RandomAccessVectorValues vectors,
+      VectorSimilarityFunction similarityFunction,
+      KnnGraphValues graphValues,
+      Bits acceptOrds)
       throws IOException {
-    int size = graphValues.size();
 
+    int size = graphValues.size();
     // MIN heap, holding the top results
-    NeighborQueue results = new NeighborQueue(numSeed, similarityFunction.reversed);
+    NeighborQueue results = new NeighborQueue(topK, similarityFunction.reversed);
     // MAX heap, from which to pull the candidate nodes
-    NeighborQueue candidates = new NeighborQueue(numSeed, !similarityFunction.reversed);
-
+    NeighborQueue candidates = new NeighborQueue(topK, !similarityFunction.reversed);
     // set of ordinals that have been visited by search on this layer, used to avoid backtracking
     SparseFixedBitSet visited = new SparseFixedBitSet(size);
-    // get initial candidates at random
-    int boundedNumSeed = Math.min(numSeed, 2 * size);
-    for (int i = 0; i < boundedNumSeed; i++) {
-      int entryPoint = random.nextInt(size);
-      if (visited.getAndSet(entryPoint) == false) {
-        // explore the topK starting points of some random numSeed probes
-        float score = similarityFunction.compare(query, vectors.vectorValue(entryPoint));
-        candidates.add(entryPoint, score);
-        if (acceptOrds == null || acceptOrds.get(entryPoint)) {
-          results.add(entryPoint, score);
+    for (int ep : eps) {
+      if (visited.getAndSet(ep) == false) {
+        float score = similarityFunction.compare(query, vectors.vectorValue(ep));
+        candidates.add(ep, score);
+        if (acceptOrds == null || acceptOrds.get(ep)) {
+          results.add(ep, score);
         }
       }
     }
 
     // Set the bound to the worst current result and below reject any newly-generated candidates
-    // failing
-    // to exceed this bound
+    // failing to exceed this bound
     BoundsChecker bound = BoundsChecker.create(similarityFunction.reversed);
     bound.set(results.topScore());
     while (candidates.size() > 0) {
@@ -136,7 +180,7 @@ public final class HnswGraph extends KnnGraphValues {
         }
       }
       int topCandidateNode = candidates.pop();
-      graphValues.seek(topCandidateNode);
+      graphValues.seek(level, topCandidateNode);
       int friendOrd;
       while ((friendOrd = graphValues.nextNeighbor()) != NO_MORE_DOCS) {
         assert friendOrd < size : "friendOrd=" + friendOrd + "; size=" + size;
@@ -145,7 +189,7 @@ public final class HnswGraph extends KnnGraphValues {
         }
 
         float score = similarityFunction.compare(query, vectors.vectorValue(friendOrd));
-        if (results.size() < numSeed || bound.check(score) == false) {
+        if (results.size() < topK || bound.check(score) == false) {
           candidates.add(friendOrd, score);
           if (acceptOrds == null || acceptOrds.get(friendOrd)) {
             results.insertWithOverflow(friendOrd, score);
@@ -164,25 +208,60 @@ public final class HnswGraph extends KnnGraphValues {
   /**
    * Returns the {@link NeighborQueue} connected to the given node.
    *
-   * @param node the node whose neighbors are returned
+   * @param level level of the graph
+   * @param node the node whose neighbors are returned, represented as an ordinal on the level 0.
    */
-  public NeighborArray getNeighbors(int node) {
-    return graph.get(node);
+  public NeighborArray getNeighbors(int level, int node) {
+    if (level == 0) {
+      return graph.get(level).get(node);
+    }
+    int nodeIndex = Arrays.binarySearch(nodesByLevel.get(level), 0, graph.get(level).size(), node);
+    assert nodeIndex >= 0;
+    return graph.get(level).get(nodeIndex);
   }
 
   @Override
   public int size() {
-    return graph.size();
+    return graph.get(0).size(); // all nodes are located on the 0th level
   }
 
-  int addNode() {
-    graph.add(new NeighborArray(maxConn + 1));
-    return graph.size() - 1;
+  /**
+   * Add node on the given level
+   *
+   * @param level level to add a node on
+   * @param node the node to add, represented as an ordinal on the level 0.
+   */
+  public void addNode(int level, int node) {
+    if (level > 0) {
+      // if the new node introduces a new level, add more levels to the graph,
+      // and make this node the graph's new entry point
+      if (level >= numLevels) {
+        for (int i = numLevels; i <= level; i++) {
+          graph.add(new ArrayList<>());
+          nodesByLevel.add(new int[] {node});
+        }
+        numLevels = level + 1;
+        entryNode = node;
+      } else {
+        // Add this node id to this level's nodes
+        int[] nodes = nodesByLevel.get(level);
+        int idx = graph.get(level).size();
+        if (idx < nodes.length) {
+          nodes[idx] = node;
+        } else {
+          nodes = ArrayUtil.grow(nodes);
+          nodes[idx] = node;
+          nodesByLevel.set(level, nodes);
+        }
+      }
+    }
+
+    graph.get(level).add(new NeighborArray(maxConn + 1));
   }
 
   @Override
-  public void seek(int targetNode) {
-    cur = getNeighbors(targetNode);
+  public void seek(int level, int targetNode) {
+    cur = getNeighbors(level, targetNode);
     upto = -1;
   }
 
@@ -193,4 +272,34 @@ public final class HnswGraph extends KnnGraphValues {
     }
     return NO_MORE_DOCS;
   }
+
+  /**
+   * Returns the current number of levels in the graph
+   *
+   * @return the current number of levels in the graph
+   */
+  @Override
+  public int numLevels() {
+    return numLevels;
+  }
+
+  /**
+   * Returns the graph's current entry node on the top level shown as ordinals of the nodes on 0th
+   * level
+   *
+   * @return the graph's current entry node on the top level
+   */
+  @Override
+  public int entryNode() {
+    return entryNode;
+  }
+
+  @Override
+  public NodesIterator getNodesOnLevel(int level) {
+    if (level == 0) {
+      return new NodesIterator(size());
+    } else {
+      return new NodesIterator(nodesByLevel.get(level), graph.get(level).size());
+    }
+  }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java b/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java
index f5cfc6a..041fd18 100644
--- a/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java
+++ b/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java
@@ -17,6 +17,8 @@
 
 package org.apache.lucene.util.hnsw;
 
+import static java.lang.Math.log;
+
 import java.io.IOException;
 import java.util.Locale;
 import java.util.Objects;
@@ -32,15 +34,17 @@ import org.apache.lucene.util.InfoStream;
  */
 public final class HnswGraphBuilder {
 
-  // default random seed for level generation
+  /** Default random seed for level generation * */
   private static final long DEFAULT_RAND_SEED = System.currentTimeMillis();
+  /** A name for the HNSW component for the info-stream * */
   public static final String HNSW_COMPONENT = "HNSW";
 
-  // expose for testing.
+  /** Random seed for level generation; public to expose for testing * */
   public static long randSeed = DEFAULT_RAND_SEED;
 
   private final int maxConn;
   private final int beamWidth;
+  private final double ml;
   private final NeighborArray scratch;
 
   private final VectorSimilarityFunction similarityFunction;
@@ -84,9 +88,12 @@ public final class HnswGraphBuilder {
     }
     this.maxConn = maxConn;
     this.beamWidth = beamWidth;
-    this.hnsw = new HnswGraph(maxConn);
+    // normalization factor for level generation; currently not configurable
+    this.ml = 1 / Math.log(1.0 * maxConn);
+    this.random = new SplittableRandom(seed);
+    int levelOfFirstNode = getRandomGraphLevel(ml, random);
+    this.hnsw = new HnswGraph(maxConn, levelOfFirstNode);
     bound = BoundsChecker.create(similarityFunction.reversed);
-    random = new SplittableRandom(seed);
     scratch = new NeighborArray(Math.max(beamWidth, maxConn + 1));
   }
 
@@ -109,43 +116,58 @@ public final class HnswGraphBuilder {
     long start = System.nanoTime(), t = start;
     // start at node 1! node 0 is added implicitly, in the constructor
     for (int node = 1; node < vectors.size(); node++) {
-      addGraphNode(vectors.vectorValue(node));
-      if (node % 10000 == 0) {
-        if (infoStream.isEnabled(HNSW_COMPONENT)) {
-          long now = System.nanoTime();
-          infoStream.message(
-              HNSW_COMPONENT,
-              String.format(
-                  Locale.ROOT,
-                  "built %d in %d/%d ms",
-                  node,
-                  ((now - t) / 1_000_000),
-                  ((now - start) / 1_000_000)));
-          t = now;
-        }
+      addGraphNode(node, vectors.vectorValue(node));
+      if ((node % 10000 == 0) && infoStream.isEnabled(HNSW_COMPONENT)) {
+        t = printGraphBuildStatus(node, start, t);
       }
     }
     return hnsw;
   }
 
+  /** Set info-stream to output debugging information * */
   public void setInfoStream(InfoStream infoStream) {
     this.infoStream = infoStream;
   }
 
   /** Inserts a doc with vector value to the graph */
-  void addGraphNode(float[] value) throws IOException {
-    // We pass 'null' for acceptOrds because there are no deletions while building the graph
-    NeighborQueue candidates =
-        HnswGraph.search(
-            value, beamWidth, beamWidth, vectorValues, similarityFunction, hnsw, null, random);
+  void addGraphNode(int node, float[] value) throws IOException {
+    NeighborQueue candidates;
+    final int nodeLevel = getRandomGraphLevel(ml, random);
+    int curMaxLevel = hnsw.numLevels() - 1;
+    int[] eps = new int[] {hnsw.entryNode()};
 
-    int node = hnsw.addNode();
+    // if a node introduces new levels to the graph, add this new node on new levels
+    for (int level = nodeLevel; level > curMaxLevel; level--) {
+      hnsw.addNode(level, node);
+    }
+    // for levels > nodeLevel search with topk = 1
+    for (int level = curMaxLevel; level > nodeLevel; level--) {
+      candidates =
+          HnswGraph.searchLevel(value, 1, level, eps, vectorValues, similarityFunction, hnsw, null);
+      eps = new int[] {candidates.pop()};
+    }
+    // for levels <= nodeLevel search with topk = beamWidth, and add connections
+    for (int level = Math.min(nodeLevel, curMaxLevel); level >= 0; level--) {
+      candidates =
+          HnswGraph.searchLevel(
+              value, beamWidth, level, eps, vectorValues, similarityFunction, hnsw, null);
+      eps = candidates.nodes();
+      hnsw.addNode(level, node);
+      addDiverseNeighbors(level, node, candidates);
+    }
+  }
 
-    /* connect neighbors to the new node, using a diversity heuristic that chooses successive
-     * nearest neighbors that are closer to the new node than they are to the previously-selected
-     * neighbors
-     */
-    addDiverseNeighbors(node, candidates);
+  private long printGraphBuildStatus(int node, long start, long t) {
+    long now = System.nanoTime();
+    infoStream.message(
+        HNSW_COMPONENT,
+        String.format(
+            Locale.ROOT,
+            "built %d in %d/%d ms",
+            node,
+            ((now - t) / 1_000_000),
+            ((now - start) / 1_000_000)));
+    return now;
   }
 
   /* TODO: we are not maintaining nodes in strict score order; the forward links
@@ -153,12 +175,13 @@ public final class HnswGraphBuilder {
    * work better if we keep the neighbor arrays sorted. Possibly we should switch back to a heap?
    * But first we should just see if sorting makes a significant difference.
    */
-  private void addDiverseNeighbors(int node, NeighborQueue candidates) throws IOException {
+  private void addDiverseNeighbors(int level, int node, NeighborQueue candidates)
+      throws IOException {
     /* For each of the beamWidth nearest candidates (going from best to worst), select it only if it
      * is closer to target than it is to any of the already-selected neighbors (ie selected in this method,
      * since the node is new and has no prior neighbors).
      */
-    NeighborArray neighbors = hnsw.getNeighbors(node);
+    NeighborArray neighbors = hnsw.getNeighbors(level, node);
     assert neighbors.size() == 0; // new node
     popToScratch(candidates);
     selectDiverse(neighbors, scratch);
@@ -168,7 +191,7 @@ public final class HnswGraphBuilder {
     int size = neighbors.size();
     for (int i = 0; i < size; i++) {
       int nbr = neighbors.node[i];
-      NeighborArray nbrNbr = hnsw.getNeighbors(nbr);
+      NeighborArray nbrNbr = hnsw.getNeighbors(level, nbr);
       nbrNbr.add(node, neighbors.score[i]);
       if (nbrNbr.size() > maxConn) {
         diversityUpdate(nbrNbr);
@@ -266,4 +289,12 @@ public final class HnswGraphBuilder {
     }
     return -1;
   }
+
+  private static int getRandomGraphLevel(double ml, SplittableRandom random) {
+    double randDouble;
+    do {
+      randDouble = random.nextDouble(); // avoid 0 value, as log(0) is undefined
+    } while (randDouble == 0.0);
+    return ((int) (-log(randDouble) * ml));
+  }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/util/hnsw/NeighborArray.java b/lucene/core/src/java/org/apache/lucene/util/hnsw/NeighborArray.java
index 9deaa64..4012575 100644
--- a/lucene/core/src/java/org/apache/lucene/util/hnsw/NeighborArray.java
+++ b/lucene/core/src/java/org/apache/lucene/util/hnsw/NeighborArray.java
@@ -32,7 +32,7 @@ public class NeighborArray {
   float[] score;
   int[] node;
 
-  NeighborArray(int maxSize) {
+  public NeighborArray(int maxSize) {
     node = new int[maxSize];
     score = new float[maxSize];
   }
@@ -60,11 +60,15 @@ public class NeighborArray {
     return node;
   }
 
+  public float[] score() {
+    return score;
+  }
+
   public void clear() {
     size = 0;
   }
 
-  void removeLast() {
+  public void removeLast() {
     size--;
   }
 
diff --git a/lucene/core/src/java/org/apache/lucene/util/hnsw/NeighborQueue.java b/lucene/core/src/java/org/apache/lucene/util/hnsw/NeighborQueue.java
index 0d2b94d..6f08a7b 100644
--- a/lucene/core/src/java/org/apache/lucene/util/hnsw/NeighborQueue.java
+++ b/lucene/core/src/java/org/apache/lucene/util/hnsw/NeighborQueue.java
@@ -54,7 +54,7 @@ public class NeighborQueue {
   // Used to track the number of neighbors visited during a single graph traversal
   private int visitedCount;
 
-  NeighborQueue(int initialSize, boolean reversed) {
+  public NeighborQueue(int initialSize, boolean reversed) {
     this.heap = new LongHeap(initialSize);
     this.order = reversed ? Order.REVERSED : Order.NATURAL;
   }
@@ -119,7 +119,7 @@ public class NeighborQueue {
     return visitedCount;
   }
 
-  void setVisitedCount(int visitedCount) {
+  public void setVisitedCount(int visitedCount) {
     this.visitedCount = visitedCount;
   }
 
diff --git a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec b/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
index 7eec415..6e977e4 100644
--- a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
+++ b/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
@@ -13,4 +13,4 @@
 #  See the License for the specific language governing permissions and
 #  limitations under the License.
 
-org.apache.lucene.codecs.lucene90.Lucene90Codec
+org.apache.lucene.codecs.lucene91.Lucene91Codec
diff --git a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.KnnVectorsFormat b/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.KnnVectorsFormat
index e7158f1..6921458 100644
--- a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.KnnVectorsFormat
+++ b/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.KnnVectorsFormat
@@ -13,4 +13,4 @@
 #  See the License for the specific language governing permissions and
 #  limitations under the License.
 
-org.apache.lucene.codecs.lucene90.Lucene90HnswVectorsFormat
\ No newline at end of file
+org.apache.lucene.codecs.lucene91.Lucene91HnswVectorsFormat
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90StoredFieldsFormatHighCompression.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90StoredFieldsFormatHighCompression.java
index a1cb543..6b747ea 100644
--- a/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90StoredFieldsFormatHighCompression.java
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90StoredFieldsFormatHighCompression.java
@@ -18,7 +18,8 @@ package org.apache.lucene.codecs.lucene90;
 
 import com.carrotsearch.randomizedtesting.generators.RandomPicks;
 import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.lucene90.Lucene90Codec.Mode;
+import org.apache.lucene.codecs.lucene91.Lucene91Codec;
+import org.apache.lucene.codecs.lucene91.Lucene91Codec.Mode;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.StoredField;
 import org.apache.lucene.index.DirectoryReader;
@@ -30,7 +31,7 @@ import org.apache.lucene.tests.index.BaseStoredFieldsFormatTestCase;
 public class TestLucene90StoredFieldsFormatHighCompression extends BaseStoredFieldsFormatTestCase {
   @Override
   protected Codec getCodec() {
-    return new Lucene90Codec(Mode.BEST_COMPRESSION);
+    return new Lucene91Codec(Mode.BEST_COMPRESSION);
   }
 
   /**
@@ -40,7 +41,7 @@ public class TestLucene90StoredFieldsFormatHighCompression extends BaseStoredFie
     Directory dir = newDirectory();
     for (int i = 0; i < 10; i++) {
       IndexWriterConfig iwc = newIndexWriterConfig();
-      iwc.setCodec(new Lucene90Codec(RandomPicks.randomFrom(random(), Mode.values())));
+      iwc.setCodec(new Lucene91Codec(RandomPicks.randomFrom(random(), Mode.values())));
       IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig());
       Document doc = new Document();
       doc.add(new StoredField("field1", "value1"));
@@ -69,7 +70,7 @@ public class TestLucene90StoredFieldsFormatHighCompression extends BaseStoredFie
     expectThrows(
         NullPointerException.class,
         () -> {
-          new Lucene90Codec(null);
+          new Lucene91Codec(null);
         });
 
     expectThrows(
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90HnswVectorsFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene91/TestLucene91HnswVectorsFormat.java
similarity index 76%
rename from lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90HnswVectorsFormat.java
rename to lucene/core/src/test/org/apache/lucene/codecs/lucene91/TestLucene91HnswVectorsFormat.java
index 7baca8c..fe828f0 100644
--- a/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90HnswVectorsFormat.java
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene91/TestLucene91HnswVectorsFormat.java
@@ -14,18 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.lucene.codecs.lucene90;
+package org.apache.lucene.codecs.lucene91;
 
 import static com.carrotsearch.randomizedtesting.RandomizedTest.randomIntBetween;
-import static org.apache.lucene.codecs.lucene90.Lucene90HnswVectorsFormat.DEFAULT_BEAM_WIDTH;
-import static org.apache.lucene.codecs.lucene90.Lucene90HnswVectorsFormat.DEFAULT_MAX_CONN;
+import static org.apache.lucene.codecs.lucene91.Lucene91HnswVectorsFormat.DEFAULT_BEAM_WIDTH;
+import static org.apache.lucene.codecs.lucene91.Lucene91HnswVectorsFormat.DEFAULT_MAX_CONN;
 
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.KnnVectorsFormat;
 import org.apache.lucene.tests.index.BaseKnnVectorsFormatTestCase;
 import org.apache.lucene.tests.util.TestUtil;
 
-public class TestLucene90HnswVectorsFormat extends BaseKnnVectorsFormatTestCase {
+public class TestLucene91HnswVectorsFormat extends BaseKnnVectorsFormatTestCase {
   @Override
   protected Codec getCodec() {
     return TestUtil.getDefaultCodec();
@@ -35,21 +35,20 @@ public class TestLucene90HnswVectorsFormat extends BaseKnnVectorsFormatTestCase
     int maxConn = randomIntBetween(DEFAULT_MAX_CONN - 10, DEFAULT_MAX_CONN + 10);
     int beamWidth = randomIntBetween(DEFAULT_BEAM_WIDTH - 50, DEFAULT_BEAM_WIDTH + 50);
     Codec customCodec =
-        new Lucene90Codec() {
+        new Lucene91Codec() {
           @Override
           public KnnVectorsFormat getKnnVectorsFormatForField(String field) {
-            return new Lucene90HnswVectorsFormat(maxConn, beamWidth);
+            return new Lucene91HnswVectorsFormat(maxConn, beamWidth);
           }
         };
     String expectedString =
-        "Lucene90HnswVectorsFormat(name = Lucene90HnswVectorsFormat, maxConn = "
+        "Lucene91HnswVectorsFormat(name = Lucene91HnswVectorsFormat, maxConn = "
             + maxConn
             + ", beamWidth="
             + beamWidth
             + ")";
-    assert (((Lucene90Codec) customCodec)
-        .getKnnVectorsFormatForField("bogus_field")
-        .toString()
-        .equals(expectedString));
+    assertEquals(
+        expectedString,
+        ((Lucene91Codec) customCodec).getKnnVectorsFormatForField("bogus_field").toString());
   }
 }
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestKnnGraph.java b/lucene/core/src/test/org/apache/lucene/index/TestKnnGraph.java
index 46b6b6d..e2366aa 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestKnnGraph.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestKnnGraph.java
@@ -16,6 +16,7 @@
  */
 package org.apache.lucene.index;
 
+import static com.carrotsearch.randomizedtesting.RandomizedTest.randomIntBetween;
 import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
 import static org.apache.lucene.util.hnsw.HnswGraphBuilder.randSeed;
 
@@ -26,11 +27,12 @@ import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Set;
+import java.util.concurrent.CountDownLatch;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.KnnVectorsFormat;
-import org.apache.lucene.codecs.lucene90.Lucene90Codec;
-import org.apache.lucene.codecs.lucene90.Lucene90HnswVectorsFormat;
-import org.apache.lucene.codecs.lucene90.Lucene90HnswVectorsReader;
+import org.apache.lucene.codecs.lucene91.Lucene91Codec;
+import org.apache.lucene.codecs.lucene91.Lucene91HnswVectorsFormat;
+import org.apache.lucene.codecs.lucene91.Lucene91HnswVectorsReader;
 import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
@@ -38,13 +40,19 @@ import org.apache.lucene.document.FieldType;
 import org.apache.lucene.document.KnnVectorField;
 import org.apache.lucene.document.SortedDocValuesField;
 import org.apache.lucene.document.StringField;
+import org.apache.lucene.index.KnnGraphValues.NodesIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.KnnVectorQuery;
 import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.SearcherFactory;
+import org.apache.lucene.search.SearcherManager;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.tests.util.LuceneTestCase;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.VectorUtil;
 import org.apache.lucene.util.hnsw.HnswGraphBuilder;
 import org.junit.After;
@@ -55,7 +63,7 @@ public class TestKnnGraph extends LuceneTestCase {
 
   private static final String KNN_GRAPH_FIELD = "vector";
 
-  private static int maxConn = Lucene90HnswVectorsFormat.DEFAULT_MAX_CONN;
+  private static int maxConn = Lucene91HnswVectorsFormat.DEFAULT_MAX_CONN;
 
   private Codec codec;
   private VectorSimilarityFunction similarityFunction;
@@ -68,11 +76,11 @@ public class TestKnnGraph extends LuceneTestCase {
     }
 
     codec =
-        new Lucene90Codec() {
+        new Lucene91Codec() {
           @Override
           public KnnVectorsFormat getKnnVectorsFormatForField(String field) {
-            return new Lucene90HnswVectorsFormat(
-                maxConn, Lucene90HnswVectorsFormat.DEFAULT_BEAM_WIDTH);
+            return new Lucene91HnswVectorsFormat(
+                maxConn, Lucene91HnswVectorsFormat.DEFAULT_BEAM_WIDTH);
           }
         };
 
@@ -82,7 +90,7 @@ public class TestKnnGraph extends LuceneTestCase {
 
   @After
   public void cleanup() {
-    maxConn = Lucene90HnswVectorsFormat.DEFAULT_MAX_CONN;
+    maxConn = Lucene91HnswVectorsFormat.DEFAULT_MAX_CONN;
   }
 
   /** Basic test of creating documents in a graph */
@@ -153,21 +161,63 @@ public class TestKnnGraph extends LuceneTestCase {
     int dimension = atLeast(10);
     float[][] values = randomVectors(numDoc, dimension);
     int mergePoint = random().nextInt(numDoc);
-    int[][] mergedGraph = getIndexedGraph(values, mergePoint, seed);
-    int[][] singleSegmentGraph = getIndexedGraph(values, -1, seed);
+    int[][][] mergedGraph = getIndexedGraph(values, mergePoint, seed);
+    int[][][] singleSegmentGraph = getIndexedGraph(values, -1, seed);
     assertGraphEquals(singleSegmentGraph, mergedGraph);
   }
 
-  private void assertGraphEquals(int[][] expected, int[][] actual) {
+  /** Test writing and reading of multiple vector fields * */
+  public void testMultipleVectorFields() throws Exception {
+    int numVectorFields = randomIntBetween(2, 5);
+    int numDoc = atLeast(100);
+    int[] dims = new int[numVectorFields];
+    float[][][] values = new float[numVectorFields][][];
+    for (int field = 0; field < numVectorFields; field++) {
+      dims[field] = atLeast(3);
+      values[field] = randomVectors(numDoc, dims[field]);
+    }
+
+    try (Directory dir = newDirectory();
+        IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig(null).setCodec(codec))) {
+      for (int docID = 0; docID < numDoc; docID++) {
+        Document doc = new Document();
+        for (int field = 0; field < numVectorFields; field++) {
+          float[] vector = values[field][docID];
+          if (vector != null) {
+            FieldType fieldType = KnnVectorField.createFieldType(vector.length, similarityFunction);
+            doc.add(new KnnVectorField(KNN_GRAPH_FIELD + field, vector, fieldType));
+          }
+        }
+        String idString = Integer.toString(docID);
+        doc.add(new StringField("id", idString, Field.Store.YES));
+        iw.addDocument(doc);
+      }
+      for (int field = 0; field < numVectorFields; field++) {
+        assertConsistentGraph(iw, values[field], KNN_GRAPH_FIELD + field);
+      }
+    }
+  }
+
+  private void assertGraphEquals(int[][][] expected, int[][][] actual) {
     assertEquals("graph sizes differ", expected.length, actual.length);
-    for (int i = 0; i < expected.length; i++) {
-      assertArrayEquals("difference at ord=" + i, expected[i], actual[i]);
+    for (int level = 0; level < expected.length; level++) {
+      for (int node = 0; node < expected[level].length; node++) {
+        assertArrayEquals("difference at ord=" + node, expected[level][node], actual[level][node]);
+      }
     }
   }
 
-  private int[][] getIndexedGraph(float[][] values, int mergePoint, long seed) throws IOException {
+  /**
+   * Return a naive representation of an HNSW graph as a 3 dimensional array: 1st dim represents a
+   * graph layer. Each layer contains an array of arrays – a list of nodes and for each node a list
+   * of the node's neighbours. 2nd dim represents a node on a layer, and contains the node's
+   * neighbourhood, or {@code null} if a node is not present on this layer. 3rd dim represents
+   * neighbours of a node.
+   */
+  private int[][][] getIndexedGraph(float[][] values, int mergePoint, long seed)
+      throws IOException {
     HnswGraphBuilder.randSeed = seed;
-    int[][] graph;
+    int[][][] graph;
     try (Directory dir = newDirectory()) {
       IndexWriterConfig iwc = newIndexWriterConfig();
       iwc.setMergePolicy(new LogDocMergePolicy()); // for predictable segment ordering when merging
@@ -186,8 +236,8 @@ public class TestKnnGraph extends LuceneTestCase {
         PerFieldKnnVectorsFormat.FieldsReader perFieldReader =
             (PerFieldKnnVectorsFormat.FieldsReader)
                 ((CodecReader) getOnlyLeafReader(reader)).getVectorReader();
-        Lucene90HnswVectorsReader vectorReader =
-            (Lucene90HnswVectorsReader) perFieldReader.getFieldReader(KNN_GRAPH_FIELD);
+        Lucene91HnswVectorsReader vectorReader =
+            (Lucene91HnswVectorsReader) perFieldReader.getFieldReader(KNN_GRAPH_FIELD);
         graph = copyGraph(vectorReader.getGraphValues(KNN_GRAPH_FIELD));
       }
     }
@@ -208,18 +258,23 @@ public class TestKnnGraph extends LuceneTestCase {
     return values;
   }
 
-  int[][] copyGraph(KnnGraphValues values) throws IOException {
-    int size = values.size();
-    int[][] graph = new int[size][];
+  int[][][] copyGraph(KnnGraphValues graphValues) throws IOException {
+    int[][][] graph = new int[graphValues.numLevels()][][];
+    int size = graphValues.size();
     int[] scratch = new int[maxConn];
-    for (int node = 0; node < size; node++) {
-      int n, count = 0;
-      values.seek(node);
-      while ((n = values.nextNeighbor()) != NO_MORE_DOCS) {
-        scratch[count++] = n;
-        // graph[node][i++] = n;
+
+    for (int level = 0; level < graphValues.numLevels(); level++) {
+      NodesIterator nodesItr = graphValues.getNodesOnLevel(level);
+      graph[level] = new int[size][];
+      while (nodesItr.hasNext()) {
+        int node = nodesItr.nextInt();
+        graphValues.seek(level, node);
+        int n, count = 0;
+        while ((n = graphValues.nextNeighbor()) != NO_MORE_DOCS) {
+          scratch[count++] = n;
+        }
+        graph[level][node] = ArrayUtil.copyOfSubArray(scratch, 0, count);
       }
-      graph[node] = ArrayUtil.copyOfSubArray(scratch, 0, count);
     }
     return graph;
   }
@@ -232,31 +287,7 @@ public class TestKnnGraph extends LuceneTestCase {
     config.setCodec(codec); // test is not compatible with simpletext
     try (Directory dir = newDirectory();
         IndexWriter iw = new IndexWriter(dir, config)) {
-      // Add a document for every cartesian point in an NxN square so we can
-      // easily know which are the nearest neighbors to every point. Insert by iterating
-      // using a prime number that is not a divisor of N*N so that we will hit each point once,
-      // and chosen so that points will be inserted in a deterministic
-      // but somewhat distributed pattern
-      int n = 5, stepSize = 17;
-      float[][] values = new float[n * n][];
-      int index = 0;
-      for (int i = 0; i < values.length; i++) {
-        // System.out.printf("%d: (%d, %d)\n", i, index % n, index / n);
-        int x = index % n, y = index / n;
-        values[i] = new float[] {x, y};
-        index = (index + stepSize) % (n * n);
-        add(iw, i, values[i]);
-        if (i == 13) {
-          // create 2 segments
-          iw.commit();
-        }
-      }
-      boolean forceMerge = random().nextBoolean();
-      // System.out.println("");
-      if (forceMerge) {
-        iw.forceMerge(1);
-      }
-      assertConsistentGraph(iw, values);
+      indexData(iw);
       try (DirectoryReader dr = DirectoryReader.open(iw)) {
         // results are ordered by score (descending) and docid (ascending);
         // This is the insertion order:
@@ -279,6 +310,77 @@ public class TestKnnGraph extends LuceneTestCase {
     }
   }
 
+  private void indexData(IndexWriter iw) throws IOException {
+    // Add a document for every cartesian point in an NxN square so we can
+    // easily know which are the nearest neighbors to every point. Insert by iterating
+    // using a prime number that is not a divisor of N*N so that we will hit each point once,
+    // and chosen so that points will be inserted in a deterministic
+    // but somewhat distributed pattern
+    int n = 5, stepSize = 17;
+    float[][] values = new float[n * n][];
+    int index = 0;
+    for (int i = 0; i < values.length; i++) {
+      // System.out.printf("%d: (%d, %d)\n", i, index % n, index / n);
+      int x = index % n, y = index / n;
+      values[i] = new float[] {x, y};
+      index = (index + stepSize) % (n * n);
+      add(iw, i, values[i]);
+      if (i == 13) {
+        // create 2 segments
+        iw.commit();
+      }
+    }
+    boolean forceMerge = random().nextBoolean();
+    if (forceMerge) {
+      iw.forceMerge(1);
+    }
+    assertConsistentGraph(iw, values);
+  }
+
+  public void testMultiThreadedSearch() throws Exception {
+    similarityFunction = VectorSimilarityFunction.EUCLIDEAN;
+    IndexWriterConfig config = newIndexWriterConfig();
+    config.setCodec(codec);
+    Directory dir = newDirectory();
+    IndexWriter iw = new IndexWriter(dir, config);
+    indexData(iw);
+
+    final SearcherManager manager = new SearcherManager(iw, new SearcherFactory());
+    Thread[] threads = new Thread[randomIntBetween(2, 5)];
+    final CountDownLatch latch = new CountDownLatch(1);
+    for (int i = 0; i < threads.length; i++) {
+      threads[i] =
+          new Thread(
+              () -> {
+                try {
+                  latch.await();
+                  IndexSearcher searcher = manager.acquire();
+                  try {
+                    KnnVectorQuery query = new KnnVectorQuery("vector", new float[] {0f, 0.1f}, 5);
+                    TopDocs results = searcher.search(query, 5);
+                    for (ScoreDoc doc : results.scoreDocs) {
+                      // map docId to insertion id
+                      doc.doc =
+                          Integer.parseInt(searcher.getIndexReader().document(doc.doc).get("id"));
+                    }
+                    assertResults(new int[] {0, 15, 3, 18, 5}, results);
+                  } finally {
+                    manager.release(searcher);
+                  }
+                } catch (Exception e) {
+                  throw new RuntimeException(e);
+                }
+              });
+      threads[i].start();
+    }
+
+    latch.countDown();
+    for (Thread t : threads) {
+      t.join();
+    }
+    IOUtils.close(manager, iw, dir);
+  }
+
   private void assertGraphSearch(int[] expected, float[] vector, IndexReader reader)
       throws IOException {
     TopDocs results = doKnnSearch(reader, vector, 5);
@@ -310,39 +412,40 @@ public class TestKnnGraph extends LuceneTestCase {
     }
   }
 
+  private void assertConsistentGraph(IndexWriter iw, float[][] values) throws IOException {
+    assertConsistentGraph(iw, values, KNN_GRAPH_FIELD);
+  }
+
   // For each leaf, verify that its graph nodes are 1-1 with vectors, that the vectors are the
-  // expected values,
-  // and that the graph is fully connected and symmetric.
+  // expected values, and that the graph is fully connected and symmetric.
   // NOTE: when we impose max-fanout on the graph it wil no longer be symmetric, but should still
   // be fully connected. Is there any other invariant we can test? Well, we can check that max
-  // fanout
-  // is respected. We can test *desirable* properties of the graph like small-world (the graph
-  // diameter
-  // should be tightly bounded).
-  private void assertConsistentGraph(IndexWriter iw, float[][] values) throws IOException {
-    int totalGraphDocs = 0;
+  // fanout is respected. We can test *desirable* properties of the graph like small-world
+  // (the graph diameter should be tightly bounded).
+  private void assertConsistentGraph(IndexWriter iw, float[][] values, String vectorField)
+      throws IOException {
+    int numDocsWithVectors = 0;
     try (DirectoryReader dr = DirectoryReader.open(iw)) {
       for (LeafReaderContext ctx : dr.leaves()) {
         LeafReader reader = ctx.reader();
-        VectorValues vectorValues = reader.getVectorValues(KNN_GRAPH_FIELD);
+        VectorValues vectorValues = reader.getVectorValues(vectorField);
         PerFieldKnnVectorsFormat.FieldsReader perFieldReader =
             (PerFieldKnnVectorsFormat.FieldsReader) ((CodecReader) reader).getVectorReader();
         if (perFieldReader == null) {
           continue;
         }
-        Lucene90HnswVectorsReader vectorReader =
-            (Lucene90HnswVectorsReader) perFieldReader.getFieldReader(KNN_GRAPH_FIELD);
-        KnnGraphValues graphValues = vectorReader.getGraphValues(KNN_GRAPH_FIELD);
-        assertEquals((vectorValues == null), (graphValues == null));
+        Lucene91HnswVectorsReader vectorReader =
+            (Lucene91HnswVectorsReader) perFieldReader.getFieldReader(vectorField);
+        KnnGraphValues graphValues = vectorReader.getGraphValues(vectorField);
         if (vectorValues == null) {
+          assert graphValues == null;
           continue;
         }
-        int[][] graph = new int[reader.maxDoc()][];
-        boolean foundOrphan = false;
-        int graphSize = 0;
+
+        // assert vector values:
+        // stored vector values are the same as original
         for (int i = 0; i < reader.maxDoc(); i++) {
           int nextDocWithVectors = vectorValues.advance(i);
-          // System.out.println("advanced to " + nextDocWithVectors);
           while (i < nextDocWithVectors && i < reader.maxDoc()) {
             int id = Integer.parseInt(reader.document(i).get("id"));
             assertNull("document " + id + " has no vector, but was expected to", values[id]);
@@ -352,7 +455,6 @@ public class TestKnnGraph extends LuceneTestCase {
             break;
           }
           int id = Integer.parseInt(reader.document(i).get("id"));
-          graphValues.seek(graphSize);
           // documents with KnnGraphValues have the expected vectors
           float[] scratch = vectorValues.vectorValue();
           assertArrayEquals(
@@ -360,54 +462,71 @@ public class TestKnnGraph extends LuceneTestCase {
               values[id],
               scratch,
               0f);
-          // We collect neighbors for analysis below
-          List<Integer> friends = new ArrayList<>();
-          int arc;
-          while ((arc = graphValues.nextNeighbor()) != NO_MORE_DOCS) {
-            friends.add(arc);
+          numDocsWithVectors++;
+        }
+        assertEquals(NO_MORE_DOCS, vectorValues.nextDoc());
+
+        // assert graph values:
+        // For each level of the graph assert that:
+        // 1. There are no orphan nodes without any friends
+        // 2. If orphans are found, than the level must contain only 0 or a single node
+        // 3. If the number of nodes on the level doesn't exceed maxConn, assert that the graph is
+        //   fully connected, i.e. any node is reachable from any other node.
+        // 4. If the number of nodes on the level exceeds maxConn, assert that maxConn is respected.
+        for (int level = 0; level < graphValues.numLevels(); level++) {
+          int[][] graphOnLevel = new int[graphValues.size()][];
+          int countOnLevel = 0;
+          boolean foundOrphan = false;
+          NodesIterator nodesItr = graphValues.getNodesOnLevel(level);
+          while (nodesItr.hasNext()) {
+            int node = nodesItr.nextInt();
+            graphValues.seek(level, node);
+            int arc;
+            List<Integer> friends = new ArrayList<>();
+            while ((arc = graphValues.nextNeighbor()) != NO_MORE_DOCS) {
+              friends.add(arc);
+            }
+            if (friends.size() == 0) {
+              foundOrphan = true;
+            } else {
+              int[] friendsCopy = new int[friends.size()];
+              Arrays.setAll(friendsCopy, friends::get);
+              graphOnLevel[node] = friendsCopy;
+            }
+            countOnLevel++;
           }
-          if (friends.size() == 0) {
-            // System.out.printf("knngraph @%d is singleton (advance returns %d)\n", i,
-            // nextWithNeighbors);
-            foundOrphan = true;
+          // System.out.println("Level[" + level + "] has [" + nodesCount + "] nodes.");
+          assertEquals(nodesItr.size(), countOnLevel);
+          assertFalse("No nodes on level [" + level + "]", countOnLevel == 0);
+          if (countOnLevel == 1) {
+            assertTrue(
+                "Graph with 1 node has unexpected neighbors on level [" + level + "]", foundOrphan);
           } else {
-            // NOTE: these friends are dense ordinals, not docIds.
-            int[] friendCopy = new int[friends.size()];
-            for (int j = 0; j < friends.size(); j++) {
-              friendCopy[j] = friends.get(j);
+            assertFalse(
+                "Graph has orphan nodes with no friends on level [" + level + "]", foundOrphan);
+            if (maxConn > countOnLevel) {
+              // assert that the graph is fully connected,
+              // i.e. any node can be reached from any other node
+              assertConnected(graphOnLevel);
+            } else {
+              // assert that max-connections was respected
+              assertMaxConn(graphOnLevel, maxConn);
             }
-            graph[graphSize] = friendCopy;
-            // System.out.printf("knngraph @%d => %s\n", i, Arrays.toString(graph[i]));
           }
-          graphSize++;
         }
-        assertEquals(NO_MORE_DOCS, vectorValues.nextDoc());
-        if (foundOrphan) {
-          assertEquals("graph is not fully connected", 1, graphSize);
-        } else {
-          assertTrue(
-              "Graph has " + graphSize + " nodes, but one of them has no neighbors", graphSize > 1);
-        }
-        if (maxConn > graphSize) {
-          // assert that the graph in each leaf is connected
-          assertConnected(graph);
-        } else {
-          // assert that max-connections was respected
-          assertMaxConn(graph, maxConn);
-        }
-        totalGraphDocs += graphSize;
       }
     }
-    int expectedCount = 0;
-    for (float[] friends : values) {
-      if (friends != null) {
-        ++expectedCount;
+
+    int expectedNumDocsWithVectors = 0;
+    for (float[] value : values) {
+      if (value != null) {
+        ++expectedNumDocsWithVectors;
       }
     }
-    assertEquals(expectedCount, totalGraphDocs);
+    assertEquals(expectedNumDocsWithVectors, numDocsWithVectors);
   }
 
-  private void assertMaxConn(int[][] graph, int maxConn) {
+  public static void assertMaxConn(int[][] graph, int maxConn) {
     for (int[] ints : graph) {
       if (ints != null) {
         assert (ints.length <= maxConn);
@@ -418,37 +537,36 @@ public class TestKnnGraph extends LuceneTestCase {
     }
   }
 
-  private void assertConnected(int[][] graph) {
-    // every node in the graph is reachable from every other node
+  /** Assert that every node is reachable from some other node */
+  private static void assertConnected(int[][] graph) {
+    List<Integer> nodes = new ArrayList<>();
     Set<Integer> visited = new HashSet<>();
     List<Integer> queue = new LinkedList<>();
-    int count = 0;
-    for (int[] entry : graph) {
-      if (entry != null) {
-        if (queue.isEmpty()) {
-          queue.add(entry[0]); // start from any node
-          // System.out.println("start at " + entry[0]);
-        }
-        ++count;
+    for (int i = 0; i < graph.length; i++) {
+      if (graph[i] != null) {
+        nodes.add(i);
       }
     }
+
+    // start from any node
+    int startIdx = random().nextInt(nodes.size());
+    queue.add(nodes.get(startIdx));
     while (queue.isEmpty() == false) {
       int i = queue.remove(0);
       assertNotNull("expected neighbors of " + i, graph[i]);
       visited.add(i);
       for (int j : graph[i]) {
         if (visited.contains(j) == false) {
-          // System.out.println("  ... " + j);
           queue.add(j);
         }
       }
     }
-    for (int i = 0; i < count; i++) {
-      assertTrue("Attempted to walk entire graph but never visited " + i, visited.contains(i));
+    // assert that every node is reachable from some other node as it was visited
+    for (int node : nodes) {
+      assertTrue(
+          "Attempted to walk entire graph but never visited node [" + node + "]",
+          visited.contains(node));
     }
-    // we visited each node exactly once
-    assertEquals(
-        "Attempted to walk entire graph but only visited " + visited.size(), count, visited.size());
   }
 
   private void add(IndexWriter iw, int id, float[] vector) throws IOException {
diff --git a/lucene/core/src/test/org/apache/lucene/util/hnsw/KnnGraphTester.java b/lucene/core/src/test/org/apache/lucene/util/hnsw/KnnGraphTester.java
index 339d8f8..be600ae 100644
--- a/lucene/core/src/test/org/apache/lucene/util/hnsw/KnnGraphTester.java
+++ b/lucene/core/src/test/org/apache/lucene/util/hnsw/KnnGraphTester.java
@@ -37,9 +37,9 @@ import java.util.Locale;
 import java.util.Set;
 import org.apache.lucene.codecs.KnnVectorsFormat;
 import org.apache.lucene.codecs.KnnVectorsReader;
-import org.apache.lucene.codecs.lucene90.Lucene90Codec;
-import org.apache.lucene.codecs.lucene90.Lucene90HnswVectorsFormat;
-import org.apache.lucene.codecs.lucene90.Lucene90HnswVectorsReader;
+import org.apache.lucene.codecs.lucene91.Lucene91Codec;
+import org.apache.lucene.codecs.lucene91.Lucene91HnswVectorsFormat;
+import org.apache.lucene.codecs.lucene91.Lucene91HnswVectorsReader;
 import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.FieldType;
@@ -253,7 +253,7 @@ public class KnnGraphTester {
             ((PerFieldKnnVectorsFormat.FieldsReader) ((CodecReader) leafReader).getVectorReader())
                 .getFieldReader(KNN_FIELD);
         KnnGraphValues knnValues =
-            ((Lucene90HnswVectorsReader) vectorsReader).getGraphValues(KNN_FIELD);
+            ((Lucene91HnswVectorsReader) vectorsReader).getGraphValues(KNN_FIELD);
         System.out.printf("Leaf %d has %d documents\n", context.ord, leafReader.maxDoc());
         printGraphFanout(knnValues, leafReader.maxDoc());
       }
@@ -267,7 +267,7 @@ public class KnnGraphTester {
           new HnswGraphBuilder(vectors, similarityFunction, maxConn, beamWidth, 0);
       // start at node 1
       for (int i = 1; i < numDocs; i++) {
-        builder.addGraphNode(values.vectorValue(i));
+        builder.addGraphNode(i, values.vectorValue(i));
         System.out.println("\nITERATION " + i);
         dumpGraph(builder.hnsw);
       }
@@ -276,7 +276,7 @@ public class KnnGraphTester {
 
   private void dumpGraph(HnswGraph hnsw) {
     for (int i = 0; i < hnsw.size(); i++) {
-      NeighborArray neighbors = hnsw.getNeighbors(i);
+      NeighborArray neighbors = hnsw.getNeighbors(0, i);
       System.out.printf(Locale.ROOT, "%5d", i);
       NeighborArray sorted = new NeighborArray(neighbors.size());
       for (int j = 0; j < neighbors.size(); j++) {
@@ -308,7 +308,7 @@ public class KnnGraphTester {
     int count = 0;
     int[] leafHist = new int[numDocs];
     for (int node = 0; node < numDocs; node++) {
-      knnValues.seek(node);
+      knnValues.seek(0, node);
       int n = 0;
       while (knnValues.nextNeighbor() != NO_MORE_DOCS) {
         ++n;
@@ -580,10 +580,10 @@ public class KnnGraphTester {
   private int createIndex(Path docsPath, Path indexPath) throws IOException {
     IndexWriterConfig iwc = new IndexWriterConfig().setOpenMode(IndexWriterConfig.OpenMode.CREATE);
     iwc.setCodec(
-        new Lucene90Codec() {
+        new Lucene91Codec() {
           @Override
           public KnnVectorsFormat getKnnVectorsFormatForField(String field) {
-            return new Lucene90HnswVectorsFormat(maxConn, beamWidth);
+            return new Lucene91HnswVectorsFormat(maxConn, beamWidth);
           }
         });
     // iwc.setMergePolicy(NoMergePolicy.INSTANCE);
diff --git a/lucene/core/src/test/org/apache/lucene/util/hnsw/TestHnswGraph.java b/lucene/core/src/test/org/apache/lucene/util/hnsw/TestHnswGraph.java
index 38bb50a..de21bef 100644
--- a/lucene/core/src/test/org/apache/lucene/util/hnsw/TestHnswGraph.java
+++ b/lucene/core/src/test/org/apache/lucene/util/hnsw/TestHnswGraph.java
@@ -24,11 +24,10 @@ import java.util.Arrays;
 import java.util.HashSet;
 import java.util.Random;
 import java.util.Set;
-import java.util.SplittableRandom;
 import org.apache.lucene.codecs.KnnVectorsFormat;
-import org.apache.lucene.codecs.lucene90.Lucene90Codec;
-import org.apache.lucene.codecs.lucene90.Lucene90HnswVectorsFormat;
-import org.apache.lucene.codecs.lucene90.Lucene90HnswVectorsReader;
+import org.apache.lucene.codecs.lucene91.Lucene91Codec;
+import org.apache.lucene.codecs.lucene91.Lucene91HnswVectorsFormat;
+import org.apache.lucene.codecs.lucene91.Lucene91HnswVectorsReader;
 import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.KnnVectorField;
@@ -39,6 +38,7 @@ import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.KnnGraphValues;
+import org.apache.lucene.index.KnnGraphValues.NodesIterator;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.RandomAccessVectorValues;
 import org.apache.lucene.index.RandomAccessVectorValuesProducer;
@@ -81,10 +81,10 @@ public class TestHnswGraph extends LuceneTestCase {
       IndexWriterConfig iwc =
           new IndexWriterConfig()
               .setCodec(
-                  new Lucene90Codec() {
+                  new Lucene91Codec() {
                     @Override
                     public KnnVectorsFormat getKnnVectorsFormatForField(String field) {
-                      return new Lucene90HnswVectorsFormat(maxConn, beamWidth);
+                      return new Lucene91HnswVectorsFormat(maxConn, beamWidth);
                     }
                   });
       try (IndexWriter iw = new IndexWriter(dir, iwc)) {
@@ -111,38 +111,64 @@ public class TestHnswGraph extends LuceneTestCase {
           assertEquals(indexedDoc, ctx.reader().numDocs());
           assertVectorsEqual(v3, values);
           KnnGraphValues graphValues =
-              ((Lucene90HnswVectorsReader)
+              ((Lucene91HnswVectorsReader)
                       ((PerFieldKnnVectorsFormat.FieldsReader)
                               ((CodecReader) ctx.reader()).getVectorReader())
                           .getFieldReader("field"))
                   .getGraphValues("field");
-          assertGraphEqual(hnsw, graphValues, nVec);
+          assertGraphEqual(hnsw, graphValues);
         }
       }
     }
   }
 
+  private void assertGraphEqual(KnnGraphValues g, KnnGraphValues h) throws IOException {
+    assertEquals("the number of levels in the graphs are different!", g.numLevels(), h.numLevels());
+    assertEquals("the number of nodes in the graphs are different!", g.size(), h.size());
+
+    // assert equal nodes on each level
+    for (int level = 0; level < g.numLevels(); level++) {
+      NodesIterator nodesOnLevel = g.getNodesOnLevel(level);
+      NodesIterator nodesOnLevel2 = h.getNodesOnLevel(level);
+      while (nodesOnLevel.hasNext() && nodesOnLevel2.hasNext()) {
+        int node = nodesOnLevel.nextInt();
+        int node2 = nodesOnLevel2.nextInt();
+        assertEquals("nodes in the graphs are different", node, node2);
+      }
+    }
+
+    // assert equal nodes' neighbours on each level
+    for (int level = 0; level < g.numLevels(); level++) {
+      NodesIterator nodesOnLevel = g.getNodesOnLevel(level);
+      while (nodesOnLevel.hasNext()) {
+        int node = nodesOnLevel.nextInt();
+        g.seek(level, node);
+        h.seek(level, node);
+        assertEquals("arcs differ for node " + node, getNeighborNodes(g), getNeighborNodes(h));
+      }
+    }
+  }
+
   // Make sure we actually approximately find the closest k elements. Mostly this is about
   // ensuring that we have all the distance functions, comparators, priority queues and so on
   // oriented in the right directions
   public void testAknnDiverse() throws IOException {
+    int maxConn = 10;
     int nDoc = 100;
     CircularVectorValues vectors = new CircularVectorValues(nDoc);
     HnswGraphBuilder builder =
         new HnswGraphBuilder(
-            vectors, VectorSimilarityFunction.DOT_PRODUCT, 16, 100, random().nextInt());
+            vectors, VectorSimilarityFunction.DOT_PRODUCT, maxConn, 100, random().nextInt());
     HnswGraph hnsw = builder.build(vectors);
     // run some searches
     NeighborQueue nn =
         HnswGraph.search(
             new float[] {1, 0},
             10,
-            10,
             vectors.randomAccess(),
             VectorSimilarityFunction.DOT_PRODUCT,
             hnsw,
-            null,
-            new SplittableRandom(random().nextLong()));
+            null);
 
     int[] nodes = nn.nodes();
     assertTrue("Number of found results is not equal to [10].", nodes.length == 10);
@@ -155,7 +181,7 @@ public class TestHnswGraph extends LuceneTestCase {
     assertTrue("sum(result docs)=" + sum, sum < 75);
 
     for (int i = 0; i < nDoc; i++) {
-      NeighborArray neighbors = hnsw.getNeighbors(i);
+      NeighborArray neighbors = hnsw.getNeighbors(0, i);
       int[] nnodes = neighbors.node;
       for (int j = 0; j < neighbors.size(); j++) {
         // all neighbors should be valid node ids.
@@ -166,24 +192,22 @@ public class TestHnswGraph extends LuceneTestCase {
 
   public void testSearchWithAcceptOrds() throws IOException {
     int nDoc = 100;
+    int maxConn = 16;
     CircularVectorValues vectors = new CircularVectorValues(nDoc);
     HnswGraphBuilder builder =
         new HnswGraphBuilder(
-            vectors, VectorSimilarityFunction.DOT_PRODUCT, 16, 100, random().nextInt());
+            vectors, VectorSimilarityFunction.DOT_PRODUCT, maxConn, 100, random().nextInt());
     HnswGraph hnsw = builder.build(vectors);
-
     // the first 10 docs must not be deleted to ensure the expected recall
     Bits acceptOrds = createRandomAcceptOrds(10, vectors.size);
     NeighborQueue nn =
         HnswGraph.search(
             new float[] {1, 0},
             10,
-            10,
             vectors.randomAccess(),
             VectorSimilarityFunction.DOT_PRODUCT,
             hnsw,
-            acceptOrds,
-            new SplittableRandom(random().nextLong()));
+            acceptOrds);
     int[] nodes = nn.nodes();
     assertTrue("Number of found results is not equal to [10].", nodes.length == 10);
     int sum = 0;
@@ -213,12 +237,10 @@ public class TestHnswGraph extends LuceneTestCase {
         HnswGraph.search(
             new float[] {1, 0},
             10,
-            10,
             vectors.randomAccess(),
             VectorSimilarityFunction.EUCLIDEAN,
             hnsw,
-            acceptOrds,
-            new SplittableRandom(random().nextLong()));
+            acceptOrds);
     int[] nodes = nn.nodes();
     assertTrue("Number of found results is not equal to [10].", nodes.length == 10);
     int sum = 0;
@@ -295,46 +317,46 @@ public class TestHnswGraph extends LuceneTestCase {
             vectors, VectorSimilarityFunction.DOT_PRODUCT, 2, 10, random().nextInt());
     // node 0 is added by the builder constructor
     // builder.addGraphNode(vectors.vectorValue(0));
-    builder.addGraphNode(vectors.vectorValue(1));
-    builder.addGraphNode(vectors.vectorValue(2));
+    builder.addGraphNode(1, vectors.vectorValue(1));
+    builder.addGraphNode(2, vectors.vectorValue(2));
     // now every node has tried to attach every other node as a neighbor, but
     // some were excluded based on diversity check.
-    assertNeighbors(builder.hnsw, 0, 1, 2);
-    assertNeighbors(builder.hnsw, 1, 0);
-    assertNeighbors(builder.hnsw, 2, 0);
+    assertLevel0Neighbors(builder.hnsw, 0, 1, 2);
+    assertLevel0Neighbors(builder.hnsw, 1, 0);
+    assertLevel0Neighbors(builder.hnsw, 2, 0);
 
-    builder.addGraphNode(vectors.vectorValue(3));
-    assertNeighbors(builder.hnsw, 0, 1, 2);
+    builder.addGraphNode(3, vectors.vectorValue(3));
+    assertLevel0Neighbors(builder.hnsw, 0, 1, 2);
     // we added 3 here
-    assertNeighbors(builder.hnsw, 1, 0, 3);
-    assertNeighbors(builder.hnsw, 2, 0);
-    assertNeighbors(builder.hnsw, 3, 1);
+    assertLevel0Neighbors(builder.hnsw, 1, 0, 3);
+    assertLevel0Neighbors(builder.hnsw, 2, 0);
+    assertLevel0Neighbors(builder.hnsw, 3, 1);
 
     // supplant an existing neighbor
-    builder.addGraphNode(vectors.vectorValue(4));
+    builder.addGraphNode(4, vectors.vectorValue(4));
     // 4 is the same distance from 0 that 2 is; we leave the existing node in place
-    assertNeighbors(builder.hnsw, 0, 1, 2);
+    assertLevel0Neighbors(builder.hnsw, 0, 1, 2);
     // 4 is closer to 1 than either existing neighbor (0, 3). 3 fails diversity check with 4, so
     // replace it
-    assertNeighbors(builder.hnsw, 1, 0, 4);
-    assertNeighbors(builder.hnsw, 2, 0);
+    assertLevel0Neighbors(builder.hnsw, 1, 0, 4);
+    assertLevel0Neighbors(builder.hnsw, 2, 0);
     // 1 survives the diversity check
-    assertNeighbors(builder.hnsw, 3, 1, 4);
-    assertNeighbors(builder.hnsw, 4, 1, 3);
+    assertLevel0Neighbors(builder.hnsw, 3, 1, 4);
+    assertLevel0Neighbors(builder.hnsw, 4, 1, 3);
 
-    builder.addGraphNode(vectors.vectorValue(5));
-    assertNeighbors(builder.hnsw, 0, 1, 2);
-    assertNeighbors(builder.hnsw, 1, 0, 5);
-    assertNeighbors(builder.hnsw, 2, 0);
+    builder.addGraphNode(5, vectors.vectorValue(5));
+    assertLevel0Neighbors(builder.hnsw, 0, 1, 2);
+    assertLevel0Neighbors(builder.hnsw, 1, 0, 5);
+    assertLevel0Neighbors(builder.hnsw, 2, 0);
     // even though 5 is closer, 3 is not a neighbor of 5, so no update to *its* neighbors occurs
-    assertNeighbors(builder.hnsw, 3, 1, 4);
-    assertNeighbors(builder.hnsw, 4, 3, 5);
-    assertNeighbors(builder.hnsw, 5, 1, 4);
+    assertLevel0Neighbors(builder.hnsw, 3, 1, 4);
+    assertLevel0Neighbors(builder.hnsw, 4, 3, 5);
+    assertLevel0Neighbors(builder.hnsw, 5, 1, 4);
   }
 
-  private void assertNeighbors(HnswGraph graph, int node, int... expected) {
+  private void assertLevel0Neighbors(HnswGraph graph, int node, int... expected) {
     Arrays.sort(expected);
-    NeighborArray nn = graph.getNeighbors(node);
+    NeighborArray nn = graph.getNeighbors(0, node);
     int[] actual = ArrayUtil.copyOfSubArray(nn.node, 0, nn.size());
     Arrays.sort(actual);
     assertArrayEquals(
@@ -346,13 +368,14 @@ public class TestHnswGraph extends LuceneTestCase {
   public void testRandom() throws IOException {
     int size = atLeast(100);
     int dim = atLeast(10);
-    int topK = 5;
+    int maxConn = 10;
     RandomVectorValues vectors = new RandomVectorValues(size, dim, random());
     VectorSimilarityFunction similarityFunction =
         VectorSimilarityFunction.values()[
             random().nextInt(VectorSimilarityFunction.values().length - 1) + 1];
+    int topK = 5;
     HnswGraphBuilder builder =
-        new HnswGraphBuilder(vectors, similarityFunction, 10, 30, random().nextLong());
+        new HnswGraphBuilder(vectors, similarityFunction, maxConn, 30, random().nextLong());
     HnswGraph hnsw = builder.build(vectors);
     Bits acceptOrds = random().nextBoolean() ? null : createRandomAcceptOrds(0, size);
 
@@ -360,15 +383,10 @@ public class TestHnswGraph extends LuceneTestCase {
     for (int i = 0; i < 100; i++) {
       float[] query = randomVector(random(), dim);
       NeighborQueue actual =
-          HnswGraph.search(
-              query,
-              topK,
-              100,
-              vectors,
-              similarityFunction,
-              hnsw,
-              acceptOrds,
-              new SplittableRandom(random().nextLong()));
+          HnswGraph.search(query, 100, vectors, similarityFunction, hnsw, acceptOrds);
+      while (actual.size() > topK) {
+        actual.pop();
+      }
       NeighborQueue expected = new NeighborQueue(topK, similarityFunction.reversed);
       for (int j = 0; j < size; j++) {
         if (vectors.vectorValue(j) != null && (acceptOrds == null || acceptOrds.get(j))) {
@@ -487,14 +505,6 @@ public class TestHnswGraph extends LuceneTestCase {
     return value;
   }
 
-  private void assertGraphEqual(KnnGraphValues g, KnnGraphValues h, int size) throws IOException {
-    for (int node = 0; node < size; node++) {
-      g.seek(node);
-      h.seek(node);
-      assertEquals("arcs differ for node " + node, getNeighborNodes(g), getNeighborNodes(h));
-    }
-  }
-
   private Set<Integer> getNeighborNodes(KnnGraphValues g) throws IOException {
     Set<Integer> neighbors = new HashSet<>();
     for (int n = g.nextNeighbor(); n != NO_MORE_DOCS; n = g.nextNeighbor()) {
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java
index 3220647..0fe4426 100644
--- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java
+++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java
@@ -40,7 +40,7 @@ import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
 import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene90.Lucene90Codec;
+import org.apache.lucene.codecs.lucene91.Lucene91Codec;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.IntPoint;
@@ -961,7 +961,7 @@ public class TestSuggestField extends LuceneTestCase {
     IndexWriterConfig iwc = newIndexWriterConfig(random(), analyzer);
     iwc.setMergePolicy(newLogMergePolicy());
     Codec filterCodec =
-        new Lucene90Codec() {
+        new Lucene91Codec() {
           CompletionPostingsFormat.FSTLoadMode fstLoadMode =
               RandomPicks.randomFrom(random(), CompletionPostingsFormat.FSTLoadMode.values());
           PostingsFormat postingsFormat = new Completion90PostingsFormat(fstLoadMode);
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestRuleSetupAndRestoreClassEnv.java b/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestRuleSetupAndRestoreClassEnv.java
index 8d97675..a7f9a7a 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestRuleSetupAndRestoreClassEnv.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestRuleSetupAndRestoreClassEnv.java
@@ -38,7 +38,7 @@ import java.util.TimeZone;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene90.Lucene90Codec;
+import org.apache.lucene.codecs.lucene91.Lucene91Codec;
 import org.apache.lucene.codecs.simpletext.SimpleTextCodec;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.tests.codecs.asserting.AssertingCodec;
@@ -193,9 +193,9 @@ final class TestRuleSetupAndRestoreClassEnv extends AbstractBeforeAfterRule {
     } else if ("Compressing".equals(TEST_CODEC)
         || ("random".equals(TEST_CODEC) && randomVal == 6 && !shouldAvoidCodec("Compressing"))) {
       codec = CompressingCodec.randomInstance(random);
-    } else if ("Lucene90".equals(TEST_CODEC)
-        || ("random".equals(TEST_CODEC) && randomVal == 5 && !shouldAvoidCodec("Lucene90"))) {
-      codec = new Lucene90Codec(RandomPicks.randomFrom(random, Lucene90Codec.Mode.values()));
+    } else if ("Lucene91".equals(TEST_CODEC)
+        || ("random".equals(TEST_CODEC) && randomVal == 5 && !shouldAvoidCodec("Lucene91"))) {
+      codec = new Lucene91Codec(RandomPicks.randomFrom(random, Lucene91Codec.Mode.values()));
     } else if (!"random".equals(TEST_CODEC)) {
       codec = Codec.forName(TEST_CODEC);
     } else if ("random".equals(TEST_POSTINGSFORMAT)) {
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestUtil.java b/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestUtil.java
index 68741d0..0d1c982 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestUtil.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestUtil.java
@@ -53,10 +53,10 @@ import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.KnnVectorsFormat;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.blocktreeords.BlockTreeOrdsPostingsFormat;
-import org.apache.lucene.codecs.lucene90.Lucene90Codec;
 import org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat;
-import org.apache.lucene.codecs.lucene90.Lucene90HnswVectorsFormat;
 import org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat;
+import org.apache.lucene.codecs.lucene91.Lucene91Codec;
+import org.apache.lucene.codecs.lucene91.Lucene91HnswVectorsFormat;
 import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
 import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
 import org.apache.lucene.document.BinaryDocValuesField;
@@ -1236,7 +1236,7 @@ public final class TestUtil {
    * different than {@link Codec#getDefault()} because that is randomized.
    */
   public static Codec getDefaultCodec() {
-    return new Lucene90Codec();
+    return new Lucene91Codec();
   }
 
   /**
@@ -1322,7 +1322,7 @@ public final class TestUtil {
    * Lucene.
    */
   public static KnnVectorsFormat getDefaultKnnVectorsFormat() {
-    return new Lucene90HnswVectorsFormat();
+    return new Lucene91HnswVectorsFormat();
   }
 
   public static boolean anyFilesExceptWriteLock(Directory dir) throws IOException {