You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2022/11/17 21:25:32 UTC

[GitHub] [lucene] jpountz commented on a diff in pull request #11860: GITHUB-11830 Better optimize storage for vector connections

jpountz commented on code in PR #11860:
URL: https://github.com/apache/lucene/pull/11860#discussion_r1025387828


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/Lucene95Codec.java:
##########
@@ -0,0 +1,197 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.codecs.lucene95;
+
+import java.util.Objects;
+import org.apache.lucene.codecs.*;
+import org.apache.lucene.codecs.lucene90.*;

Review Comment:
   We generally like to avoid star imports.



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/Lucene95Codec.java:
##########
@@ -0,0 +1,197 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.codecs.lucene95;
+
+import java.util.Objects;
+import org.apache.lucene.codecs.*;
+import org.apache.lucene.codecs.lucene90.*;
+import org.apache.lucene.codecs.lucene94.Lucene94FieldInfosFormat;
+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.4 index format

Review Comment:
   ```suggestion
    * Implements the Lucene 9.5 index format
   ```



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/Lucene95HnswVectorsFormat.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.codecs.lucene95;
+
+import java.io.IOException;
+import org.apache.lucene.codecs.KnnVectorsFormat;
+import org.apache.lucene.codecs.KnnVectorsReader;
+import org.apache.lucene.codecs.KnnVectorsWriter;
+import org.apache.lucene.codecs.lucene90.IndexedDISI;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.hnsw.HnswGraph;
+
+/**
+ * Lucene 9.4 vector format, which encodes numeric vector values and an optional associated graph
+ * connecting the documents having values. The graph is used to power HNSW search. The format
+ * consists of three files:
+ *
+ * <h2>.vec (vector data) file</h2>
+ *
+ * <p>For each field:
+ *
+ * <ul>
+ *   <li>Vector data ordered by field, document ordinal, and vector dimension. When the
+ *       vectorEncoding is BYTE, each sample is stored as a single byte. When it is FLOAT32, each
+ *       sample is stored as an IEEE float in little-endian byte order.
+ *   <li>DocIds encoded by {@link IndexedDISI#writeBitSet(DocIdSetIterator, IndexOutput, byte)},
+ *       note that only in sparse case
+ *   <li>OrdToDoc was encoded by {@link org.apache.lucene.util.packed.DirectMonotonicWriter}, note
+ *       that only in sparse case
+ * </ul>
+ *
+ * <h2>.vex (vector index)</h2>
+ *
+ * <p>Stores graphs connecting the documents for each field organized as a list of nodes' neighbours
+ * as following:
+ *
+ * <ul>
+ *   <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 if the number of the node's neighbors is less than
+ *                   the maximum number of connections allowed on this level. Padding is equal to
+ *                   ((maxConnOnLevel – the number of neighbours) * 4) bytes.
+ *             </ul>
+ *       </ul>
+ * </ul>
+ *
+ * <h2>.vem (vector metadata) file</h2>
+ *
+ * <p>For each field:
+ *
+ * <ul>
+ *   <li><b>[int32]</b> field number
+ *   <li><b>[int32]</b> vector similarity function ordinal
+ *   <li><b>[vlong]</b> offset to this field's vectors in the .vec file
+ *   <li><b>[vlong]</b> length of this field's vectors, in bytes
+ *   <li><b>[vlong]</b> offset to this field's index in the .vex file
+ *   <li><b>[vlong]</b> length of this field's index data, in bytes
+ *   <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>[int8]</b> if equals to -1, dense – all documents have values for a field. If equals to
+ *       0, sparse – some documents missing values.
+ *   <li>DocIds were encoded by {@link IndexedDISI#writeBitSet(DocIdSetIterator, IndexOutput, byte)}
+ *   <li>OrdToDoc was encoded by {@link org.apache.lucene.util.packed.DirectMonotonicWriter}, note
+ *       that only in sparse case
+ *   <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[int]</b> for levels greater than 0 list of nodes on this level, stored as
+ *             the level 0th nodes' ordinals.
+ *       </ul>
+ * </ul>
+ *
+ * @lucene.experimental
+ */
+public final class Lucene95HnswVectorsFormat extends KnnVectorsFormat {
+
+  static final String META_CODEC_NAME = "lucene95HnswVectorsFormatMeta";
+  static final String VECTOR_DATA_CODEC_NAME = "lucene95HnswVectorsFormatData";
+  static final String VECTOR_INDEX_CODEC_NAME = "lucene95HnswVectorsFormatIndex";
+  static final String META_EXTENSION = "vem";
+  static final String VECTOR_DATA_EXTENSION = "vec";
+  static final String VECTOR_INDEX_EXTENSION = "vex";
+
+  public static final int VERSION_START = 2;

Review Comment:
   Shall we reset the version to 0 since it's a new file format?



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/Lucene95HnswVectorsFormat.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.codecs.lucene95;
+
+import java.io.IOException;
+import org.apache.lucene.codecs.KnnVectorsFormat;
+import org.apache.lucene.codecs.KnnVectorsReader;
+import org.apache.lucene.codecs.KnnVectorsWriter;
+import org.apache.lucene.codecs.lucene90.IndexedDISI;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.hnsw.HnswGraph;
+
+/**
+ * Lucene 9.4 vector format, which encodes numeric vector values and an optional associated graph
+ * connecting the documents having values. The graph is used to power HNSW search. The format
+ * consists of three files:
+ *
+ * <h2>.vec (vector data) file</h2>
+ *
+ * <p>For each field:
+ *
+ * <ul>
+ *   <li>Vector data ordered by field, document ordinal, and vector dimension. When the
+ *       vectorEncoding is BYTE, each sample is stored as a single byte. When it is FLOAT32, each
+ *       sample is stored as an IEEE float in little-endian byte order.
+ *   <li>DocIds encoded by {@link IndexedDISI#writeBitSet(DocIdSetIterator, IndexOutput, byte)},
+ *       note that only in sparse case
+ *   <li>OrdToDoc was encoded by {@link org.apache.lucene.util.packed.DirectMonotonicWriter}, note
+ *       that only in sparse case
+ * </ul>
+ *
+ * <h2>.vex (vector index)</h2>
+ *
+ * <p>Stores graphs connecting the documents for each field organized as a list of nodes' neighbours
+ * as following:
+ *
+ * <ul>
+ *   <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 if the number of the node's neighbors is less than
+ *                   the maximum number of connections allowed on this level. Padding is equal to
+ *                   ((maxConnOnLevel – the number of neighbours) * 4) bytes.
+ *             </ul>
+ *       </ul>
+ * </ul>
+ *
+ * <h2>.vem (vector metadata) file</h2>
+ *
+ * <p>For each field:
+ *
+ * <ul>
+ *   <li><b>[int32]</b> field number
+ *   <li><b>[int32]</b> vector similarity function ordinal
+ *   <li><b>[vlong]</b> offset to this field's vectors in the .vec file
+ *   <li><b>[vlong]</b> length of this field's vectors, in bytes
+ *   <li><b>[vlong]</b> offset to this field's index in the .vex file
+ *   <li><b>[vlong]</b> length of this field's index data, in bytes
+ *   <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>[int8]</b> if equals to -1, dense – all documents have values for a field. If equals to
+ *       0, sparse – some documents missing values.
+ *   <li>DocIds were encoded by {@link IndexedDISI#writeBitSet(DocIdSetIterator, IndexOutput, byte)}
+ *   <li>OrdToDoc was encoded by {@link org.apache.lucene.util.packed.DirectMonotonicWriter}, note
+ *       that only in sparse case
+ *   <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[int]</b> for levels greater than 0 list of nodes on this level, stored as
+ *             the level 0th nodes' ordinals.
+ *       </ul>
+ * </ul>
+ *
+ * @lucene.experimental
+ */
+public final class Lucene95HnswVectorsFormat extends KnnVectorsFormat {
+
+  static final String META_CODEC_NAME = "lucene95HnswVectorsFormatMeta";

Review Comment:
   nit: we generally use uppercase for codec names
   ```suggestion
     static final String META_CODEC_NAME = "Lucene95HnswVectorsFormatMeta";
   ```



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/Lucene95HnswVectorsReader.java:
##########
@@ -0,0 +1,497 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.codecs.lucene95;
+
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.KnnVectorsReader;
+import org.apache.lucene.index.*;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TotalHits;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.RandomAccessInput;
+import org.apache.lucene.util.Bits;
+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.HnswGraphSearcher;
+import org.apache.lucene.util.hnsw.NeighborQueue;
+import org.apache.lucene.util.packed.DirectMonotonicReader;
+
+/**
+ * Reads vectors from the index segments along with index data structures supporting KNN search.
+ *
+ * @lucene.experimental
+ */
+public final class Lucene95HnswVectorsReader extends KnnVectorsReader {
+
+  private final FieldInfos fieldInfos;
+  private final Map<String, FieldEntry> fields = new HashMap<>();
+  private final IndexInput vectorData;
+  private final IndexInput vectorIndex;
+
+  Lucene95HnswVectorsReader(SegmentReadState state) throws IOException {
+    this.fieldInfos = state.fieldInfos;
+    int versionMeta = readMetadata(state);
+    boolean success = false;
+    try {
+      vectorData =
+          openDataInput(
+              state,
+              versionMeta,
+              Lucene95HnswVectorsFormat.VECTOR_DATA_EXTENSION,
+              Lucene95HnswVectorsFormat.VECTOR_DATA_CODEC_NAME);
+      vectorIndex =
+          openDataInput(
+              state,
+              versionMeta,
+              Lucene95HnswVectorsFormat.VECTOR_INDEX_EXTENSION,
+              Lucene95HnswVectorsFormat.VECTOR_INDEX_CODEC_NAME);
+      success = true;
+    } finally {
+      if (success == false) {
+        IOUtils.closeWhileHandlingException(this);
+      }
+    }
+  }
+
+  private int readMetadata(SegmentReadState state) throws IOException {
+    String metaFileName =
+        IndexFileNames.segmentFileName(
+            state.segmentInfo.name, state.segmentSuffix, Lucene95HnswVectorsFormat.META_EXTENSION);
+    int versionMeta = -1;
+    try (ChecksumIndexInput meta = state.directory.openChecksumInput(metaFileName, state.context)) {
+      Throwable priorE = null;
+      try {
+        versionMeta =
+            CodecUtil.checkIndexHeader(
+                meta,
+                Lucene95HnswVectorsFormat.META_CODEC_NAME,
+                Lucene95HnswVectorsFormat.VERSION_START,
+                Lucene95HnswVectorsFormat.VERSION_CURRENT,
+                state.segmentInfo.getId(),
+                state.segmentSuffix);
+        readFields(meta, state.fieldInfos);
+      } catch (Throwable exception) {
+        priorE = exception;
+      } finally {
+        CodecUtil.checkFooter(meta, priorE);
+      }
+    }
+    return versionMeta;
+  }
+
+  private static IndexInput openDataInput(
+      SegmentReadState state, int versionMeta, String fileExtension, String codecName)
+      throws IOException {
+    String fileName =
+        IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, fileExtension);
+    IndexInput in = state.directory.openInput(fileName, state.context);
+    boolean success = false;
+    try {
+      int versionVectorData =
+          CodecUtil.checkIndexHeader(
+              in,
+              codecName,
+              Lucene95HnswVectorsFormat.VERSION_START,
+              Lucene95HnswVectorsFormat.VERSION_CURRENT,
+              state.segmentInfo.getId(),
+              state.segmentSuffix);
+      if (versionMeta != versionVectorData) {
+        throw new CorruptIndexException(
+            "Format versions mismatch: meta="
+                + versionMeta
+                + ", "
+                + codecName
+                + "="
+                + versionVectorData,
+            in);
+      }
+      CodecUtil.retrieveChecksum(in);
+      success = true;
+      return in;
+    } finally {
+      if (success == false) {
+        IOUtils.closeWhileHandlingException(in);
+      }
+    }
+  }
+
+  private void readFields(ChecksumIndexInput meta, FieldInfos infos) throws IOException {
+    for (int fieldNumber = meta.readInt(); fieldNumber != -1; fieldNumber = meta.readInt()) {
+      FieldInfo info = infos.fieldInfo(fieldNumber);
+      if (info == null) {
+        throw new CorruptIndexException("Invalid field number: " + fieldNumber, meta);
+      }
+      FieldEntry fieldEntry = readField(meta);
+      validateFieldEntry(info, fieldEntry);
+      fields.put(info.name, fieldEntry);
+    }
+  }
+
+  private void validateFieldEntry(FieldInfo info, FieldEntry fieldEntry) {
+    int dimension = info.getVectorDimension();
+    if (dimension != fieldEntry.dimension) {
+      throw new IllegalStateException(
+          "Inconsistent vector dimension for field=\""
+              + info.name
+              + "\"; "
+              + dimension
+              + " != "
+              + fieldEntry.dimension);
+    }
+
+    int byteSize =
+        switch (info.getVectorEncoding()) {
+          case BYTE -> Byte.BYTES;
+          case FLOAT32 -> Float.BYTES;
+        };
+    long vectorBytes = Math.multiplyExact((long) dimension, byteSize);
+    long numBytes = Math.multiplyExact(vectorBytes, fieldEntry.size);
+    if (numBytes != fieldEntry.vectorDataLength) {
+      throw new IllegalStateException(
+          "Vector data length "
+              + fieldEntry.vectorDataLength
+              + " not matching size="
+              + fieldEntry.size
+              + " * dim="
+              + dimension
+              + " * byteSize="
+              + byteSize
+              + " = "
+              + numBytes);
+    }
+  }
+
+  private VectorSimilarityFunction readSimilarityFunction(DataInput input) throws IOException {
+    int similarityFunctionId = input.readInt();
+    if (similarityFunctionId < 0
+        || similarityFunctionId >= VectorSimilarityFunction.values().length) {
+      throw new CorruptIndexException(
+          "Invalid similarity function id: " + similarityFunctionId, input);
+    }
+    return VectorSimilarityFunction.values()[similarityFunctionId];
+  }
+
+  private VectorEncoding readVectorEncoding(DataInput input) throws IOException {
+    int encodingId = input.readInt();
+    if (encodingId < 0 || encodingId >= VectorEncoding.values().length) {
+      throw new CorruptIndexException("Invalid vector encoding id: " + encodingId, input);
+    }
+    return VectorEncoding.values()[encodingId];
+  }
+
+  private FieldEntry readField(IndexInput input) throws IOException {
+    VectorEncoding vectorEncoding = readVectorEncoding(input);
+    VectorSimilarityFunction similarityFunction = readSimilarityFunction(input);
+    return new FieldEntry(input, vectorEncoding, similarityFunction);
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    long totalBytes = RamUsageEstimator.shallowSizeOfInstance(Lucene95HnswVectorsFormat.class);
+    totalBytes +=
+        RamUsageEstimator.sizeOfMap(
+            fields, RamUsageEstimator.shallowSizeOfInstance(FieldEntry.class));
+    return totalBytes;

Review Comment:
   It looks like it's significantly underestimating memory usage, it also needs to include the DirectMonotonic metas and the int[]?



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/Lucene95HnswVectorsReader.java:
##########
@@ -0,0 +1,497 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.codecs.lucene95;
+
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.KnnVectorsReader;
+import org.apache.lucene.index.*;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TotalHits;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.RandomAccessInput;
+import org.apache.lucene.util.Bits;
+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.HnswGraphSearcher;
+import org.apache.lucene.util.hnsw.NeighborQueue;
+import org.apache.lucene.util.packed.DirectMonotonicReader;
+
+/**
+ * Reads vectors from the index segments along with index data structures supporting KNN search.
+ *
+ * @lucene.experimental
+ */
+public final class Lucene95HnswVectorsReader extends KnnVectorsReader {
+
+  private final FieldInfos fieldInfos;
+  private final Map<String, FieldEntry> fields = new HashMap<>();
+  private final IndexInput vectorData;
+  private final IndexInput vectorIndex;
+
+  Lucene95HnswVectorsReader(SegmentReadState state) throws IOException {
+    this.fieldInfos = state.fieldInfos;
+    int versionMeta = readMetadata(state);
+    boolean success = false;
+    try {
+      vectorData =
+          openDataInput(
+              state,
+              versionMeta,
+              Lucene95HnswVectorsFormat.VECTOR_DATA_EXTENSION,
+              Lucene95HnswVectorsFormat.VECTOR_DATA_CODEC_NAME);
+      vectorIndex =
+          openDataInput(
+              state,
+              versionMeta,
+              Lucene95HnswVectorsFormat.VECTOR_INDEX_EXTENSION,
+              Lucene95HnswVectorsFormat.VECTOR_INDEX_CODEC_NAME);
+      success = true;
+    } finally {
+      if (success == false) {
+        IOUtils.closeWhileHandlingException(this);
+      }
+    }
+  }
+
+  private int readMetadata(SegmentReadState state) throws IOException {
+    String metaFileName =
+        IndexFileNames.segmentFileName(
+            state.segmentInfo.name, state.segmentSuffix, Lucene95HnswVectorsFormat.META_EXTENSION);
+    int versionMeta = -1;
+    try (ChecksumIndexInput meta = state.directory.openChecksumInput(metaFileName, state.context)) {
+      Throwable priorE = null;
+      try {
+        versionMeta =
+            CodecUtil.checkIndexHeader(
+                meta,
+                Lucene95HnswVectorsFormat.META_CODEC_NAME,
+                Lucene95HnswVectorsFormat.VERSION_START,
+                Lucene95HnswVectorsFormat.VERSION_CURRENT,
+                state.segmentInfo.getId(),
+                state.segmentSuffix);
+        readFields(meta, state.fieldInfos);
+      } catch (Throwable exception) {
+        priorE = exception;
+      } finally {
+        CodecUtil.checkFooter(meta, priorE);
+      }
+    }
+    return versionMeta;
+  }
+
+  private static IndexInput openDataInput(
+      SegmentReadState state, int versionMeta, String fileExtension, String codecName)
+      throws IOException {
+    String fileName =
+        IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, fileExtension);
+    IndexInput in = state.directory.openInput(fileName, state.context);
+    boolean success = false;
+    try {
+      int versionVectorData =
+          CodecUtil.checkIndexHeader(
+              in,
+              codecName,
+              Lucene95HnswVectorsFormat.VERSION_START,
+              Lucene95HnswVectorsFormat.VERSION_CURRENT,
+              state.segmentInfo.getId(),
+              state.segmentSuffix);
+      if (versionMeta != versionVectorData) {
+        throw new CorruptIndexException(
+            "Format versions mismatch: meta="
+                + versionMeta
+                + ", "
+                + codecName
+                + "="
+                + versionVectorData,
+            in);
+      }
+      CodecUtil.retrieveChecksum(in);
+      success = true;
+      return in;
+    } finally {
+      if (success == false) {
+        IOUtils.closeWhileHandlingException(in);
+      }
+    }
+  }
+
+  private void readFields(ChecksumIndexInput meta, FieldInfos infos) throws IOException {
+    for (int fieldNumber = meta.readInt(); fieldNumber != -1; fieldNumber = meta.readInt()) {
+      FieldInfo info = infos.fieldInfo(fieldNumber);
+      if (info == null) {
+        throw new CorruptIndexException("Invalid field number: " + fieldNumber, meta);
+      }
+      FieldEntry fieldEntry = readField(meta);
+      validateFieldEntry(info, fieldEntry);
+      fields.put(info.name, fieldEntry);
+    }
+  }
+
+  private void validateFieldEntry(FieldInfo info, FieldEntry fieldEntry) {
+    int dimension = info.getVectorDimension();
+    if (dimension != fieldEntry.dimension) {
+      throw new IllegalStateException(
+          "Inconsistent vector dimension for field=\""
+              + info.name
+              + "\"; "
+              + dimension
+              + " != "
+              + fieldEntry.dimension);
+    }
+
+    int byteSize =
+        switch (info.getVectorEncoding()) {
+          case BYTE -> Byte.BYTES;
+          case FLOAT32 -> Float.BYTES;
+        };
+    long vectorBytes = Math.multiplyExact((long) dimension, byteSize);
+    long numBytes = Math.multiplyExact(vectorBytes, fieldEntry.size);
+    if (numBytes != fieldEntry.vectorDataLength) {
+      throw new IllegalStateException(
+          "Vector data length "
+              + fieldEntry.vectorDataLength
+              + " not matching size="
+              + fieldEntry.size
+              + " * dim="
+              + dimension
+              + " * byteSize="
+              + byteSize
+              + " = "
+              + numBytes);
+    }
+  }
+
+  private VectorSimilarityFunction readSimilarityFunction(DataInput input) throws IOException {
+    int similarityFunctionId = input.readInt();
+    if (similarityFunctionId < 0
+        || similarityFunctionId >= VectorSimilarityFunction.values().length) {
+      throw new CorruptIndexException(
+          "Invalid similarity function id: " + similarityFunctionId, input);
+    }
+    return VectorSimilarityFunction.values()[similarityFunctionId];
+  }
+
+  private VectorEncoding readVectorEncoding(DataInput input) throws IOException {
+    int encodingId = input.readInt();
+    if (encodingId < 0 || encodingId >= VectorEncoding.values().length) {
+      throw new CorruptIndexException("Invalid vector encoding id: " + encodingId, input);
+    }
+    return VectorEncoding.values()[encodingId];
+  }
+
+  private FieldEntry readField(IndexInput input) throws IOException {
+    VectorEncoding vectorEncoding = readVectorEncoding(input);
+    VectorSimilarityFunction similarityFunction = readSimilarityFunction(input);
+    return new FieldEntry(input, vectorEncoding, similarityFunction);
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    long totalBytes = RamUsageEstimator.shallowSizeOfInstance(Lucene95HnswVectorsFormat.class);
+    totalBytes +=
+        RamUsageEstimator.sizeOfMap(
+            fields, RamUsageEstimator.shallowSizeOfInstance(FieldEntry.class));
+    return totalBytes;
+  }
+
+  @Override
+  public void checkIntegrity() throws IOException {
+    CodecUtil.checksumEntireFile(vectorData);
+    CodecUtil.checksumEntireFile(vectorIndex);
+  }
+
+  @Override
+  public VectorValues getVectorValues(String field) throws IOException {
+    FieldEntry fieldEntry = fields.get(field);
+    VectorValues values = OffHeapVectorValues.load(fieldEntry, vectorData);
+    if (fieldEntry.vectorEncoding == VectorEncoding.BYTE) {
+      return new ExpandingVectorValues(values);
+    } else {
+      return values;
+    }
+  }
+
+  @Override
+  public TopDocs search(String field, float[] target, int k, Bits acceptDocs, int visitedLimit)
+      throws IOException {
+    FieldEntry fieldEntry = fields.get(field);
+
+    if (fieldEntry.size() == 0) {
+      return new TopDocs(new TotalHits(0, TotalHits.Relation.EQUAL_TO), new ScoreDoc[0]);
+    }
+
+    // bound k by total number of vectors to prevent oversizing data structures
+    k = Math.min(k, fieldEntry.size());
+    OffHeapVectorValues vectorValues = OffHeapVectorValues.load(fieldEntry, vectorData);
+
+    NeighborQueue results =
+        HnswGraphSearcher.search(
+            target,
+            k,
+            vectorValues,
+            fieldEntry.vectorEncoding,
+            fieldEntry.similarityFunction,
+            getGraph(fieldEntry),
+            vectorValues.getAcceptOrds(acceptDocs),
+            visitedLimit);
+
+    int i = 0;
+    ScoreDoc[] scoreDocs = new ScoreDoc[Math.min(results.size(), k)];
+    while (results.size() > 0) {
+      int node = results.topNode();
+      float score = results.topScore();
+      results.pop();
+      scoreDocs[scoreDocs.length - ++i] = new ScoreDoc(vectorValues.ordToDoc(node), score);
+    }
+
+    TotalHits.Relation relation =
+        results.incomplete()
+            ? TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO
+            : TotalHits.Relation.EQUAL_TO;
+    return new TopDocs(new TotalHits(results.visitedCount(), relation), scoreDocs);
+  }
+
+  /** Get knn graph values; used for testing */
+  public HnswGraph getGraph(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.vectorIndexLength > 0) {
+      return getGraph(entry);
+    } else {
+      return HnswGraph.EMPTY;
+    }
+  }
+
+  private HnswGraph getGraph(FieldEntry entry) throws IOException {
+    return new OffHeapHnswGraph(entry, vectorIndex);
+  }
+
+  @Override
+  public void close() throws IOException {
+    IOUtils.close(vectorData, vectorIndex);
+  }
+
+  static class FieldEntry {
+
+    final VectorSimilarityFunction similarityFunction;
+    final VectorEncoding vectorEncoding;
+    final long vectorDataOffset;
+    final long vectorDataLength;
+    final long vectorIndexOffset;
+    final long vectorIndexLength;
+    final int M;
+    final int numLevels;
+    final int dimension;
+    final int size;
+    final int[][] nodesByLevel;
+    // for each level the start offsets in vectorIndex file from where to read neighbours
+    final DirectMonotonicReader.Meta offsetsMeta;
+    final long offsetsOffset;
+    final int offsetsBlockShift;
+    final long offsetsLength;
+
+    // the following four variables used to read docIds encoded by IndexDISI
+    // special values of docsWithFieldOffset are -1 and -2
+    // -1 : dense
+    // -2 : empty
+    // other: sparse
+    final long docsWithFieldOffset;
+    final long docsWithFieldLength;
+    final short jumpTableEntryCount;
+    final byte denseRankPower;
+
+    // the following four variables used to read ordToDoc encoded by DirectMonotonicWriter
+    // note that only spare case needs to store ordToDoc
+    final long addressesOffset;
+    final int blockShift;
+    final DirectMonotonicReader.Meta meta;
+    final long addressesLength;
+
+    FieldEntry(
+        IndexInput input,
+        VectorEncoding vectorEncoding,
+        VectorSimilarityFunction similarityFunction)
+        throws IOException {
+      this.similarityFunction = similarityFunction;
+      this.vectorEncoding = vectorEncoding;
+      vectorDataOffset = input.readVLong();
+      vectorDataLength = input.readVLong();
+      vectorIndexOffset = input.readVLong();
+      vectorIndexLength = input.readVLong();
+      dimension = input.readVInt();
+      size = input.readInt();
+
+      docsWithFieldOffset = input.readLong();
+      docsWithFieldLength = input.readLong();
+      jumpTableEntryCount = input.readShort();
+      denseRankPower = input.readByte();
+
+      // dense or empty
+      if (docsWithFieldOffset == -1 || docsWithFieldOffset == -2) {
+        addressesOffset = 0;
+        blockShift = 0;
+        meta = null;
+        addressesLength = 0;
+      } else {
+        // sparse
+        addressesOffset = input.readLong();
+        blockShift = input.readVInt();
+        meta = DirectMonotonicReader.loadMeta(input, size, blockShift);
+        addressesLength = input.readLong();
+      }
+
+      // read nodes by level
+      M = input.readVInt();
+      numLevels = input.readVInt();
+      nodesByLevel = new int[numLevels][];
+      long numberOfOffsets = 0;
+      for (int level = 0; level < numLevels; level++) {
+        if (level > 0) {
+          int numNodesOnLevel = input.readVInt();
+          numberOfOffsets += numNodesOnLevel;
+          nodesByLevel[level] = new int[numNodesOnLevel];
+          for (int i = 0; i < numNodesOnLevel; i++) {
+            nodesByLevel[level][i] = input.readInt();
+          }

Review Comment:
   you could use `input.readInts(nodesByLevel[level])` instead of this naive loop



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/Lucene95HnswVectorsReader.java:
##########
@@ -0,0 +1,497 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.codecs.lucene95;
+
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.KnnVectorsReader;
+import org.apache.lucene.index.*;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TotalHits;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.RandomAccessInput;
+import org.apache.lucene.util.Bits;
+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.HnswGraphSearcher;
+import org.apache.lucene.util.hnsw.NeighborQueue;
+import org.apache.lucene.util.packed.DirectMonotonicReader;
+
+/**
+ * Reads vectors from the index segments along with index data structures supporting KNN search.
+ *
+ * @lucene.experimental
+ */
+public final class Lucene95HnswVectorsReader extends KnnVectorsReader {
+
+  private final FieldInfos fieldInfos;
+  private final Map<String, FieldEntry> fields = new HashMap<>();
+  private final IndexInput vectorData;
+  private final IndexInput vectorIndex;
+
+  Lucene95HnswVectorsReader(SegmentReadState state) throws IOException {
+    this.fieldInfos = state.fieldInfos;
+    int versionMeta = readMetadata(state);
+    boolean success = false;
+    try {
+      vectorData =
+          openDataInput(
+              state,
+              versionMeta,
+              Lucene95HnswVectorsFormat.VECTOR_DATA_EXTENSION,
+              Lucene95HnswVectorsFormat.VECTOR_DATA_CODEC_NAME);
+      vectorIndex =
+          openDataInput(
+              state,
+              versionMeta,
+              Lucene95HnswVectorsFormat.VECTOR_INDEX_EXTENSION,
+              Lucene95HnswVectorsFormat.VECTOR_INDEX_CODEC_NAME);
+      success = true;
+    } finally {
+      if (success == false) {
+        IOUtils.closeWhileHandlingException(this);
+      }
+    }
+  }
+
+  private int readMetadata(SegmentReadState state) throws IOException {
+    String metaFileName =
+        IndexFileNames.segmentFileName(
+            state.segmentInfo.name, state.segmentSuffix, Lucene95HnswVectorsFormat.META_EXTENSION);
+    int versionMeta = -1;
+    try (ChecksumIndexInput meta = state.directory.openChecksumInput(metaFileName, state.context)) {
+      Throwable priorE = null;
+      try {
+        versionMeta =
+            CodecUtil.checkIndexHeader(
+                meta,
+                Lucene95HnswVectorsFormat.META_CODEC_NAME,
+                Lucene95HnswVectorsFormat.VERSION_START,
+                Lucene95HnswVectorsFormat.VERSION_CURRENT,
+                state.segmentInfo.getId(),
+                state.segmentSuffix);
+        readFields(meta, state.fieldInfos);
+      } catch (Throwable exception) {
+        priorE = exception;
+      } finally {
+        CodecUtil.checkFooter(meta, priorE);
+      }
+    }
+    return versionMeta;
+  }
+
+  private static IndexInput openDataInput(
+      SegmentReadState state, int versionMeta, String fileExtension, String codecName)
+      throws IOException {
+    String fileName =
+        IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, fileExtension);
+    IndexInput in = state.directory.openInput(fileName, state.context);
+    boolean success = false;
+    try {
+      int versionVectorData =
+          CodecUtil.checkIndexHeader(
+              in,
+              codecName,
+              Lucene95HnswVectorsFormat.VERSION_START,
+              Lucene95HnswVectorsFormat.VERSION_CURRENT,
+              state.segmentInfo.getId(),
+              state.segmentSuffix);
+      if (versionMeta != versionVectorData) {
+        throw new CorruptIndexException(
+            "Format versions mismatch: meta="
+                + versionMeta
+                + ", "
+                + codecName
+                + "="
+                + versionVectorData,
+            in);
+      }
+      CodecUtil.retrieveChecksum(in);
+      success = true;
+      return in;
+    } finally {
+      if (success == false) {
+        IOUtils.closeWhileHandlingException(in);
+      }
+    }
+  }
+
+  private void readFields(ChecksumIndexInput meta, FieldInfos infos) throws IOException {
+    for (int fieldNumber = meta.readInt(); fieldNumber != -1; fieldNumber = meta.readInt()) {
+      FieldInfo info = infos.fieldInfo(fieldNumber);
+      if (info == null) {
+        throw new CorruptIndexException("Invalid field number: " + fieldNumber, meta);
+      }
+      FieldEntry fieldEntry = readField(meta);
+      validateFieldEntry(info, fieldEntry);
+      fields.put(info.name, fieldEntry);
+    }
+  }
+
+  private void validateFieldEntry(FieldInfo info, FieldEntry fieldEntry) {
+    int dimension = info.getVectorDimension();
+    if (dimension != fieldEntry.dimension) {
+      throw new IllegalStateException(
+          "Inconsistent vector dimension for field=\""
+              + info.name
+              + "\"; "
+              + dimension
+              + " != "
+              + fieldEntry.dimension);
+    }
+
+    int byteSize =
+        switch (info.getVectorEncoding()) {
+          case BYTE -> Byte.BYTES;
+          case FLOAT32 -> Float.BYTES;
+        };
+    long vectorBytes = Math.multiplyExact((long) dimension, byteSize);
+    long numBytes = Math.multiplyExact(vectorBytes, fieldEntry.size);
+    if (numBytes != fieldEntry.vectorDataLength) {
+      throw new IllegalStateException(
+          "Vector data length "
+              + fieldEntry.vectorDataLength
+              + " not matching size="
+              + fieldEntry.size
+              + " * dim="
+              + dimension
+              + " * byteSize="
+              + byteSize
+              + " = "
+              + numBytes);
+    }
+  }
+
+  private VectorSimilarityFunction readSimilarityFunction(DataInput input) throws IOException {
+    int similarityFunctionId = input.readInt();
+    if (similarityFunctionId < 0
+        || similarityFunctionId >= VectorSimilarityFunction.values().length) {
+      throw new CorruptIndexException(
+          "Invalid similarity function id: " + similarityFunctionId, input);
+    }
+    return VectorSimilarityFunction.values()[similarityFunctionId];
+  }
+
+  private VectorEncoding readVectorEncoding(DataInput input) throws IOException {
+    int encodingId = input.readInt();
+    if (encodingId < 0 || encodingId >= VectorEncoding.values().length) {
+      throw new CorruptIndexException("Invalid vector encoding id: " + encodingId, input);
+    }
+    return VectorEncoding.values()[encodingId];
+  }
+
+  private FieldEntry readField(IndexInput input) throws IOException {
+    VectorEncoding vectorEncoding = readVectorEncoding(input);
+    VectorSimilarityFunction similarityFunction = readSimilarityFunction(input);
+    return new FieldEntry(input, vectorEncoding, similarityFunction);
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    long totalBytes = RamUsageEstimator.shallowSizeOfInstance(Lucene95HnswVectorsFormat.class);
+    totalBytes +=
+        RamUsageEstimator.sizeOfMap(
+            fields, RamUsageEstimator.shallowSizeOfInstance(FieldEntry.class));
+    return totalBytes;
+  }
+
+  @Override
+  public void checkIntegrity() throws IOException {
+    CodecUtil.checksumEntireFile(vectorData);
+    CodecUtil.checksumEntireFile(vectorIndex);
+  }
+
+  @Override
+  public VectorValues getVectorValues(String field) throws IOException {
+    FieldEntry fieldEntry = fields.get(field);
+    VectorValues values = OffHeapVectorValues.load(fieldEntry, vectorData);
+    if (fieldEntry.vectorEncoding == VectorEncoding.BYTE) {
+      return new ExpandingVectorValues(values);
+    } else {
+      return values;
+    }
+  }
+
+  @Override
+  public TopDocs search(String field, float[] target, int k, Bits acceptDocs, int visitedLimit)
+      throws IOException {
+    FieldEntry fieldEntry = fields.get(field);
+
+    if (fieldEntry.size() == 0) {
+      return new TopDocs(new TotalHits(0, TotalHits.Relation.EQUAL_TO), new ScoreDoc[0]);
+    }
+
+    // bound k by total number of vectors to prevent oversizing data structures
+    k = Math.min(k, fieldEntry.size());
+    OffHeapVectorValues vectorValues = OffHeapVectorValues.load(fieldEntry, vectorData);
+
+    NeighborQueue results =
+        HnswGraphSearcher.search(
+            target,
+            k,
+            vectorValues,
+            fieldEntry.vectorEncoding,
+            fieldEntry.similarityFunction,
+            getGraph(fieldEntry),
+            vectorValues.getAcceptOrds(acceptDocs),
+            visitedLimit);
+
+    int i = 0;
+    ScoreDoc[] scoreDocs = new ScoreDoc[Math.min(results.size(), k)];
+    while (results.size() > 0) {
+      int node = results.topNode();
+      float score = results.topScore();
+      results.pop();
+      scoreDocs[scoreDocs.length - ++i] = new ScoreDoc(vectorValues.ordToDoc(node), score);
+    }
+
+    TotalHits.Relation relation =
+        results.incomplete()
+            ? TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO
+            : TotalHits.Relation.EQUAL_TO;
+    return new TopDocs(new TotalHits(results.visitedCount(), relation), scoreDocs);
+  }
+
+  /** Get knn graph values; used for testing */
+  public HnswGraph getGraph(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.vectorIndexLength > 0) {
+      return getGraph(entry);
+    } else {
+      return HnswGraph.EMPTY;
+    }
+  }
+
+  private HnswGraph getGraph(FieldEntry entry) throws IOException {
+    return new OffHeapHnswGraph(entry, vectorIndex);
+  }
+
+  @Override
+  public void close() throws IOException {
+    IOUtils.close(vectorData, vectorIndex);
+  }
+
+  static class FieldEntry {
+
+    final VectorSimilarityFunction similarityFunction;
+    final VectorEncoding vectorEncoding;
+    final long vectorDataOffset;
+    final long vectorDataLength;
+    final long vectorIndexOffset;
+    final long vectorIndexLength;
+    final int M;
+    final int numLevels;
+    final int dimension;
+    final int size;
+    final int[][] nodesByLevel;
+    // for each level the start offsets in vectorIndex file from where to read neighbours
+    final DirectMonotonicReader.Meta offsetsMeta;
+    final long offsetsOffset;
+    final int offsetsBlockShift;
+    final long offsetsLength;
+
+    // the following four variables used to read docIds encoded by IndexDISI
+    // special values of docsWithFieldOffset are -1 and -2
+    // -1 : dense
+    // -2 : empty
+    // other: sparse
+    final long docsWithFieldOffset;
+    final long docsWithFieldLength;
+    final short jumpTableEntryCount;
+    final byte denseRankPower;
+
+    // the following four variables used to read ordToDoc encoded by DirectMonotonicWriter
+    // note that only spare case needs to store ordToDoc
+    final long addressesOffset;
+    final int blockShift;
+    final DirectMonotonicReader.Meta meta;
+    final long addressesLength;
+
+    FieldEntry(
+        IndexInput input,
+        VectorEncoding vectorEncoding,
+        VectorSimilarityFunction similarityFunction)
+        throws IOException {
+      this.similarityFunction = similarityFunction;
+      this.vectorEncoding = vectorEncoding;
+      vectorDataOffset = input.readVLong();
+      vectorDataLength = input.readVLong();
+      vectorIndexOffset = input.readVLong();
+      vectorIndexLength = input.readVLong();
+      dimension = input.readVInt();
+      size = input.readInt();
+
+      docsWithFieldOffset = input.readLong();
+      docsWithFieldLength = input.readLong();
+      jumpTableEntryCount = input.readShort();
+      denseRankPower = input.readByte();
+
+      // dense or empty
+      if (docsWithFieldOffset == -1 || docsWithFieldOffset == -2) {
+        addressesOffset = 0;
+        blockShift = 0;
+        meta = null;
+        addressesLength = 0;
+      } else {
+        // sparse
+        addressesOffset = input.readLong();
+        blockShift = input.readVInt();
+        meta = DirectMonotonicReader.loadMeta(input, size, blockShift);
+        addressesLength = input.readLong();
+      }
+
+      // read nodes by level
+      M = input.readVInt();
+      numLevels = input.readVInt();
+      nodesByLevel = new int[numLevels][];
+      long numberOfOffsets = 0;
+      for (int level = 0; level < numLevels; level++) {
+        if (level > 0) {
+          int numNodesOnLevel = input.readVInt();
+          numberOfOffsets += numNodesOnLevel;
+          nodesByLevel[level] = new int[numNodesOnLevel];
+          for (int i = 0; i < numNodesOnLevel; i++) {
+            nodesByLevel[level][i] = input.readInt();
+          }
+        } else {
+          numberOfOffsets += size;
+        }
+      }

Review Comment:
   I wouldn't compress these ints with delta+vint if they get loaded in memory in this 2D array uncompressed anyway?



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/package-info.java:
##########
@@ -0,0 +1,423 @@
+/*
+ * 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.5 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.lucene94.Lucene94FieldInfosFormat 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.lucene95.Lucene95HnswVectorsFormat 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.lucene94.Lucene94FieldInfosFormat 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.lucene95.Lucene95HnswVectorsFormat 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.
+ *   <li>In version 9.1, vector-valued fields were modified to add a graph hierarchy.
+ *   <li>In version 9.2, docs of vector-valued fields were moved from .vem to .vec and encoded by
+ *       IndexDISI. ordToDoc mappings was added to .vem.
+ *   <li>In version 9.5, HNSW graph connections were changed to be stored with PackedInts.

Review Comment:
   ```suggestion
    *   <li>In version 9.5, HNSW graph connections were changed to be delta-encoded with vints.
   ```



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/Lucene95Codec.java:
##########
@@ -0,0 +1,197 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.codecs.lucene95;
+
+import java.util.Objects;
+import org.apache.lucene.codecs.*;
+import org.apache.lucene.codecs.lucene90.*;
+import org.apache.lucene.codecs.lucene94.Lucene94FieldInfosFormat;
+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.4 index format
+ *
+ * <p>If you want to reuse functionality of this codec in another codec, extend {@link FilterCodec}.
+ *
+ * @see org.apache.lucene.codecs.lucene95 package documentation for file format details.
+ * @lucene.experimental
+ */
+public class Lucene95Codec extends Codec {
+
+  /** Configuration option for the codec. */
+  public enum Mode {
+    /** Trade compression ratio for retrieval speed. */
+    BEST_SPEED(Lucene90StoredFieldsFormat.Mode.BEST_SPEED),
+    /** Trade retrieval speed for compression ratio. */
+    BEST_COMPRESSION(Lucene90StoredFieldsFormat.Mode.BEST_COMPRESSION);
+
+    private final Lucene90StoredFieldsFormat.Mode storedMode;
+
+    private Mode(Lucene90StoredFieldsFormat.Mode storedMode) {
+      this.storedMode = Objects.requireNonNull(storedMode);
+    }
+  }
+
+  private final TermVectorsFormat vectorsFormat = new Lucene90TermVectorsFormat();
+  private final FieldInfosFormat fieldInfosFormat = new Lucene94FieldInfosFormat();
+  private final SegmentInfoFormat segmentInfosFormat = new Lucene90SegmentInfoFormat();
+  private final LiveDocsFormat liveDocsFormat = new Lucene90LiveDocsFormat();
+  private final CompoundFormat compoundFormat = new Lucene90CompoundFormat();
+  private final NormsFormat normsFormat = new Lucene90NormsFormat();
+
+  private final PostingsFormat defaultPostingsFormat;
+  private final PostingsFormat postingsFormat =
+      new PerFieldPostingsFormat() {
+        @Override
+        public PostingsFormat getPostingsFormatForField(String field) {
+          return Lucene95Codec.this.getPostingsFormatForField(field);
+        }
+      };
+
+  private final DocValuesFormat defaultDVFormat;
+  private final DocValuesFormat docValuesFormat =
+      new PerFieldDocValuesFormat() {
+        @Override
+        public DocValuesFormat getDocValuesFormatForField(String field) {
+          return Lucene95Codec.this.getDocValuesFormatForField(field);
+        }
+      };
+
+  private final KnnVectorsFormat defaultKnnVectorsFormat;
+  private final KnnVectorsFormat knnVectorsFormat =
+      new PerFieldKnnVectorsFormat() {
+        @Override
+        public KnnVectorsFormat getKnnVectorsFormatForField(String field) {
+          return Lucene95Codec.this.getKnnVectorsFormatForField(field);
+        }
+      };
+
+  private final StoredFieldsFormat storedFieldsFormat;
+
+  /** Instantiates a new codec. */
+  public Lucene95Codec() {
+    this(Mode.BEST_SPEED);
+  }
+
+  /**
+   * Instantiates a new codec, specifying the stored fields compression mode to use.
+   *
+   * @param mode stored fields compression mode to use for newly flushed/merged segments.
+   */
+  public Lucene95Codec(Mode mode) {
+    super("Lucene95");
+    this.storedFieldsFormat =
+        new Lucene90StoredFieldsFormat(Objects.requireNonNull(mode).storedMode);
+    this.defaultPostingsFormat = new Lucene90PostingsFormat();
+    this.defaultDVFormat = new Lucene90DocValuesFormat();
+    this.defaultKnnVectorsFormat = new Lucene95HnswVectorsFormat();
+  }
+
+  @Override
+  public final StoredFieldsFormat storedFieldsFormat() {
+    return storedFieldsFormat;
+  }
+
+  @Override
+  public final TermVectorsFormat termVectorsFormat() {
+    return vectorsFormat;
+  }
+
+  @Override
+  public final PostingsFormat postingsFormat() {
+    return postingsFormat;
+  }
+
+  @Override
+  public final FieldInfosFormat fieldInfosFormat() {
+    return fieldInfosFormat;
+  }
+
+  @Override
+  public final SegmentInfoFormat segmentInfoFormat() {
+    return segmentInfosFormat;
+  }
+
+  @Override
+  public final LiveDocsFormat liveDocsFormat() {
+    return liveDocsFormat;
+  }
+
+  @Override
+  public final CompoundFormat compoundFormat() {
+    return compoundFormat;
+  }
+
+  @Override
+  public final PointsFormat pointsFormat() {
+    return new Lucene90PointsFormat();
+  }
+
+  @Override
+  public final KnnVectorsFormat knnVectorsFormat() {
+    return knnVectorsFormat;
+  }
+
+  /**
+   * Returns the postings format that should be used for writing new segments of <code>field</code>.
+   *
+   * <p>The default implementation always returns "Lucene90".
+   *
+   * <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,
+   */
+  public PostingsFormat getPostingsFormatForField(String field) {
+    return defaultPostingsFormat;
+  }
+
+  /**
+   * Returns the docvalues format that should be used for writing new segments of <code>field</code>
+   * .
+   *
+   * <p>The default implementation always returns "Lucene90".
+   *
+   * <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.
+   */
+  public DocValuesFormat getDocValuesFormatForField(String field) {
+    return defaultDVFormat;
+  }
+
+  /**
+   * Returns the vectors format that should be used for writing new segments of <code>field</code>
+   *
+   * <p>The default implementation always returns "lucene95".

Review Comment:
   ```suggestion
      * <p>The default implementation always returns "Lucene95".
   ```



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/Lucene95HnswVectorsFormat.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.codecs.lucene95;
+
+import java.io.IOException;
+import org.apache.lucene.codecs.KnnVectorsFormat;
+import org.apache.lucene.codecs.KnnVectorsReader;
+import org.apache.lucene.codecs.KnnVectorsWriter;
+import org.apache.lucene.codecs.lucene90.IndexedDISI;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.hnsw.HnswGraph;
+
+/**
+ * Lucene 9.4 vector format, which encodes numeric vector values and an optional associated graph
+ * connecting the documents having values. The graph is used to power HNSW search. The format
+ * consists of three files:
+ *
+ * <h2>.vec (vector data) file</h2>
+ *
+ * <p>For each field:
+ *
+ * <ul>
+ *   <li>Vector data ordered by field, document ordinal, and vector dimension. When the
+ *       vectorEncoding is BYTE, each sample is stored as a single byte. When it is FLOAT32, each
+ *       sample is stored as an IEEE float in little-endian byte order.
+ *   <li>DocIds encoded by {@link IndexedDISI#writeBitSet(DocIdSetIterator, IndexOutput, byte)},
+ *       note that only in sparse case
+ *   <li>OrdToDoc was encoded by {@link org.apache.lucene.util.packed.DirectMonotonicWriter}, note
+ *       that only in sparse case
+ * </ul>
+ *
+ * <h2>.vex (vector index)</h2>
+ *
+ * <p>Stores graphs connecting the documents for each field organized as a list of nodes' neighbours
+ * as following:
+ *
+ * <ul>
+ *   <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 if the number of the node's neighbors is less than
+ *                   the maximum number of connections allowed on this level. Padding is equal to
+ *                   ((maxConnOnLevel – the number of neighbours) * 4) bytes.
+ *             </ul>
+ *       </ul>
+ * </ul>

Review Comment:
   Update that section to reflect usage of delta + vints, and a mapping from nodes to offsets using DirectMonotonicWriter?



##########
lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene94/ExpandingVectorValues.java:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.lucene94;
+
+import java.io.IOException;
+import org.apache.lucene.index.FilterVectorValues;
+import org.apache.lucene.index.VectorValues;
+import org.apache.lucene.util.BytesRef;
+
+/** reads from byte-encoded data */
+public class ExpandingVectorValues extends FilterVectorValues {

Review Comment:
   Does it actually need to be public?



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/Lucene95HnswVectorsFormat.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.codecs.lucene95;
+
+import java.io.IOException;
+import org.apache.lucene.codecs.KnnVectorsFormat;
+import org.apache.lucene.codecs.KnnVectorsReader;
+import org.apache.lucene.codecs.KnnVectorsWriter;
+import org.apache.lucene.codecs.lucene90.IndexedDISI;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.hnsw.HnswGraph;
+
+/**
+ * Lucene 9.4 vector format, which encodes numeric vector values and an optional associated graph

Review Comment:
   ```suggestion
    * Lucene 9.5 vector format, which encodes numeric vector values and an optional associated graph
   ```



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/Lucene95HnswVectorsReader.java:
##########
@@ -0,0 +1,532 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.codecs.lucene95;
+
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.KnnVectorsReader;
+import org.apache.lucene.index.*;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TotalHits;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.Bits;
+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.HnswGraphSearcher;
+import org.apache.lucene.util.hnsw.NeighborQueue;
+import org.apache.lucene.util.packed.DirectMonotonicReader;
+import org.apache.lucene.util.packed.PackedInts;
+
+/**
+ * Reads vectors from the index segments along with index data structures supporting KNN search.
+ *
+ * @lucene.experimental
+ */
+public final class Lucene95HnswVectorsReader extends KnnVectorsReader {
+
+  private final FieldInfos fieldInfos;
+  private final Map<String, FieldEntry> fields = new HashMap<>();
+  private final IndexInput vectorData;
+  private final IndexInput vectorIndex;
+
+  Lucene95HnswVectorsReader(SegmentReadState state) throws IOException {
+    this.fieldInfos = state.fieldInfos;
+    int versionMeta = readMetadata(state);
+    boolean success = false;
+    try {
+      vectorData =
+          openDataInput(
+              state,
+              versionMeta,
+              Lucene95HnswVectorsFormat.VECTOR_DATA_EXTENSION,
+              Lucene95HnswVectorsFormat.VECTOR_DATA_CODEC_NAME);
+      vectorIndex =
+          openDataInput(
+              state,
+              versionMeta,
+              Lucene95HnswVectorsFormat.VECTOR_INDEX_EXTENSION,
+              Lucene95HnswVectorsFormat.VECTOR_INDEX_CODEC_NAME);
+      success = true;
+    } finally {
+      if (success == false) {
+        IOUtils.closeWhileHandlingException(this);
+      }
+    }
+  }
+
+  private int readMetadata(SegmentReadState state) throws IOException {
+    String metaFileName =
+        IndexFileNames.segmentFileName(
+            state.segmentInfo.name, state.segmentSuffix, Lucene95HnswVectorsFormat.META_EXTENSION);
+    int versionMeta = -1;
+    try (ChecksumIndexInput meta = state.directory.openChecksumInput(metaFileName, state.context)) {

Review Comment:
   this should use `IOContext.READONCE` as an `IOContext` rather than the default `IOContext`



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/Lucene95HnswVectorsReader.java:
##########
@@ -0,0 +1,532 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.codecs.lucene95;
+
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.KnnVectorsReader;
+import org.apache.lucene.index.*;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TotalHits;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.Bits;
+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.HnswGraphSearcher;
+import org.apache.lucene.util.hnsw.NeighborQueue;
+import org.apache.lucene.util.packed.DirectMonotonicReader;
+import org.apache.lucene.util.packed.PackedInts;
+
+/**
+ * Reads vectors from the index segments along with index data structures supporting KNN search.
+ *
+ * @lucene.experimental
+ */
+public final class Lucene95HnswVectorsReader extends KnnVectorsReader {
+
+  private final FieldInfos fieldInfos;
+  private final Map<String, FieldEntry> fields = new HashMap<>();
+  private final IndexInput vectorData;
+  private final IndexInput vectorIndex;
+
+  Lucene95HnswVectorsReader(SegmentReadState state) throws IOException {
+    this.fieldInfos = state.fieldInfos;
+    int versionMeta = readMetadata(state);
+    boolean success = false;
+    try {
+      vectorData =
+          openDataInput(
+              state,
+              versionMeta,
+              Lucene95HnswVectorsFormat.VECTOR_DATA_EXTENSION,
+              Lucene95HnswVectorsFormat.VECTOR_DATA_CODEC_NAME);
+      vectorIndex =
+          openDataInput(
+              state,
+              versionMeta,
+              Lucene95HnswVectorsFormat.VECTOR_INDEX_EXTENSION,
+              Lucene95HnswVectorsFormat.VECTOR_INDEX_CODEC_NAME);
+      success = true;
+    } finally {
+      if (success == false) {
+        IOUtils.closeWhileHandlingException(this);
+      }
+    }
+  }
+
+  private int readMetadata(SegmentReadState state) throws IOException {
+    String metaFileName =
+        IndexFileNames.segmentFileName(
+            state.segmentInfo.name, state.segmentSuffix, Lucene95HnswVectorsFormat.META_EXTENSION);
+    int versionMeta = -1;
+    try (ChecksumIndexInput meta = state.directory.openChecksumInput(metaFileName, state.context)) {
+      Throwable priorE = null;
+      try {
+        versionMeta =
+            CodecUtil.checkIndexHeader(
+                meta,
+                Lucene95HnswVectorsFormat.META_CODEC_NAME,
+                Lucene95HnswVectorsFormat.VERSION_START,
+                Lucene95HnswVectorsFormat.VERSION_CURRENT,
+                state.segmentInfo.getId(),
+                state.segmentSuffix);
+        readFields(meta, state.fieldInfos);
+      } catch (Throwable exception) {
+        priorE = exception;
+      } finally {
+        CodecUtil.checkFooter(meta, priorE);
+      }
+    }
+    return versionMeta;
+  }
+
+  private static IndexInput openDataInput(
+      SegmentReadState state, int versionMeta, String fileExtension, String codecName)
+      throws IOException {
+    String fileName =
+        IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, fileExtension);
+    IndexInput in = state.directory.openInput(fileName, state.context);
+    boolean success = false;
+    try {
+      int versionVectorData =
+          CodecUtil.checkIndexHeader(
+              in,
+              codecName,
+              Lucene95HnswVectorsFormat.VERSION_START,
+              Lucene95HnswVectorsFormat.VERSION_CURRENT,
+              state.segmentInfo.getId(),
+              state.segmentSuffix);
+      if (versionMeta != versionVectorData) {
+        throw new CorruptIndexException(
+            "Format versions mismatch: meta="
+                + versionMeta
+                + ", "
+                + codecName
+                + "="
+                + versionVectorData,
+            in);
+      }
+      CodecUtil.retrieveChecksum(in);
+      success = true;
+      return in;
+    } finally {
+      if (success == false) {
+        IOUtils.closeWhileHandlingException(in);
+      }
+    }
+  }
+
+  private void readFields(ChecksumIndexInput meta, FieldInfos infos) throws IOException {
+    for (int fieldNumber = meta.readInt(); fieldNumber != -1; fieldNumber = meta.readInt()) {
+      FieldInfo info = infos.fieldInfo(fieldNumber);
+      if (info == null) {
+        throw new CorruptIndexException("Invalid field number: " + fieldNumber, meta);
+      }
+      FieldEntry fieldEntry = readField(meta);
+      validateFieldEntry(info, fieldEntry);
+      fields.put(info.name, fieldEntry);
+    }
+  }
+
+  private void validateFieldEntry(FieldInfo info, FieldEntry fieldEntry) {
+    int dimension = info.getVectorDimension();
+    if (dimension != fieldEntry.dimension) {
+      throw new IllegalStateException(
+          "Inconsistent vector dimension for field=\""
+              + info.name
+              + "\"; "
+              + dimension
+              + " != "
+              + fieldEntry.dimension);
+    }
+
+    int byteSize =
+        switch (info.getVectorEncoding()) {
+          case BYTE -> Byte.BYTES;
+          case FLOAT32 -> Float.BYTES;
+        };
+    long vectorBytes = Math.multiplyExact((long) dimension, byteSize);
+    long numBytes = Math.multiplyExact(vectorBytes, fieldEntry.size);
+    if (numBytes != fieldEntry.vectorDataLength) {
+      throw new IllegalStateException(
+          "Vector data length "
+              + fieldEntry.vectorDataLength
+              + " not matching size="
+              + fieldEntry.size
+              + " * dim="
+              + dimension
+              + " * byteSize="
+              + byteSize
+              + " = "
+              + numBytes);
+    }
+  }
+
+  private VectorSimilarityFunction readSimilarityFunction(DataInput input) throws IOException {
+    int similarityFunctionId = input.readInt();
+    if (similarityFunctionId < 0
+        || similarityFunctionId >= VectorSimilarityFunction.values().length) {
+      throw new CorruptIndexException(
+          "Invalid similarity function id: " + similarityFunctionId, input);
+    }
+    return VectorSimilarityFunction.values()[similarityFunctionId];
+  }
+
+  private VectorEncoding readVectorEncoding(DataInput input) throws IOException {
+    int encodingId = input.readInt();
+    if (encodingId < 0 || encodingId >= VectorEncoding.values().length) {
+      throw new CorruptIndexException("Invalid vector encoding id: " + encodingId, input);
+    }
+    return VectorEncoding.values()[encodingId];
+  }
+
+  private FieldEntry readField(IndexInput input) throws IOException {
+    VectorEncoding vectorEncoding = readVectorEncoding(input);
+    VectorSimilarityFunction similarityFunction = readSimilarityFunction(input);
+    return new FieldEntry(input, vectorEncoding, similarityFunction);
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    long totalBytes = RamUsageEstimator.shallowSizeOfInstance(Lucene95HnswVectorsFormat.class);
+    totalBytes +=
+        RamUsageEstimator.sizeOfMap(
+            fields, RamUsageEstimator.shallowSizeOfInstance(FieldEntry.class));
+    return totalBytes;
+  }
+
+  @Override
+  public void checkIntegrity() throws IOException {
+    CodecUtil.checksumEntireFile(vectorData);
+    CodecUtil.checksumEntireFile(vectorIndex);
+  }
+
+  @Override
+  public VectorValues getVectorValues(String field) throws IOException {
+    FieldEntry fieldEntry = fields.get(field);
+    VectorValues values = OffHeapVectorValues.load(fieldEntry, vectorData);
+    if (fieldEntry.vectorEncoding == VectorEncoding.BYTE) {
+      return new ExpandingVectorValues(values);
+    } else {
+      return values;
+    }
+  }
+
+  @Override
+  public TopDocs search(String field, float[] target, int k, Bits acceptDocs, int visitedLimit)
+      throws IOException {
+    FieldEntry fieldEntry = fields.get(field);
+
+    if (fieldEntry.size() == 0) {
+      return new TopDocs(new TotalHits(0, TotalHits.Relation.EQUAL_TO), new ScoreDoc[0]);
+    }
+
+    // bound k by total number of vectors to prevent oversizing data structures
+    k = Math.min(k, fieldEntry.size());
+    OffHeapVectorValues vectorValues = OffHeapVectorValues.load(fieldEntry, vectorData);
+
+    NeighborQueue results =
+        HnswGraphSearcher.search(
+            target,
+            k,
+            vectorValues,
+            fieldEntry.vectorEncoding,
+            fieldEntry.similarityFunction,
+            getGraph(fieldEntry),
+            vectorValues.getAcceptOrds(acceptDocs),
+            visitedLimit);
+
+    int i = 0;
+    ScoreDoc[] scoreDocs = new ScoreDoc[Math.min(results.size(), k)];
+    while (results.size() > 0) {
+      int node = results.topNode();
+      float score = results.topScore();
+      results.pop();
+      scoreDocs[scoreDocs.length - ++i] = new ScoreDoc(vectorValues.ordToDoc(node), score);
+    }
+
+    TotalHits.Relation relation =
+        results.incomplete()
+            ? TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO
+            : TotalHits.Relation.EQUAL_TO;
+    return new TopDocs(new TotalHits(results.visitedCount(), relation), scoreDocs);
+  }
+
+  /** Get knn graph values; used for testing */
+  public HnswGraph getGraph(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.vectorIndexLength > 0) {
+      return getGraph(entry);
+    } else {
+      return HnswGraph.EMPTY;
+    }
+  }
+
+  private HnswGraph getGraph(FieldEntry entry) throws IOException {
+    IndexInput bytesSlice =
+        vectorIndex.slice("graph-data", entry.vectorIndexOffset, entry.vectorIndexLength);
+    return new OffHeapHnswGraph(entry, bytesSlice);
+  }
+
+  @Override
+  public void close() throws IOException {
+    IOUtils.close(vectorData, vectorIndex);
+  }
+
+  static class FieldEntry {
+
+    final VectorSimilarityFunction similarityFunction;
+    final VectorEncoding vectorEncoding;
+    final long vectorDataOffset;
+    final long vectorDataLength;
+    final long vectorIndexOffset;
+    final long vectorIndexLength;
+    final int M;
+    final int numLevels;
+    final int dimension;
+    final int size;
+    final int[][] nodesByLevel;
+    // for each level the start offsets in vectorIndex file from where to read neighbours
+    final long[] graphOffsetsByLevel;
+    // the packed ints version used to store the graph connections
+    final int packedIntsVersion;

Review Comment:
   we no longer need it, do we?



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/Lucene95HnswVectorsReader.java:
##########
@@ -0,0 +1,497 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.codecs.lucene95;
+
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.KnnVectorsReader;
+import org.apache.lucene.index.*;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TotalHits;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.RandomAccessInput;
+import org.apache.lucene.util.Bits;
+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.HnswGraphSearcher;
+import org.apache.lucene.util.hnsw.NeighborQueue;
+import org.apache.lucene.util.packed.DirectMonotonicReader;
+
+/**
+ * Reads vectors from the index segments along with index data structures supporting KNN search.
+ *
+ * @lucene.experimental
+ */
+public final class Lucene95HnswVectorsReader extends KnnVectorsReader {
+
+  private final FieldInfos fieldInfos;
+  private final Map<String, FieldEntry> fields = new HashMap<>();
+  private final IndexInput vectorData;
+  private final IndexInput vectorIndex;
+
+  Lucene95HnswVectorsReader(SegmentReadState state) throws IOException {
+    this.fieldInfos = state.fieldInfos;
+    int versionMeta = readMetadata(state);
+    boolean success = false;
+    try {
+      vectorData =
+          openDataInput(
+              state,
+              versionMeta,
+              Lucene95HnswVectorsFormat.VECTOR_DATA_EXTENSION,
+              Lucene95HnswVectorsFormat.VECTOR_DATA_CODEC_NAME);
+      vectorIndex =
+          openDataInput(
+              state,
+              versionMeta,
+              Lucene95HnswVectorsFormat.VECTOR_INDEX_EXTENSION,
+              Lucene95HnswVectorsFormat.VECTOR_INDEX_CODEC_NAME);
+      success = true;
+    } finally {
+      if (success == false) {
+        IOUtils.closeWhileHandlingException(this);
+      }
+    }
+  }
+
+  private int readMetadata(SegmentReadState state) throws IOException {
+    String metaFileName =
+        IndexFileNames.segmentFileName(
+            state.segmentInfo.name, state.segmentSuffix, Lucene95HnswVectorsFormat.META_EXTENSION);
+    int versionMeta = -1;
+    try (ChecksumIndexInput meta = state.directory.openChecksumInput(metaFileName, state.context)) {
+      Throwable priorE = null;
+      try {
+        versionMeta =
+            CodecUtil.checkIndexHeader(
+                meta,
+                Lucene95HnswVectorsFormat.META_CODEC_NAME,
+                Lucene95HnswVectorsFormat.VERSION_START,
+                Lucene95HnswVectorsFormat.VERSION_CURRENT,
+                state.segmentInfo.getId(),
+                state.segmentSuffix);
+        readFields(meta, state.fieldInfos);
+      } catch (Throwable exception) {
+        priorE = exception;
+      } finally {
+        CodecUtil.checkFooter(meta, priorE);
+      }
+    }
+    return versionMeta;
+  }
+
+  private static IndexInput openDataInput(
+      SegmentReadState state, int versionMeta, String fileExtension, String codecName)
+      throws IOException {
+    String fileName =
+        IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, fileExtension);
+    IndexInput in = state.directory.openInput(fileName, state.context);
+    boolean success = false;
+    try {
+      int versionVectorData =
+          CodecUtil.checkIndexHeader(
+              in,
+              codecName,
+              Lucene95HnswVectorsFormat.VERSION_START,
+              Lucene95HnswVectorsFormat.VERSION_CURRENT,
+              state.segmentInfo.getId(),
+              state.segmentSuffix);
+      if (versionMeta != versionVectorData) {
+        throw new CorruptIndexException(
+            "Format versions mismatch: meta="
+                + versionMeta
+                + ", "
+                + codecName
+                + "="
+                + versionVectorData,
+            in);
+      }
+      CodecUtil.retrieveChecksum(in);
+      success = true;
+      return in;
+    } finally {
+      if (success == false) {
+        IOUtils.closeWhileHandlingException(in);
+      }
+    }
+  }
+
+  private void readFields(ChecksumIndexInput meta, FieldInfos infos) throws IOException {
+    for (int fieldNumber = meta.readInt(); fieldNumber != -1; fieldNumber = meta.readInt()) {
+      FieldInfo info = infos.fieldInfo(fieldNumber);
+      if (info == null) {
+        throw new CorruptIndexException("Invalid field number: " + fieldNumber, meta);
+      }
+      FieldEntry fieldEntry = readField(meta);
+      validateFieldEntry(info, fieldEntry);
+      fields.put(info.name, fieldEntry);
+    }
+  }
+
+  private void validateFieldEntry(FieldInfo info, FieldEntry fieldEntry) {
+    int dimension = info.getVectorDimension();
+    if (dimension != fieldEntry.dimension) {
+      throw new IllegalStateException(
+          "Inconsistent vector dimension for field=\""
+              + info.name
+              + "\"; "
+              + dimension
+              + " != "
+              + fieldEntry.dimension);
+    }
+
+    int byteSize =
+        switch (info.getVectorEncoding()) {
+          case BYTE -> Byte.BYTES;
+          case FLOAT32 -> Float.BYTES;
+        };
+    long vectorBytes = Math.multiplyExact((long) dimension, byteSize);
+    long numBytes = Math.multiplyExact(vectorBytes, fieldEntry.size);
+    if (numBytes != fieldEntry.vectorDataLength) {
+      throw new IllegalStateException(
+          "Vector data length "
+              + fieldEntry.vectorDataLength
+              + " not matching size="
+              + fieldEntry.size
+              + " * dim="
+              + dimension
+              + " * byteSize="
+              + byteSize
+              + " = "
+              + numBytes);
+    }
+  }
+
+  private VectorSimilarityFunction readSimilarityFunction(DataInput input) throws IOException {
+    int similarityFunctionId = input.readInt();
+    if (similarityFunctionId < 0
+        || similarityFunctionId >= VectorSimilarityFunction.values().length) {
+      throw new CorruptIndexException(
+          "Invalid similarity function id: " + similarityFunctionId, input);
+    }
+    return VectorSimilarityFunction.values()[similarityFunctionId];
+  }
+
+  private VectorEncoding readVectorEncoding(DataInput input) throws IOException {
+    int encodingId = input.readInt();
+    if (encodingId < 0 || encodingId >= VectorEncoding.values().length) {
+      throw new CorruptIndexException("Invalid vector encoding id: " + encodingId, input);
+    }
+    return VectorEncoding.values()[encodingId];
+  }
+
+  private FieldEntry readField(IndexInput input) throws IOException {
+    VectorEncoding vectorEncoding = readVectorEncoding(input);
+    VectorSimilarityFunction similarityFunction = readSimilarityFunction(input);
+    return new FieldEntry(input, vectorEncoding, similarityFunction);
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    long totalBytes = RamUsageEstimator.shallowSizeOfInstance(Lucene95HnswVectorsFormat.class);
+    totalBytes +=
+        RamUsageEstimator.sizeOfMap(
+            fields, RamUsageEstimator.shallowSizeOfInstance(FieldEntry.class));
+    return totalBytes;
+  }
+
+  @Override
+  public void checkIntegrity() throws IOException {
+    CodecUtil.checksumEntireFile(vectorData);
+    CodecUtil.checksumEntireFile(vectorIndex);
+  }
+
+  @Override
+  public VectorValues getVectorValues(String field) throws IOException {
+    FieldEntry fieldEntry = fields.get(field);
+    VectorValues values = OffHeapVectorValues.load(fieldEntry, vectorData);
+    if (fieldEntry.vectorEncoding == VectorEncoding.BYTE) {
+      return new ExpandingVectorValues(values);
+    } else {
+      return values;
+    }
+  }
+
+  @Override
+  public TopDocs search(String field, float[] target, int k, Bits acceptDocs, int visitedLimit)
+      throws IOException {
+    FieldEntry fieldEntry = fields.get(field);
+
+    if (fieldEntry.size() == 0) {
+      return new TopDocs(new TotalHits(0, TotalHits.Relation.EQUAL_TO), new ScoreDoc[0]);
+    }
+
+    // bound k by total number of vectors to prevent oversizing data structures
+    k = Math.min(k, fieldEntry.size());
+    OffHeapVectorValues vectorValues = OffHeapVectorValues.load(fieldEntry, vectorData);
+
+    NeighborQueue results =
+        HnswGraphSearcher.search(
+            target,
+            k,
+            vectorValues,
+            fieldEntry.vectorEncoding,
+            fieldEntry.similarityFunction,
+            getGraph(fieldEntry),
+            vectorValues.getAcceptOrds(acceptDocs),
+            visitedLimit);
+
+    int i = 0;
+    ScoreDoc[] scoreDocs = new ScoreDoc[Math.min(results.size(), k)];
+    while (results.size() > 0) {
+      int node = results.topNode();
+      float score = results.topScore();
+      results.pop();
+      scoreDocs[scoreDocs.length - ++i] = new ScoreDoc(vectorValues.ordToDoc(node), score);
+    }
+
+    TotalHits.Relation relation =
+        results.incomplete()
+            ? TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO
+            : TotalHits.Relation.EQUAL_TO;
+    return new TopDocs(new TotalHits(results.visitedCount(), relation), scoreDocs);
+  }
+
+  /** Get knn graph values; used for testing */
+  public HnswGraph getGraph(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.vectorIndexLength > 0) {
+      return getGraph(entry);
+    } else {
+      return HnswGraph.EMPTY;
+    }
+  }
+
+  private HnswGraph getGraph(FieldEntry entry) throws IOException {
+    return new OffHeapHnswGraph(entry, vectorIndex);
+  }
+
+  @Override
+  public void close() throws IOException {
+    IOUtils.close(vectorData, vectorIndex);
+  }
+
+  static class FieldEntry {
+
+    final VectorSimilarityFunction similarityFunction;
+    final VectorEncoding vectorEncoding;
+    final long vectorDataOffset;
+    final long vectorDataLength;
+    final long vectorIndexOffset;
+    final long vectorIndexLength;
+    final int M;
+    final int numLevels;
+    final int dimension;
+    final int size;
+    final int[][] nodesByLevel;
+    // for each level the start offsets in vectorIndex file from where to read neighbours
+    final DirectMonotonicReader.Meta offsetsMeta;
+    final long offsetsOffset;
+    final int offsetsBlockShift;
+    final long offsetsLength;
+
+    // the following four variables used to read docIds encoded by IndexDISI
+    // special values of docsWithFieldOffset are -1 and -2
+    // -1 : dense
+    // -2 : empty
+    // other: sparse
+    final long docsWithFieldOffset;
+    final long docsWithFieldLength;
+    final short jumpTableEntryCount;
+    final byte denseRankPower;
+
+    // the following four variables used to read ordToDoc encoded by DirectMonotonicWriter
+    // note that only spare case needs to store ordToDoc
+    final long addressesOffset;
+    final int blockShift;
+    final DirectMonotonicReader.Meta meta;
+    final long addressesLength;
+
+    FieldEntry(
+        IndexInput input,
+        VectorEncoding vectorEncoding,
+        VectorSimilarityFunction similarityFunction)
+        throws IOException {
+      this.similarityFunction = similarityFunction;
+      this.vectorEncoding = vectorEncoding;
+      vectorDataOffset = input.readVLong();
+      vectorDataLength = input.readVLong();
+      vectorIndexOffset = input.readVLong();
+      vectorIndexLength = input.readVLong();
+      dimension = input.readVInt();
+      size = input.readInt();
+
+      docsWithFieldOffset = input.readLong();
+      docsWithFieldLength = input.readLong();
+      jumpTableEntryCount = input.readShort();
+      denseRankPower = input.readByte();
+
+      // dense or empty
+      if (docsWithFieldOffset == -1 || docsWithFieldOffset == -2) {
+        addressesOffset = 0;
+        blockShift = 0;
+        meta = null;
+        addressesLength = 0;
+      } else {
+        // sparse
+        addressesOffset = input.readLong();
+        blockShift = input.readVInt();
+        meta = DirectMonotonicReader.loadMeta(input, size, blockShift);
+        addressesLength = input.readLong();
+      }
+
+      // read nodes by level
+      M = input.readVInt();
+      numLevels = input.readVInt();
+      nodesByLevel = new int[numLevels][];
+      long numberOfOffsets = 0;
+      for (int level = 0; level < numLevels; level++) {
+        if (level > 0) {
+          int numNodesOnLevel = input.readVInt();
+          numberOfOffsets += numNodesOnLevel;
+          nodesByLevel[level] = new int[numNodesOnLevel];
+          for (int i = 0; i < numNodesOnLevel; i++) {
+            nodesByLevel[level][i] = input.readInt();
+          }
+        } else {
+          numberOfOffsets += size;
+        }
+      }
+      if (numberOfOffsets > 0) {
+        offsetsOffset = input.readLong();
+        offsetsBlockShift = input.readVInt();
+        offsetsMeta = DirectMonotonicReader.loadMeta(input, numberOfOffsets, offsetsBlockShift);
+        offsetsLength = input.readLong();
+      } else {
+        offsetsOffset = 0;
+        offsetsBlockShift = 0;
+        offsetsMeta = null;
+        offsetsLength = 0;
+      }
+    }
+
+    int size() {
+      return size;
+    }
+  }
+
+  /** Read the nearest-neighbors graph from the index input */
+  private static final class OffHeapHnswGraph extends HnswGraph {
+
+    final IndexInput dataIn;
+    final int[][] nodesByLevel;
+    final int numLevels;
+    final int entryNode;
+    final int size;
+    int arcCount;
+    int arcUpTo;
+    int arc;
+    private final DirectMonotonicReader graphLevelNodeOffsets;
+    private final long[] graphLevelNodeIndexOffsets;
+    // Allocated to be M*2 to track the current neighbors being explored
+    private final int[] currentNeighborsBuffer;
+
+    OffHeapHnswGraph(FieldEntry entry, IndexInput vectorIndex) throws IOException {
+      this.dataIn =
+          vectorIndex.slice("graph-data", entry.vectorIndexOffset, entry.vectorIndexLength);
+      this.nodesByLevel = entry.nodesByLevel;
+      this.numLevels = entry.numLevels;
+      this.entryNode = numLevels > 1 ? nodesByLevel[numLevels - 1][0] : 0;
+      this.size = entry.size();
+      final RandomAccessInput addressesData =
+          vectorIndex.randomAccessSlice(entry.offsetsOffset, entry.offsetsLength);
+      this.graphLevelNodeOffsets =
+          DirectMonotonicReader.getInstance(entry.offsetsMeta, addressesData);
+      this.currentNeighborsBuffer = new int[entry.M * 2];
+      graphLevelNodeIndexOffsets = new long[numLevels];
+      graphLevelNodeIndexOffsets[0] = 0;
+      for (int i = 1; i < numLevels; i++) {
+        // nodesByLevel is `null` for the zeroth level as we know its the size

Review Comment:
   ```suggestion
           // nodesByLevel is `null` for the zeroth level as we know its size
   ```



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/Lucene95HnswVectorsWriter.java:
##########
@@ -0,0 +1,751 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.codecs.lucene95;
+
+import static org.apache.lucene.codecs.lucene95.Lucene95HnswVectorsFormat.DIRECT_MONOTONIC_BLOCK_SHIFT;
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.KnnFieldVectorsWriter;
+import org.apache.lucene.codecs.KnnVectorsWriter;
+import org.apache.lucene.codecs.lucene90.IndexedDISI;
+import org.apache.lucene.index.*;
+import org.apache.lucene.index.Sorter;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.*;
+import org.apache.lucene.util.hnsw.HnswGraph;
+import org.apache.lucene.util.hnsw.HnswGraph.NodesIterator;
+import org.apache.lucene.util.hnsw.HnswGraphBuilder;
+import org.apache.lucene.util.hnsw.NeighborArray;
+import org.apache.lucene.util.hnsw.OnHeapHnswGraph;
+import org.apache.lucene.util.packed.DirectMonotonicWriter;
+
+/**
+ * Writes vector values and knn graphs to index segments.
+ *
+ * @lucene.experimental
+ */
+public final class Lucene95HnswVectorsWriter extends KnnVectorsWriter {
+
+  private final SegmentWriteState segmentWriteState;
+  private final IndexOutput meta, vectorData, vectorIndex;
+  private final int M;
+  private final int beamWidth;
+
+  private final List<FieldWriter<?>> fields = new ArrayList<>();
+  private boolean finished;
+
+  Lucene95HnswVectorsWriter(SegmentWriteState state, int M, int beamWidth) throws IOException {
+    this.M = M;
+    this.beamWidth = beamWidth;
+    segmentWriteState = state;
+    String metaFileName =
+        IndexFileNames.segmentFileName(
+            state.segmentInfo.name, state.segmentSuffix, Lucene95HnswVectorsFormat.META_EXTENSION);
+
+    String vectorDataFileName =
+        IndexFileNames.segmentFileName(
+            state.segmentInfo.name,
+            state.segmentSuffix,
+            Lucene95HnswVectorsFormat.VECTOR_DATA_EXTENSION);
+
+    String indexDataFileName =
+        IndexFileNames.segmentFileName(
+            state.segmentInfo.name,
+            state.segmentSuffix,
+            Lucene95HnswVectorsFormat.VECTOR_INDEX_EXTENSION);
+    boolean success = false;
+    try {
+      meta = state.directory.createOutput(metaFileName, state.context);
+      vectorData = state.directory.createOutput(vectorDataFileName, state.context);
+      vectorIndex = state.directory.createOutput(indexDataFileName, state.context);
+
+      CodecUtil.writeIndexHeader(
+          meta,
+          Lucene95HnswVectorsFormat.META_CODEC_NAME,
+          Lucene95HnswVectorsFormat.VERSION_CURRENT,
+          state.segmentInfo.getId(),
+          state.segmentSuffix);
+      CodecUtil.writeIndexHeader(
+          vectorData,
+          Lucene95HnswVectorsFormat.VECTOR_DATA_CODEC_NAME,
+          Lucene95HnswVectorsFormat.VERSION_CURRENT,
+          state.segmentInfo.getId(),
+          state.segmentSuffix);
+      CodecUtil.writeIndexHeader(
+          vectorIndex,
+          Lucene95HnswVectorsFormat.VECTOR_INDEX_CODEC_NAME,
+          Lucene95HnswVectorsFormat.VERSION_CURRENT,
+          state.segmentInfo.getId(),
+          state.segmentSuffix);
+      success = true;
+    } finally {
+      if (success == false) {
+        IOUtils.closeWhileHandlingException(this);
+      }
+    }
+  }
+
+  @Override
+  public KnnFieldVectorsWriter<?> addField(FieldInfo fieldInfo) throws IOException {
+    FieldWriter<?> newField =
+        FieldWriter.create(fieldInfo, M, beamWidth, segmentWriteState.infoStream);
+    fields.add(newField);
+    return newField;
+  }
+
+  @Override
+  public void flush(int maxDoc, Sorter.DocMap sortMap) throws IOException {
+    for (FieldWriter<?> field : fields) {
+      if (sortMap == null) {
+        writeField(field, maxDoc);
+      } else {
+        writeSortingField(field, maxDoc, sortMap);
+      }
+    }
+  }
+
+  @Override
+  public void finish() throws IOException {
+    if (finished) {
+      throw new IllegalStateException("already finished");
+    }
+    finished = true;
+
+    if (meta != null) {
+      // write end of fields marker
+      meta.writeInt(-1);
+      CodecUtil.writeFooter(meta);
+    }
+    if (vectorData != null) {
+      CodecUtil.writeFooter(vectorData);
+      CodecUtil.writeFooter(vectorIndex);
+    }
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    long total = 0;
+    for (FieldWriter<?> field : fields) {
+      total += field.ramBytesUsed();
+    }
+    return total;
+  }
+
+  private void writeField(FieldWriter<?> fieldData, int maxDoc) throws IOException {
+    // write vector values
+    long vectorDataOffset = vectorData.alignFilePointer(Float.BYTES);
+    switch (fieldData.fieldInfo.getVectorEncoding()) {
+      case BYTE -> writeByteVectors(fieldData);
+      case FLOAT32 -> writeFloat32Vectors(fieldData);
+    }
+    long vectorDataLength = vectorData.getFilePointer() - vectorDataOffset;
+
+    // write graph
+    long vectorIndexOffset = vectorIndex.getFilePointer();
+    OnHeapHnswGraph graph = fieldData.getGraph();
+    int[][] graphLevelNodeOffsets = writeGraph(graph);
+    long vectorIndexLength = vectorIndex.getFilePointer() - vectorIndexOffset;
+
+    writeMeta(
+        fieldData.fieldInfo,
+        maxDoc,
+        vectorDataOffset,
+        vectorDataLength,
+        vectorIndexOffset,
+        vectorIndexLength,
+        fieldData.docsWithField,
+        graph,
+        graphLevelNodeOffsets);
+  }
+
+  private void writeFloat32Vectors(FieldWriter<?> fieldData) throws IOException {
+    final ByteBuffer buffer =
+        ByteBuffer.allocate(fieldData.dim * Float.BYTES).order(ByteOrder.LITTLE_ENDIAN);
+    final BytesRef binaryValue = new BytesRef(buffer.array());
+    for (Object v : fieldData.vectors) {
+      buffer.asFloatBuffer().put((float[]) v);
+      vectorData.writeBytes(binaryValue.bytes, binaryValue.offset, binaryValue.length);
+    }
+  }
+
+  private void writeByteVectors(FieldWriter<?> fieldData) throws IOException {
+    for (Object v : fieldData.vectors) {
+      BytesRef vector = (BytesRef) v;
+      vectorData.writeBytes(vector.bytes, vector.offset, vector.length);
+    }
+  }
+
+  private void writeSortingField(FieldWriter<?> fieldData, int maxDoc, Sorter.DocMap sortMap)
+      throws IOException {
+    final int[] docIdOffsets = new int[sortMap.size()];
+    int offset = 1; // 0 means no vector for this (field, document)
+    DocIdSetIterator iterator = fieldData.docsWithField.iterator();
+    for (int docID = iterator.nextDoc();
+        docID != DocIdSetIterator.NO_MORE_DOCS;
+        docID = iterator.nextDoc()) {
+      int newDocID = sortMap.oldToNew(docID);
+      docIdOffsets[newDocID] = offset++;
+    }
+    DocsWithFieldSet newDocsWithField = new DocsWithFieldSet();
+    final int[] ordMap = new int[offset - 1]; // new ord to old ord
+    final int[] oldOrdMap = new int[offset - 1]; // old ord to new ord
+    int ord = 0;
+    int doc = 0;
+    for (int docIdOffset : docIdOffsets) {
+      if (docIdOffset != 0) {
+        ordMap[ord] = docIdOffset - 1;
+        oldOrdMap[docIdOffset - 1] = ord;
+        newDocsWithField.add(doc);
+        ord++;
+      }
+      doc++;
+    }
+
+    // write vector values
+    long vectorDataOffset =
+        switch (fieldData.fieldInfo.getVectorEncoding()) {
+          case BYTE -> writeSortedByteVectors(fieldData, ordMap);
+          case FLOAT32 -> writeSortedFloat32Vectors(fieldData, ordMap);
+        };
+    long vectorDataLength = vectorData.getFilePointer() - vectorDataOffset;
+
+    // write graph
+    long vectorIndexOffset = vectorIndex.getFilePointer();
+    OnHeapHnswGraph graph = fieldData.getGraph();
+    int[][] graphLevelNodeOffsets = graph == null ? new int[0][] : new int[graph.numLevels()][];
+    HnswGraph mockGraph = reconstructAndWriteGraph(graph, ordMap, oldOrdMap, graphLevelNodeOffsets);
+    long vectorIndexLength = vectorIndex.getFilePointer() - vectorIndexOffset;
+
+    writeMeta(
+        fieldData.fieldInfo,
+        maxDoc,
+        vectorDataOffset,
+        vectorDataLength,
+        vectorIndexOffset,
+        vectorIndexLength,
+        newDocsWithField,
+        mockGraph,
+        graphLevelNodeOffsets);
+  }
+
+  private long writeSortedFloat32Vectors(FieldWriter<?> fieldData, int[] ordMap)
+      throws IOException {
+    long vectorDataOffset = vectorData.alignFilePointer(Float.BYTES);
+    final ByteBuffer buffer =
+        ByteBuffer.allocate(fieldData.dim * Float.BYTES).order(ByteOrder.LITTLE_ENDIAN);
+    final BytesRef binaryValue = new BytesRef(buffer.array());
+    for (int ordinal : ordMap) {
+      float[] vector = (float[]) fieldData.vectors.get(ordinal);
+      buffer.asFloatBuffer().put(vector);
+      vectorData.writeBytes(binaryValue.bytes, binaryValue.offset, binaryValue.length);
+    }
+    return vectorDataOffset;
+  }
+
+  private long writeSortedByteVectors(FieldWriter<?> fieldData, int[] ordMap) throws IOException {
+    long vectorDataOffset = vectorData.alignFilePointer(Float.BYTES);
+    for (int ordinal : ordMap) {
+      BytesRef vector = (BytesRef) fieldData.vectors.get(ordinal);
+      vectorData.writeBytes(vector.bytes, vector.offset, vector.length);
+    }
+    return vectorDataOffset;
+  }
+
+  /**
+   * Reconstructs the graph given the old and new node ids.
+   *
+   * <p>Additionally, the graph node connections are written to the vectorIndex.
+   *
+   * @param graph The current on heap graph
+   * @param newToOldMap the new node ids indexed to the old node ids
+   * @param oldToNewMap the old node ids indexed to the new node ids
+   * @param levelNodeOffsets where to place the new offsets for the nodes in the vector index.
+   * @return The graph
+   * @throws IOException if writing to vectorIndex fails
+   */
+  private HnswGraph reconstructAndWriteGraph(
+      OnHeapHnswGraph graph, int[] newToOldMap, int[] oldToNewMap, int[][] levelNodeOffsets)
+      throws IOException {
+    if (graph == null) return null;
+
+    List<int[]> nodesByLevel = new ArrayList<>(graph.numLevels());
+    nodesByLevel.add(null);
+
+    int maxOrd = graph.size();
+    int maxConnOnLevel = M * 2;
+    NodesIterator nodesOnLevel0 = graph.getNodesOnLevel(0);
+    levelNodeOffsets[0] = new int[nodesOnLevel0.size()];
+    while (nodesOnLevel0.hasNext()) {
+      int node = nodesOnLevel0.nextInt();
+      NeighborArray neighbors = graph.getNeighbors(0, newToOldMap[node]);
+      long offset = vectorIndex.getFilePointer();
+      reconstructAndWriteNeigbours(neighbors, oldToNewMap, maxConnOnLevel, maxOrd);
+      levelNodeOffsets[0][node] = Math.toIntExact(vectorIndex.getFilePointer() - offset);
+    }
+
+    maxConnOnLevel = M;
+    for (int level = 1; level < graph.numLevels(); level++) {
+      NodesIterator nodesOnLevel = graph.getNodesOnLevel(level);
+      int[] newNodes = new int[nodesOnLevel.size()];
+      int n = 0;
+      while (nodesOnLevel.hasNext()) {
+        newNodes[n++] = oldToNewMap[nodesOnLevel.nextInt()];
+      }
+      Arrays.sort(newNodes);
+      nodesByLevel.add(newNodes);
+      levelNodeOffsets[level] = new int[newNodes.length];
+      int nodeOffsetIndex = 0;
+      for (int node : newNodes) {
+        NeighborArray neighbors = graph.getNeighbors(level, newToOldMap[node]);
+        long offset = vectorIndex.getFilePointer();
+        reconstructAndWriteNeigbours(neighbors, oldToNewMap, maxConnOnLevel, maxOrd);
+        levelNodeOffsets[level][nodeOffsetIndex++] =
+            Math.toIntExact(vectorIndex.getFilePointer() - offset);
+      }
+    }
+    return new HnswGraph() {
+      @Override
+      public int nextNeighbor() {
+        throw new UnsupportedOperationException("Not supported on a mock graph");
+      }
+
+      @Override
+      public void seek(int level, int target) {
+        throw new UnsupportedOperationException("Not supported on a mock graph");
+      }
+
+      @Override
+      public int size() {
+        return graph.size();
+      }
+
+      @Override
+      public int numLevels() {
+        return graph.numLevels();
+      }
+
+      @Override
+      public int entryNode() {
+        throw new UnsupportedOperationException("Not supported on a mock graph");
+      }
+
+      @Override
+      public NodesIterator getNodesOnLevel(int level) {
+        if (level == 0) {
+          return graph.getNodesOnLevel(0);
+        } else {
+          return new NodesIterator(nodesByLevel.get(level), nodesByLevel.get(level).length);
+        }
+      }
+    };
+  }
+
+  private void reconstructAndWriteNeigbours(
+      NeighborArray neighbors, int[] oldToNewMap, int maxConnOnLevel, int maxOrd)
+      throws IOException {
+    int size = neighbors.size();
+    vectorIndex.writeVInt(size);
+
+    // Destructively modify; it's ok we are discarding it after this
+    int[] nnodes = neighbors.node();
+    for (int i = 0; i < size; i++) {
+      nnodes[i] = oldToNewMap[nnodes[i]];
+    }
+    Arrays.sort(nnodes, 0, size);
+    // Now that we have sorted, do delta encoding to minimize the required bits to store the
+    // information
+    for (int i = size - 1; i > 0; --i) {
+      assert nnodes[i] < maxOrd : "node too large: " + nnodes[i] + ">=" + maxOrd;
+      nnodes[i] -= nnodes[i - 1];
+    }
+    for (int i = 0; i < size; i++) {
+      vectorIndex.writeVInt(nnodes[i]);
+    }
+  }
+
+  @Override
+  public void mergeOneField(FieldInfo fieldInfo, MergeState mergeState) throws IOException {
+    long vectorDataOffset = vectorData.alignFilePointer(Float.BYTES);
+    VectorValues vectors = MergedVectorValues.mergeVectorValues(fieldInfo, mergeState);
+
+    IndexOutput tempVectorData =
+        segmentWriteState.directory.createTempOutput(
+            vectorData.getName(), "temp", segmentWriteState.context);
+    IndexInput vectorDataInput = null;
+    boolean success = false;
+    try {
+      // write the vector data to a temporary file
+      DocsWithFieldSet docsWithField =
+          writeVectorData(tempVectorData, vectors, fieldInfo.getVectorEncoding().byteSize);
+      CodecUtil.writeFooter(tempVectorData);
+      IOUtils.close(tempVectorData);
+
+      // copy the temporary file vectors to the actual data file
+      vectorDataInput =
+          segmentWriteState.directory.openInput(
+              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
+      // we use Lucene95HnswVectorsReader.DenseOffHeapVectorValues for the graph construction
+      // doesn't need to know docIds
+      // TODO: separate random access vector values from DocIdSetIterator?
+      int byteSize = vectors.dimension() * fieldInfo.getVectorEncoding().byteSize;
+      OffHeapVectorValues offHeapVectors =
+          new OffHeapVectorValues.DenseOffHeapVectorValues(
+              vectors.dimension(), docsWithField.cardinality(), vectorDataInput, byteSize);
+      OnHeapHnswGraph graph = null;
+      int[][] vectorIndexNodeOffsets = null;
+      if (offHeapVectors.size() != 0) {
+        // build graph
+        HnswGraphBuilder<?> hnswGraphBuilder =
+            HnswGraphBuilder.create(
+                offHeapVectors,
+                fieldInfo.getVectorEncoding(),
+                fieldInfo.getVectorSimilarityFunction(),
+                M,
+                beamWidth,
+                HnswGraphBuilder.randSeed);
+        hnswGraphBuilder.setInfoStream(segmentWriteState.infoStream);
+        graph = hnswGraphBuilder.build(offHeapVectors.copy());
+        vectorIndexNodeOffsets = writeGraph(graph);
+      }
+      long vectorIndexLength = vectorIndex.getFilePointer() - vectorIndexOffset;
+      writeMeta(
+          fieldInfo,
+          segmentWriteState.segmentInfo.maxDoc(),
+          vectorDataOffset,
+          vectorDataLength,
+          vectorIndexOffset,
+          vectorIndexLength,
+          docsWithField,
+          graph,
+          vectorIndexNodeOffsets);
+      success = true;
+    } finally {
+      IOUtils.close(vectorDataInput);
+      if (success) {
+        segmentWriteState.directory.deleteFile(tempVectorData.getName());
+      } else {
+        IOUtils.closeWhileHandlingException(tempVectorData);
+        IOUtils.deleteFilesIgnoringExceptions(
+            segmentWriteState.directory, tempVectorData.getName());
+      }
+    }
+  }
+
+  /**
+   * @param graph Write the graph in a compressed format
+   * @return The non-cumulative offsets for the nodes. Should be used to create cumulative offsets.
+   * @throws IOException if writing to vectorIndex fails
+   */
+  private int[][] writeGraph(OnHeapHnswGraph graph) throws IOException {
+    if (graph == null) return new int[0][0];
+    // write vectors' neighbours on each level into the vectorIndex file
+    int countOnLevel0 = graph.size();
+    int[][] offsets = new int[graph.numLevels()][];
+    for (int level = 0; level < graph.numLevels(); level++) {
+      NodesIterator nodesOnLevel = graph.getNodesOnLevel(level);
+      offsets[level] = new int[nodesOnLevel.size()];
+      int nodeOffsetId = 0;
+      while (nodesOnLevel.hasNext()) {
+        int node = nodesOnLevel.nextInt();
+        NeighborArray neighbors = graph.getNeighbors(level, node);
+        int size = neighbors.size();
+        // Write size in VInt as the neighbors list is typically small
+        long offsetStart = vectorIndex.getFilePointer();
+        vectorIndex.writeVInt(size);
+        // Destructively modify; it's ok we are discarding it after this
+        int[] nnodes = neighbors.node();
+        Arrays.sort(nnodes, 0, size);
+        // Now that we have sorted, do delta encoding to minimize the required bits to store the
+        // information
+        for (int i = size - 1; i > 0; --i) {
+          assert nnodes[i] < countOnLevel0 : "node too large: " + nnodes[i] + ">=" + countOnLevel0;
+          nnodes[i] -= nnodes[i - 1];
+        }
+        for (int i = 0; i < size; i++) {
+          vectorIndex.writeVInt(nnodes[i]);
+        }
+        offsets[level][nodeOffsetId++] =
+            Math.toIntExact(vectorIndex.getFilePointer() - offsetStart);
+      }
+    }
+    return offsets;
+  }
+
+  private void writeMeta(
+      FieldInfo field,
+      int maxDoc,
+      long vectorDataOffset,
+      long vectorDataLength,
+      long vectorIndexOffset,
+      long vectorIndexLength,
+      DocsWithFieldSet docsWithField,
+      HnswGraph graph,
+      int[][] graphLevelNodeOffsets)

Review Comment:
   One way how we commonly solve this problem is by writing `graphLevelNodeOffsets` to a in-memory `ByteBuffersDataOutput` and then appending its content to the right file.



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/package-info.java:
##########
@@ -0,0 +1,423 @@
+/*
+ * 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.5 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.lucene94.Lucene94FieldInfosFormat 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.lucene95.Lucene95HnswVectorsFormat 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.lucene94.Lucene94FieldInfosFormat 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.lucene95.Lucene95HnswVectorsFormat 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.
+ *   <li>In version 9.1, vector-valued fields were modified to add a graph hierarchy.
+ *   <li>In version 9.2, docs of vector-valued fields were moved from .vem to .vec and encoded by
+ *       IndexDISI. ordToDoc mappings was added to .vem.
+ *   <li>In version 9.5, HNSW graph connections were changed to be stored with PackedInts.

Review Comment:
   maybe also mention no longer explicitly storing level 0 nodes, this feels like a great reduction to memory usage?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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


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