jimczi commented on code in PR #12582:
URL: https://github.com/apache/lucene/pull/12582#discussion_r1343896112


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsWriter.java:
##########
@@ -0,0 +1,1170 @@
+/*
+ * 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.lucene99;
+
+import static 
org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat.DIRECT_MONOTONIC_BLOCK_SHIFT;
+import static 
org.apache.lucene.codecs.lucene99.Lucene99ScalarQuantizedVectorsFormat.calculateDefaultQuantile;
+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.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.HnswGraphProvider;
+import org.apache.lucene.codecs.KnnFieldVectorsWriter;
+import org.apache.lucene.codecs.KnnVectorsReader;
+import org.apache.lucene.codecs.KnnVectorsWriter;
+import org.apache.lucene.codecs.lucene90.IndexedDISI;
+import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
+import org.apache.lucene.index.ByteVectorValues;
+import org.apache.lucene.index.DocsWithFieldSet;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FloatVectorValues;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.MergeState;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.index.Sorter;
+import org.apache.lucene.index.VectorEncoding;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.InfoStream;
+import org.apache.lucene.util.RamUsageEstimator;
+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.hnsw.RandomAccessVectorValues;
+import org.apache.lucene.util.hnsw.RandomVectorScorerSupplier;
+import org.apache.lucene.util.packed.DirectMonotonicWriter;
+
+/**
+ * Writes vector values and knn graphs to index segments.
+ *
+ * @lucene.experimental
+ */
+public final class Lucene99HnswVectorsWriter extends KnnVectorsWriter {
+
+  private final SegmentWriteState segmentWriteState;
+  private final IndexOutput meta, vectorData, quantizedVectorData, vectorIndex;
+  private final int M;
+  private final int beamWidth;
+  private final Lucene99ScalarQuantizedVectorsWriter quantizedVectorsWriter;
+
+  private final List<FieldWriter<?>> fields = new ArrayList<>();
+  private boolean finished;
+
+  Lucene99HnswVectorsWriter(
+      SegmentWriteState state,
+      int M,
+      int beamWidth,
+      Lucene99ScalarQuantizedVectorsFormat quantizedVectorsFormat)
+      throws IOException {
+    this.M = M;
+    this.beamWidth = beamWidth;
+    segmentWriteState = state;
+    String metaFileName =
+        IndexFileNames.segmentFileName(
+            state.segmentInfo.name, state.segmentSuffix, 
Lucene99HnswVectorsFormat.META_EXTENSION);
+
+    String vectorDataFileName =
+        IndexFileNames.segmentFileName(
+            state.segmentInfo.name,
+            state.segmentSuffix,
+            Lucene99HnswVectorsFormat.VECTOR_DATA_EXTENSION);
+
+    String indexDataFileName =
+        IndexFileNames.segmentFileName(
+            state.segmentInfo.name,
+            state.segmentSuffix,
+            Lucene99HnswVectorsFormat.VECTOR_INDEX_EXTENSION);
+
+    final String quantizedVectorDataFileName =
+        quantizedVectorsFormat != null
+            ? IndexFileNames.segmentFileName(
+                state.segmentInfo.name,
+                state.segmentSuffix,
+                
Lucene99ScalarQuantizedVectorsFormat.QUANTIZED_VECTOR_DATA_EXTENSION)
+            : null;
+    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,
+          Lucene99HnswVectorsFormat.META_CODEC_NAME,
+          Lucene99HnswVectorsFormat.VERSION_CURRENT,
+          state.segmentInfo.getId(),
+          state.segmentSuffix);
+      CodecUtil.writeIndexHeader(
+          vectorData,
+          Lucene99HnswVectorsFormat.VECTOR_DATA_CODEC_NAME,
+          Lucene99HnswVectorsFormat.VERSION_CURRENT,
+          state.segmentInfo.getId(),
+          state.segmentSuffix);
+      CodecUtil.writeIndexHeader(
+          vectorIndex,
+          Lucene99HnswVectorsFormat.VECTOR_INDEX_CODEC_NAME,
+          Lucene99HnswVectorsFormat.VERSION_CURRENT,
+          state.segmentInfo.getId(),
+          state.segmentSuffix);
+      if (quantizedVectorDataFileName != null) {
+        quantizedVectorData =
+            state.directory.createOutput(quantizedVectorDataFileName, 
state.context);
+        CodecUtil.writeIndexHeader(
+            quantizedVectorData,
+            
Lucene99ScalarQuantizedVectorsFormat.QUANTIZED_VECTOR_DATA_CODEC_NAME,
+            Lucene99ScalarQuantizedVectorsFormat.VERSION_CURRENT,
+            state.segmentInfo.getId(),
+            state.segmentSuffix);
+        quantizedVectorsWriter =
+            new Lucene99ScalarQuantizedVectorsWriter(
+                quantizedVectorData, quantizedVectorsFormat.quantile);
+      } else {
+        quantizedVectorData = null;
+        quantizedVectorsWriter = null;
+      }
+      success = true;
+    } finally {
+      if (success == false) {
+        IOUtils.closeWhileHandlingException(this);
+      }
+    }
+  }
+
+  @Override
+  public KnnFieldVectorsWriter<?> addField(FieldInfo fieldInfo) throws 
IOException {
+    Lucene99ScalarQuantizedVectorsWriter.QuantizationVectorWriter 
quantizedVectorFieldWriter = null;
+    // Quantization only supports FLOAT32 for now
+    if (quantizedVectorsWriter != null
+        && fieldInfo.getVectorEncoding().equals(VectorEncoding.FLOAT32)) {
+      quantizedVectorFieldWriter = quantizedVectorsWriter.addField(fieldInfo);
+    }
+    FieldWriter<?> newField =
+        FieldWriter.create(
+            fieldInfo, M, beamWidth, segmentWriteState.infoStream, 
quantizedVectorFieldWriter);
+    fields.add(newField);
+    return newField;
+  }
+
+  @Override
+  public void flush(int maxDoc, Sorter.DocMap sortMap) throws IOException {
+    for (FieldWriter<?> field : fields) {
+      long[] quantizedVectorOffsetAndLen = null;
+      if (field.quantizedWriter != null) {
+        assert quantizedVectorsWriter != null;
+        quantizedVectorOffsetAndLen =
+            quantizedVectorsWriter.flush(sortMap, field.quantizedWriter, 
field.docsWithField);
+      }
+      if (sortMap == null) {
+        writeField(field, maxDoc, quantizedVectorOffsetAndLen);
+      } else {
+        writeSortingField(field, maxDoc, sortMap, quantizedVectorOffsetAndLen);
+      }
+    }
+  }
+
+  @Override
+  public void finish() throws IOException {
+    if (finished) {
+      throw new IllegalStateException("already finished");
+    }
+    finished = true;
+    if (quantizedVectorsWriter != null) {
+      quantizedVectorsWriter.finish();
+    }
+
+    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, long[] 
quantizedVecOffsetAndLen)
+      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.isQuantized(),
+        fieldData.fieldInfo,
+        maxDoc,
+        fieldData.getConfiguredQuantile(),
+        fieldData.getMinQuantile(),
+        fieldData.getMaxQuantile(),
+        quantizedVecOffsetAndLen,
+        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);
+    for (Object v : fieldData.vectors) {
+      buffer.asFloatBuffer().put((float[]) v);
+      vectorData.writeBytes(buffer.array(), buffer.array().length);
+    }
+  }
+
+  private void writeByteVectors(FieldWriter<?> fieldData) throws IOException {
+    for (Object v : fieldData.vectors) {
+      byte[] vector = (byte[]) v;
+      vectorData.writeBytes(vector, vector.length);
+    }
+  }
+
+  private void writeSortingField(
+      FieldWriter<?> fieldData,
+      int maxDoc,
+      Sorter.DocMap sortMap,
+      long[] quantizedVectorOffsetAndLen)
+      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.isQuantized(),
+        fieldData.fieldInfo,
+        maxDoc,
+        fieldData.getConfiguredQuantile(),
+        fieldData.getMinQuantile(),
+        fieldData.getMaxQuantile(),
+        quantizedVectorOffsetAndLen,
+        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);
+    for (int ordinal : ordMap) {
+      float[] vector = (float[]) fieldData.vectors.get(ordinal);
+      buffer.asFloatBuffer().put(vector);
+      vectorData.writeBytes(buffer.array(), buffer.array().length);
+    }
+    return vectorDataOffset;
+  }
+
+  private long writeSortedByteVectors(FieldWriter<?> fieldData, int[] ordMap) 
throws IOException {
+    long vectorDataOffset = vectorData.alignFilePointer(Float.BYTES);
+    for (int ordinal : ordMap) {
+      byte[] vector = (byte[]) fieldData.vectors.get(ordinal);
+      vectorData.writeBytes(vector, 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();
+    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, maxOrd);
+      levelNodeOffsets[0][node] = Math.toIntExact(vectorIndex.getFilePointer() 
- offset);
+    }
+
+    for (int level = 1; level < graph.numLevels(); level++) {
+      NodesIterator nodesOnLevel = graph.getNodesOnLevel(level);
+      int[] newNodes = new int[nodesOnLevel.size()];
+      for (int n = 0; nodesOnLevel.hasNext(); n++) {
+        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, 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 ArrayNodesIterator(nodesByLevel.get(level), 
nodesByLevel.get(level).length);
+        }
+      }
+    };
+  }
+
+  private void reconstructAndWriteNeigbours(NeighborArray neighbors, int[] 
oldToNewMap, 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);
+    IndexOutput tempVectorData =
+        segmentWriteState.directory.createTempOutput(
+            vectorData.getName(), "temp", segmentWriteState.context);
+    IndexInput vectorDataInput = null;
+    IndexInput quantizationDataInput = null;
+    Lucene99ScalarQuantizedVectorsWriter.MergedQuantileState quantizationState 
= null;
+    boolean success = false;
+    try {
+      long[] quantizedVectorDataOffsetAndLength = null;
+      if (quantizedVectorsWriter != null) {
+        quantizationState =
+            quantizedVectorsWriter.mergeOneField(segmentWriteState, fieldInfo, 
mergeState);
+        if (quantizationState != null) {
+          quantizedVectorDataOffsetAndLength = new long[2];
+          quantizedVectorDataOffsetAndLength[0] = 
quantizedVectorData.alignFilePointer(Float.BYTES);
+          quantizationDataInput =
+              segmentWriteState.directory.openInput(
+                  quantizationState.tempVectorFileName, 
segmentWriteState.context);
+          quantizedVectorData.copyBytes(
+              quantizationDataInput, quantizationDataInput.length() - 
CodecUtil.footerLength());
+          quantizedVectorDataOffsetAndLength[1] =
+              quantizedVectorData.getFilePointer() - 
quantizedVectorDataOffsetAndLength[0];
+          CodecUtil.retrieveChecksum(quantizationDataInput);
+        }
+      }
+      // write the vector data to a temporary file
+      DocsWithFieldSet docsWithField =
+          switch (fieldInfo.getVectorEncoding()) {
+            case BYTE -> writeByteVectorData(
+                tempVectorData, 
MergedVectorValues.mergeByteVectorValues(fieldInfo, mergeState));
+            case FLOAT32 -> writeVectorData(
+                tempVectorData, 
MergedVectorValues.mergeFloatVectorValues(fieldInfo, mergeState));
+          };
+      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 Lucene99HnswVectorsReader.DenseOffHeapVectorValues for the 
graph construction
+      // doesn't need to know docIds
+      // TODO: separate random access vector values from DocIdSetIterator?
+      int byteSize = fieldInfo.getVectorDimension() * 
fieldInfo.getVectorEncoding().byteSize;
+      OnHeapHnswGraph graph = null;
+      int[][] vectorIndexNodeOffsets = null;
+      if (docsWithField.cardinality() != 0) {
+        // build graph
+        int initializerIndex = selectGraphForInitialization(mergeState, 
fieldInfo);
+        graph =
+            switch (fieldInfo.getVectorEncoding()) {
+              case BYTE -> {
+                OffHeapByteVectorValues.DenseOffHeapVectorValues vectorValues =
+                    new OffHeapByteVectorValues.DenseOffHeapVectorValues(
+                        fieldInfo.getVectorDimension(),
+                        docsWithField.cardinality(),
+                        vectorDataInput,
+                        byteSize);
+                RandomVectorScorerSupplier scorerSupplier =
+                    RandomVectorScorerSupplier.createBytes(
+                        vectorValues, fieldInfo.getVectorSimilarityFunction());
+                HnswGraphBuilder hnswGraphBuilder =
+                    createHnswGraphBuilder(mergeState, fieldInfo, 
scorerSupplier, initializerIndex);
+                hnswGraphBuilder.setInfoStream(segmentWriteState.infoStream);
+                yield hnswGraphBuilder.build(vectorValues.size());
+              }
+              case FLOAT32 -> {
+                final int maxOrd;
+                final RandomVectorScorerSupplier scorerSupplier;
+                if (quantizationState != null) {
+                  RandomAccessQuantizedByteVectorValues values =
+                      new 
OffHeapQuantizedByteVectorValues.DenseOffHeapVectorValues(
+                          fieldInfo.getVectorDimension(),
+                          docsWithField.cardinality(),
+                          quantizationDataInput);
+                  maxOrd = values.size();
+                  scorerSupplier =
+                      new ScalarQuantizedRandomVectorScorerSupplier(
+                          fieldInfo.getVectorSimilarityFunction(),
+                          quantizationState.mergeQuantile,
+                          values);
+                } else {
+                  OffHeapFloatVectorValues.DenseOffHeapVectorValues 
vectorValues =
+                      new OffHeapFloatVectorValues.DenseOffHeapVectorValues(
+                          fieldInfo.getVectorDimension(),
+                          docsWithField.cardinality(),
+                          vectorDataInput,
+                          byteSize);
+                  maxOrd = vectorValues.size();
+                  scorerSupplier =
+                      RandomVectorScorerSupplier.createFloats(
+                          vectorValues, 
fieldInfo.getVectorSimilarityFunction());
+                }
+                HnswGraphBuilder hnswGraphBuilder =
+                    createHnswGraphBuilder(mergeState, fieldInfo, 
scorerSupplier, initializerIndex);
+                hnswGraphBuilder.setInfoStream(segmentWriteState.infoStream);
+                yield hnswGraphBuilder.build(maxOrd);
+              }
+            };
+        vectorIndexNodeOffsets = writeGraph(graph);
+      }
+      long vectorIndexLength = vectorIndex.getFilePointer() - 
vectorIndexOffset;
+      writeMeta(
+          quantizationState != null,
+          fieldInfo,
+          segmentWriteState.segmentInfo.maxDoc(),
+          quantizationState == null ? null : 
quantizationState.configuredQuantile,
+          quantizationState == null ? null : 
quantizationState.mergeQuantile.getLowerQuantile(),
+          quantizationState == null ? null : 
quantizationState.mergeQuantile.getUpperQuantile(),
+          quantizedVectorDataOffsetAndLength,
+          vectorDataOffset,
+          vectorDataLength,
+          vectorIndexOffset,
+          vectorIndexLength,
+          docsWithField,
+          graph,
+          vectorIndexNodeOffsets);
+      success = true;
+    } finally {
+      IOUtils.close(vectorDataInput, quantizationDataInput);
+      if (success) {
+        segmentWriteState.directory.deleteFile(tempVectorData.getName());
+        if (quantizationState != null) {
+          
segmentWriteState.directory.deleteFile(quantizationState.tempVectorFileName);
+        }
+      } else {
+        IOUtils.closeWhileHandlingException(tempVectorData);
+        IOUtils.deleteFilesIgnoringExceptions(
+            segmentWriteState.directory, tempVectorData.getName());
+      }
+    }
+  }
+
+  private HnswGraphBuilder createHnswGraphBuilder(
+      MergeState mergeState,
+      FieldInfo fieldInfo,
+      RandomVectorScorerSupplier scorerSupplier,
+      int initializerIndex)
+      throws IOException {
+    if (initializerIndex == -1) {
+      return HnswGraphBuilder.create(scorerSupplier, M, beamWidth, 
HnswGraphBuilder.randSeed);
+    }
+
+    HnswGraph initializerGraph =
+        getHnswGraphFromReader(fieldInfo.name, 
mergeState.knnVectorsReaders[initializerIndex]);
+    Map<Integer, Integer> ordinalMapper =
+        getOldToNewOrdinalMap(mergeState, fieldInfo, initializerIndex);
+    return HnswGraphBuilder.create(
+        scorerSupplier, M, beamWidth, HnswGraphBuilder.randSeed, 
initializerGraph, ordinalMapper);
+  }
+
+  private int selectGraphForInitialization(MergeState mergeState, FieldInfo 
fieldInfo)
+      throws IOException {
+    // Find the KnnVectorReader with the most docs that meets the following 
criteria:
+    //  1. Does not contain any deleted docs
+    //  2. Is a Lucene99HnswVectorsReader/PerFieldKnnVectorReader
+    // If no readers exist that meet this criteria, return -1. If they do, 
return their index in
+    // merge state
+    int maxCandidateVectorCount = 0;
+    int initializerIndex = -1;
+
+    for (int i = 0; i < mergeState.liveDocs.length; i++) {
+      KnnVectorsReader currKnnVectorsReader = mergeState.knnVectorsReaders[i];
+      if (mergeState.knnVectorsReaders[i]
+          instanceof PerFieldKnnVectorsFormat.FieldsReader candidateReader) {
+        currKnnVectorsReader = candidateReader.getFieldReader(fieldInfo.name);
+      }
+
+      if (!allMatch(mergeState.liveDocs[i])
+          || !(currKnnVectorsReader instanceof HnswGraphProvider)) {
+        continue;
+      }
+
+      int candidateVectorCount = 0;
+      switch (fieldInfo.getVectorEncoding()) {
+        case BYTE -> {
+          ByteVectorValues byteVectorValues =
+              currKnnVectorsReader.getByteVectorValues(fieldInfo.name);
+          if (byteVectorValues == null) {
+            continue;
+          }
+          candidateVectorCount = byteVectorValues.size();
+        }
+        case FLOAT32 -> {
+          FloatVectorValues vectorValues =
+              currKnnVectorsReader.getFloatVectorValues(fieldInfo.name);
+          if (vectorValues == null) {
+            continue;
+          }
+          candidateVectorCount = vectorValues.size();
+        }
+      }
+
+      if (candidateVectorCount > maxCandidateVectorCount) {
+        maxCandidateVectorCount = candidateVectorCount;
+        initializerIndex = i;
+      }
+    }
+    return initializerIndex;
+  }
+
+  private HnswGraph getHnswGraphFromReader(String fieldName, KnnVectorsReader 
knnVectorsReader)
+      throws IOException {
+    if (knnVectorsReader instanceof PerFieldKnnVectorsFormat.FieldsReader 
perFieldReader
+        && perFieldReader.getFieldReader(fieldName) instanceof 
HnswGraphProvider fieldReader) {
+      return fieldReader.getGraph(fieldName);
+    }
+
+    if (knnVectorsReader instanceof HnswGraphProvider) {
+      return ((HnswGraphProvider) knnVectorsReader).getGraph(fieldName);
+    }
+
+    // We should not reach here because knnVectorsReader's type is checked in
+    // selectGraphForInitialization
+    throw new IllegalArgumentException(
+        "Invalid KnnVectorsReader type for field: "
+            + fieldName
+            + ". Must be Lucene99HnswVectorsReader or newer");
+  }
+
+  private Map<Integer, Integer> getOldToNewOrdinalMap(
+      MergeState mergeState, FieldInfo fieldInfo, int initializerIndex) throws 
IOException {
+
+    DocIdSetIterator initializerIterator = null;
+
+    switch (fieldInfo.getVectorEncoding()) {
+      case BYTE -> initializerIterator =
+          
mergeState.knnVectorsReaders[initializerIndex].getByteVectorValues(fieldInfo.name);
+      case FLOAT32 -> initializerIterator =
+          
mergeState.knnVectorsReaders[initializerIndex].getFloatVectorValues(fieldInfo.name);
+    }
+
+    MergeState.DocMap initializerDocMap = mergeState.docMaps[initializerIndex];
+
+    Map<Integer, Integer> newIdToOldOrdinal = new HashMap<>();
+    int oldOrd = 0;
+    int maxNewDocID = -1;
+    for (int oldId = initializerIterator.nextDoc();
+        oldId != NO_MORE_DOCS;
+        oldId = initializerIterator.nextDoc()) {
+      if (isCurrentVectorNull(initializerIterator)) {
+        continue;
+      }
+      int newId = initializerDocMap.get(oldId);
+      maxNewDocID = Math.max(newId, maxNewDocID);
+      newIdToOldOrdinal.put(newId, oldOrd);
+      oldOrd++;
+    }
+
+    if (maxNewDocID == -1) {
+      return Collections.emptyMap();
+    }
+
+    Map<Integer, Integer> oldToNewOrdinalMap = new HashMap<>();
+
+    DocIdSetIterator vectorIterator = null;
+    switch (fieldInfo.getVectorEncoding()) {
+      case BYTE -> vectorIterator = 
MergedVectorValues.mergeByteVectorValues(fieldInfo, mergeState);
+      case FLOAT32 -> vectorIterator =
+          MergedVectorValues.mergeFloatVectorValues(fieldInfo, mergeState);
+    }
+
+    int newOrd = 0;
+    for (int newDocId = vectorIterator.nextDoc();
+        newDocId <= maxNewDocID;
+        newDocId = vectorIterator.nextDoc()) {
+      if (isCurrentVectorNull(vectorIterator)) {
+        continue;
+      }
+
+      if (newIdToOldOrdinal.containsKey(newDocId)) {
+        oldToNewOrdinalMap.put(newIdToOldOrdinal.get(newDocId), newOrd);
+      }
+      newOrd++;
+    }
+
+    return oldToNewOrdinalMap;
+  }
+
+  private boolean isCurrentVectorNull(DocIdSetIterator docIdSetIterator) 
throws IOException {
+    if (docIdSetIterator instanceof FloatVectorValues) {
+      return ((FloatVectorValues) docIdSetIterator).vectorValue() == null;
+    }
+
+    if (docIdSetIterator instanceof ByteVectorValues) {
+      return ((ByteVectorValues) docIdSetIterator).vectorValue() == null;
+    }
+
+    return true;
+  }
+
+  private boolean allMatch(Bits bits) {
+    if (bits == null) {
+      return true;
+    }
+
+    for (int i = 0; i < bits.length(); i++) {
+      if (!bits.get(i)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /**
+   * @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++) {
+      int[] sortedNodes = getSortedNodes(graph.getNodesOnLevel(level));
+      offsets[level] = new int[sortedNodes.length];
+      int nodeOffsetId = 0;
+      for (int node : sortedNodes) {
+        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;
+  }
+
+  public static int[] getSortedNodes(NodesIterator nodesOnLevel) {
+    int[] sortedNodes = new int[nodesOnLevel.size()];
+    for (int n = 0; nodesOnLevel.hasNext(); n++) {
+      sortedNodes[n] = nodesOnLevel.nextInt();
+    }
+    Arrays.sort(sortedNodes);
+    return sortedNodes;
+  }
+
+  private void writeMeta(
+      boolean isQuantized,
+      FieldInfo field,
+      int maxDoc,
+      Float configuredQuantizationQuantile,
+      Float lowerQuantile,
+      Float upperQuantile,
+      long[] quantizedVectorDataOffsetAndLen,
+      long vectorDataOffset,
+      long vectorDataLength,
+      long vectorIndexOffset,
+      long vectorIndexLength,
+      DocsWithFieldSet docsWithField,
+      HnswGraph graph,
+      int[][] graphLevelNodeOffsets)
+      throws IOException {
+    meta.writeInt(field.number);
+    meta.writeInt(field.getVectorEncoding().ordinal());
+    meta.writeInt(field.getVectorSimilarityFunction().ordinal());
+    meta.writeByte(isQuantized ? (byte) 1 : (byte) 0);
+    if (isQuantized) {
+      assert lowerQuantile != null
+          && upperQuantile != null
+          && quantizedVectorDataOffsetAndLen != null;
+      assert quantizedVectorDataOffsetAndLen.length == 2;
+      meta.writeInt(
+          Float.floatToIntBits(
+              configuredQuantizationQuantile != null
+                  ? configuredQuantizationQuantile
+                  : calculateDefaultQuantile(field.getVectorDimension())));
+      meta.writeInt(Float.floatToIntBits(lowerQuantile));
+      meta.writeInt(Float.floatToIntBits(upperQuantile));
+      meta.writeVLong(quantizedVectorDataOffsetAndLen[0]);
+      meta.writeVLong(quantizedVectorDataOffsetAndLen[1]);
+    } else {
+      assert configuredQuantizationQuantile == null
+          && lowerQuantile == null
+          && upperQuantile == null
+          && quantizedVectorDataOffsetAndLen == null;
+    }
+    meta.writeVLong(vectorDataOffset);
+    meta.writeVLong(vectorDataLength);
+    meta.writeVLong(vectorIndexOffset);
+    meta.writeVLong(vectorIndexLength);
+    meta.writeVInt(field.getVectorDimension());
+
+    // write docIDs
+    int count = docsWithField.cardinality();
+    meta.writeInt(count);
+    if (isQuantized) {
+      writeOrdToDoc(quantizedVectorData, docsWithField, count, maxDoc);

Review Comment:
   Is it necessary to create a new mapping? Could we leverage the existing 
mapping that was established for the raw vectors?



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsWriter.java:
##########
@@ -0,0 +1,1170 @@
+/*
+ * 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.lucene99;
+
+import static 
org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat.DIRECT_MONOTONIC_BLOCK_SHIFT;
+import static 
org.apache.lucene.codecs.lucene99.Lucene99ScalarQuantizedVectorsFormat.calculateDefaultQuantile;
+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.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.HnswGraphProvider;
+import org.apache.lucene.codecs.KnnFieldVectorsWriter;
+import org.apache.lucene.codecs.KnnVectorsReader;
+import org.apache.lucene.codecs.KnnVectorsWriter;
+import org.apache.lucene.codecs.lucene90.IndexedDISI;
+import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
+import org.apache.lucene.index.ByteVectorValues;
+import org.apache.lucene.index.DocsWithFieldSet;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FloatVectorValues;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.MergeState;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.index.Sorter;
+import org.apache.lucene.index.VectorEncoding;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.InfoStream;
+import org.apache.lucene.util.RamUsageEstimator;
+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.hnsw.RandomAccessVectorValues;
+import org.apache.lucene.util.hnsw.RandomVectorScorerSupplier;
+import org.apache.lucene.util.packed.DirectMonotonicWriter;
+
+/**
+ * Writes vector values and knn graphs to index segments.
+ *
+ * @lucene.experimental
+ */
+public final class Lucene99HnswVectorsWriter extends KnnVectorsWriter {
+
+  private final SegmentWriteState segmentWriteState;
+  private final IndexOutput meta, vectorData, quantizedVectorData, vectorIndex;
+  private final int M;
+  private final int beamWidth;
+  private final Lucene99ScalarQuantizedVectorsWriter quantizedVectorsWriter;
+
+  private final List<FieldWriter<?>> fields = new ArrayList<>();
+  private boolean finished;
+
+  Lucene99HnswVectorsWriter(
+      SegmentWriteState state,
+      int M,
+      int beamWidth,
+      Lucene99ScalarQuantizedVectorsFormat quantizedVectorsFormat)
+      throws IOException {
+    this.M = M;
+    this.beamWidth = beamWidth;
+    segmentWriteState = state;
+    String metaFileName =
+        IndexFileNames.segmentFileName(
+            state.segmentInfo.name, state.segmentSuffix, 
Lucene99HnswVectorsFormat.META_EXTENSION);
+
+    String vectorDataFileName =
+        IndexFileNames.segmentFileName(
+            state.segmentInfo.name,
+            state.segmentSuffix,
+            Lucene99HnswVectorsFormat.VECTOR_DATA_EXTENSION);
+
+    String indexDataFileName =
+        IndexFileNames.segmentFileName(
+            state.segmentInfo.name,
+            state.segmentSuffix,
+            Lucene99HnswVectorsFormat.VECTOR_INDEX_EXTENSION);
+
+    final String quantizedVectorDataFileName =
+        quantizedVectorsFormat != null
+            ? IndexFileNames.segmentFileName(
+                state.segmentInfo.name,
+                state.segmentSuffix,
+                
Lucene99ScalarQuantizedVectorsFormat.QUANTIZED_VECTOR_DATA_EXTENSION)
+            : null;
+    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,
+          Lucene99HnswVectorsFormat.META_CODEC_NAME,
+          Lucene99HnswVectorsFormat.VERSION_CURRENT,
+          state.segmentInfo.getId(),
+          state.segmentSuffix);
+      CodecUtil.writeIndexHeader(
+          vectorData,
+          Lucene99HnswVectorsFormat.VECTOR_DATA_CODEC_NAME,
+          Lucene99HnswVectorsFormat.VERSION_CURRENT,
+          state.segmentInfo.getId(),
+          state.segmentSuffix);
+      CodecUtil.writeIndexHeader(
+          vectorIndex,
+          Lucene99HnswVectorsFormat.VECTOR_INDEX_CODEC_NAME,
+          Lucene99HnswVectorsFormat.VERSION_CURRENT,
+          state.segmentInfo.getId(),
+          state.segmentSuffix);
+      if (quantizedVectorDataFileName != null) {
+        quantizedVectorData =
+            state.directory.createOutput(quantizedVectorDataFileName, 
state.context);
+        CodecUtil.writeIndexHeader(
+            quantizedVectorData,
+            
Lucene99ScalarQuantizedVectorsFormat.QUANTIZED_VECTOR_DATA_CODEC_NAME,
+            Lucene99ScalarQuantizedVectorsFormat.VERSION_CURRENT,
+            state.segmentInfo.getId(),
+            state.segmentSuffix);
+        quantizedVectorsWriter =
+            new Lucene99ScalarQuantizedVectorsWriter(
+                quantizedVectorData, quantizedVectorsFormat.quantile);
+      } else {
+        quantizedVectorData = null;
+        quantizedVectorsWriter = null;
+      }
+      success = true;
+    } finally {
+      if (success == false) {
+        IOUtils.closeWhileHandlingException(this);
+      }
+    }
+  }
+
+  @Override
+  public KnnFieldVectorsWriter<?> addField(FieldInfo fieldInfo) throws 
IOException {
+    Lucene99ScalarQuantizedVectorsWriter.QuantizationVectorWriter 
quantizedVectorFieldWriter = null;
+    // Quantization only supports FLOAT32 for now
+    if (quantizedVectorsWriter != null
+        && fieldInfo.getVectorEncoding().equals(VectorEncoding.FLOAT32)) {
+      quantizedVectorFieldWriter = quantizedVectorsWriter.addField(fieldInfo);
+    }
+    FieldWriter<?> newField =
+        FieldWriter.create(
+            fieldInfo, M, beamWidth, segmentWriteState.infoStream, 
quantizedVectorFieldWriter);
+    fields.add(newField);
+    return newField;
+  }
+
+  @Override
+  public void flush(int maxDoc, Sorter.DocMap sortMap) throws IOException {
+    for (FieldWriter<?> field : fields) {
+      long[] quantizedVectorOffsetAndLen = null;
+      if (field.quantizedWriter != null) {
+        assert quantizedVectorsWriter != null;
+        quantizedVectorOffsetAndLen =
+            quantizedVectorsWriter.flush(sortMap, field.quantizedWriter, 
field.docsWithField);
+      }
+      if (sortMap == null) {
+        writeField(field, maxDoc, quantizedVectorOffsetAndLen);
+      } else {
+        writeSortingField(field, maxDoc, sortMap, quantizedVectorOffsetAndLen);
+      }
+    }
+  }
+
+  @Override
+  public void finish() throws IOException {
+    if (finished) {
+      throw new IllegalStateException("already finished");
+    }
+    finished = true;
+    if (quantizedVectorsWriter != null) {
+      quantizedVectorsWriter.finish();
+    }
+
+    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, long[] 
quantizedVecOffsetAndLen)
+      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.isQuantized(),
+        fieldData.fieldInfo,
+        maxDoc,
+        fieldData.getConfiguredQuantile(),
+        fieldData.getMinQuantile(),
+        fieldData.getMaxQuantile(),
+        quantizedVecOffsetAndLen,
+        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);
+    for (Object v : fieldData.vectors) {
+      buffer.asFloatBuffer().put((float[]) v);
+      vectorData.writeBytes(buffer.array(), buffer.array().length);
+    }
+  }
+
+  private void writeByteVectors(FieldWriter<?> fieldData) throws IOException {
+    for (Object v : fieldData.vectors) {
+      byte[] vector = (byte[]) v;
+      vectorData.writeBytes(vector, vector.length);
+    }
+  }
+
+  private void writeSortingField(
+      FieldWriter<?> fieldData,
+      int maxDoc,
+      Sorter.DocMap sortMap,
+      long[] quantizedVectorOffsetAndLen)
+      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.isQuantized(),
+        fieldData.fieldInfo,
+        maxDoc,
+        fieldData.getConfiguredQuantile(),
+        fieldData.getMinQuantile(),
+        fieldData.getMaxQuantile(),
+        quantizedVectorOffsetAndLen,
+        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);
+    for (int ordinal : ordMap) {
+      float[] vector = (float[]) fieldData.vectors.get(ordinal);
+      buffer.asFloatBuffer().put(vector);
+      vectorData.writeBytes(buffer.array(), buffer.array().length);
+    }
+    return vectorDataOffset;
+  }
+
+  private long writeSortedByteVectors(FieldWriter<?> fieldData, int[] ordMap) 
throws IOException {
+    long vectorDataOffset = vectorData.alignFilePointer(Float.BYTES);
+    for (int ordinal : ordMap) {
+      byte[] vector = (byte[]) fieldData.vectors.get(ordinal);
+      vectorData.writeBytes(vector, 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();
+    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, maxOrd);
+      levelNodeOffsets[0][node] = Math.toIntExact(vectorIndex.getFilePointer() 
- offset);
+    }
+
+    for (int level = 1; level < graph.numLevels(); level++) {
+      NodesIterator nodesOnLevel = graph.getNodesOnLevel(level);
+      int[] newNodes = new int[nodesOnLevel.size()];
+      for (int n = 0; nodesOnLevel.hasNext(); n++) {
+        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, 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 ArrayNodesIterator(nodesByLevel.get(level), 
nodesByLevel.get(level).length);
+        }
+      }
+    };
+  }
+
+  private void reconstructAndWriteNeigbours(NeighborArray neighbors, int[] 
oldToNewMap, 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);
+    IndexOutput tempVectorData =
+        segmentWriteState.directory.createTempOutput(
+            vectorData.getName(), "temp", segmentWriteState.context);
+    IndexInput vectorDataInput = null;
+    IndexInput quantizationDataInput = null;
+    Lucene99ScalarQuantizedVectorsWriter.MergedQuantileState quantizationState 
= null;
+    boolean success = false;
+    try {
+      long[] quantizedVectorDataOffsetAndLength = null;
+      if (quantizedVectorsWriter != null) {
+        quantizationState =
+            quantizedVectorsWriter.mergeOneField(segmentWriteState, fieldInfo, 
mergeState);
+        if (quantizationState != null) {
+          quantizedVectorDataOffsetAndLength = new long[2];
+          quantizedVectorDataOffsetAndLength[0] = 
quantizedVectorData.alignFilePointer(Float.BYTES);
+          quantizationDataInput =
+              segmentWriteState.directory.openInput(
+                  quantizationState.tempVectorFileName, 
segmentWriteState.context);
+          quantizedVectorData.copyBytes(
+              quantizationDataInput, quantizationDataInput.length() - 
CodecUtil.footerLength());
+          quantizedVectorDataOffsetAndLength[1] =
+              quantizedVectorData.getFilePointer() - 
quantizedVectorDataOffsetAndLength[0];
+          CodecUtil.retrieveChecksum(quantizationDataInput);
+        }
+      }
+      // write the vector data to a temporary file

Review Comment:
   We don't need to write the raw vectors in a temporary file if we build the 
graph with the quantised vectors? This part could be extracted in a separate 
function that writes the raw and quantised vectors and returns a "closeable" 
`RandomVectorScorerSupplier`.



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene99/package-info.java:
##########
@@ -0,0 +1,437 @@
+/*
+ * 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.8 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.lucene99.Lucene99HnswVectorsFormat 
Vector values}. The
+ *       vector format stores numeric vectors in a format optimized for random 
access and
+ *       computation, supporting high-dimensional nearest-neighbor search.
+ *   <li>{@link 
org.apache.lucene.codecs.lucene99.Lucene99ScalarQuantizedVectorsFormat quantized
+ *       vector values}. The quantized vector format stores numeric vectors in 
a scalar quantized
+ *       format optimized for random access and computation. Can be used in 
conjunction with HNSW to
+ *       support approximate nearest-neighbor search at query time.
+ * </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.lucene99.Lucene99HnswVectorsFormat 
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>
+ * <tr>
+ * <td>{@link 
org.apache.lucene.codecs.lucene99.Lucene99ScalarQuantizedVectorsFormat 
Quantized Vector values}</td>
+ * <td>.veq, .vemq</td>
+ * <td><code>.veq</code>Holds quantized vector values; the byte[] encoded 
vectors and relevant offsets
+ * <code>.vemq</code> the quantized 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 
delta-encoded with vints.
+ *       Additionally, metadata file size improvements were made by 
delta-encoding nodes by graph
+ *       layer and not writing the node ids for the zeroth layer.
+ *   <li>In version 9.8, Vector scalar quantization support was added. 
Allowing the HNSW vector

Review Comment:
   ```suggestion
    *   <li>In version 9.9, Vector scalar quantization support was added. 
Allowing the HNSW vector
   ```



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

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

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


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

Reply via email to