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 2020/09/29 15:58:59 UTC

[GitHub] [lucene-solr] msokolov opened a new pull request #1930: LUCENE-9322: add VectorValues to new Lucene90 codec

msokolov opened a new pull request #1930:
URL: https://github.com/apache/lucene-solr/pull/1930


   This adds a floating-point vector format building on the designs in Lucene-9004 and LUCENE-9322. This patch fully supports indexing and reading vectors with an iterator, and a random-access API. Support for search based on an NSW graph implementation is intended to follow soon, but I wanted to include the vector APIs that I needed to get that working, even though they are not yet used here, so eg it includes the definition of a scoring function and a nearest-neighbors search API, but no implementation of search yet. My intention is to keep the ANN implementation hidden, so graphs and other supporting data structures (eg we might want to support LSH or k-means clustering and so on) would be implementation details invoked by a configuration on the VectorField/VectorValues. At the moment you can specify a ScoringFunction, and it is implicit that NSW will be the result. In the future we could add another parameter to ScoringFunction and/or new functions to represent support for other 
 algorithms.
   
   Some open questions: 
   
   1. Should this be Lucene 9.0 only? In this patch I added Lucene90 Codec. If we do this then it would be awkward to backport.
   2. It seems messy to have the ScoringFunction implementation in the main VectorValues interface API file. I'd appreciate any better suggestion for how to organize this.
   3. Vector scoring can return negative numbers. I'd like to have first-class support for dot product distance (which can be negative) since that's what my consumers seem to have settled on. I don't think we need to be compatible with relevance scores, at least not directly in the KNN search API, but IDK maybe we should? We could renormalize/convert from dot-product scores to a positive score with math in the output layer where we return the scores. So far this is just specification question as there is no implementation of search yet.
   4. I think there is room for improvement in some of the data structures used to map docids to dense vector ordinals and back. I'd appreciate comments on that, but maybe we could revisit in a fast follow-on issue?


----------------------------------------------------------------
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.

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


[GitHub] [lucene-solr] msokolov commented on a change in pull request #1930: LUCENE-9322: add VectorValues to new Lucene90 codec

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #1930:
URL: https://github.com/apache/lucene-solr/pull/1930#discussion_r499858704



##########
File path: lucene/core/src/java/org/apache/lucene/codecs/VectorWriter.java
##########
@@ -0,0 +1,253 @@
+/*
+ * 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;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.lucene.index.DocIDMerger;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.MergeState;
+import org.apache.lucene.index.VectorValues;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.util.BytesRef;
+
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+/**
+ * Writes vectors to an index.
+ */
+public abstract class VectorWriter implements Closeable {
+
+  /** Sole constructor */
+  protected VectorWriter() {}
+
+  /** Write all values contained in the provided reader */
+  public abstract void writeField(FieldInfo fieldInfo, VectorValues values) throws IOException;
+
+  /** Called once at the end before close */
+  public abstract void finish() throws IOException;
+
+  /** Merge the vector values from multiple segments, for all fields */
+  public void merge(MergeState mergeState) throws IOException {
+    for (VectorReader reader : mergeState.vectorReaders) {
+      if (reader != null) {
+        reader.checkIntegrity();
+      }
+    }
+    for (FieldInfo fieldInfo : mergeState.mergeFieldInfos) {
+      if (fieldInfo.hasVectorValues()) {
+        mergeVectors(fieldInfo, mergeState);
+      }
+    }
+    finish();
+  }
+
+  private void mergeVectors(FieldInfo mergeFieldInfo, final MergeState mergeState) throws IOException {
+    if (mergeState.infoStream.isEnabled("VV")) {
+      mergeState.infoStream.message("VV", "merging " + mergeState.segmentInfo);
+    }
+    List<VectorValuesSub> subs = new ArrayList<>();
+    int dimension = -1;
+    VectorValues.ScoreFunction scoreFunction = null;
+    for (int i = 0; i < mergeState.vectorReaders.length; i++) {
+      VectorReader vectorReader = mergeState.vectorReaders[i];
+      if (vectorReader != null) {
+        if (mergeFieldInfo != null && mergeFieldInfo.hasVectorValues()) {
+          int segmentDimension = mergeFieldInfo.getVectorDimension();
+          VectorValues.ScoreFunction segmentScoreFunction = mergeFieldInfo.getVectorScoreFunction();
+          if (dimension == -1) {
+            dimension = segmentDimension;
+            scoreFunction = mergeFieldInfo.getVectorScoreFunction();
+          } else if (dimension != segmentDimension) {
+            throw new IllegalStateException("Varying dimensions for vector-valued field " + mergeFieldInfo.name
+                + ": " + dimension + "!=" + segmentDimension);
+          } else if (scoreFunction != segmentScoreFunction) {

Review comment:
       Yes, IndexingChain.indexVector calls FieldInfo.setVectorDimensionAndScoreFunction, which checks against existing values. It allows to go from 0 to non-zero dimension (setting scoreFunction at that time), but no other change is allowed. This is tested in TestVectorValues




----------------------------------------------------------------
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.

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


[GitHub] [lucene-solr] msokolov commented on a change in pull request #1930: LUCENE-9322: add VectorValues to new Lucene90 codec

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #1930:
URL: https://github.com/apache/lucene-solr/pull/1930#discussion_r506736220



##########
File path: lucene/core/src/java/org/apache/lucene/index/VectorValues.java
##########
@@ -0,0 +1,273 @@
+/*
+ * 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.index;
+
+import java.io.IOException;
+
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * This class provides access to per-document floating point vector values indexed as {@link
+ * org.apache.lucene.document.VectorField}.
+ */
+public abstract class VectorValues extends DocIdSetIterator {
+
+  /** The maximum length of a vector */
+  public static int MAX_DIMENSIONS = 1024;
+
+  /** Sole constructor */
+  protected VectorValues() {}
+
+  /**
+   * Return the dimension of the vectors
+   */
+  public abstract int dimension();
+
+  /**
+   * TODO: should we use cost() for this? We rely on its always being exactly the number
+   * of documents having a value for this field, which is not guaranteed by the cost() contract,
+   * but in all the implementations so far they are the same.
+   * @return the number of vectors returned by this iterator
+   */
+  public abstract int size();
+
+  /**
+   * Return the score function used to compare these vectors
+   */
+  public abstract ScoreFunction scoreFunction();
+
+  /**
+   * Return the vector value for the current document ID.
+   * It is illegal to call this method when the iterator is not positioned: before advancing, or after failing to advance.
+   * The returned array may be shared across calls, re-used, and modified as the iterator advances.
+   * @return the vector value
+   */
+  public abstract float[] vectorValue() throws IOException;
+
+  /**
+   * Return the binary encoded vector value for the current document ID. These are the bytes corresponding to the float array
+   * in IEEE 754 standard encoding, encoded using little-endian byte order.
+   * It is illegal to call this method when the iterator is not positioned: before advancing, or after failing to advance.
+   * The returned storage may be shared across calls, re-used and modified as the iterator advances.
+   * @return the binary value
+   */
+  public BytesRef binaryValue() throws IOException {

Review comment:
       Yes, I'll remove the mention of the precise encoding.




----------------------------------------------------------------
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.

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


[GitHub] [lucene-solr] msokolov commented on a change in pull request #1930: LUCENE-9322: add VectorValues to new Lucene90 codec

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #1930:
URL: https://github.com/apache/lucene-solr/pull/1930#discussion_r506955463



##########
File path: lucene/core/src/java/org/apache/lucene/index/VectorValues.java
##########
@@ -0,0 +1,273 @@
+/*
+ * 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.index;
+
+import java.io.IOException;
+
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * This class provides access to per-document floating point vector values indexed as {@link
+ * org.apache.lucene.document.VectorField}.
+ */
+public abstract class VectorValues extends DocIdSetIterator {
+
+  /** The maximum length of a vector */
+  public static int MAX_DIMENSIONS = 1024;
+
+  /** Sole constructor */
+  protected VectorValues() {}
+
+  /**
+   * Return the dimension of the vectors
+   */
+  public abstract int dimension();
+
+  /**
+   * TODO: should we use cost() for this? We rely on its always being exactly the number
+   * of documents having a value for this field, which is not guaranteed by the cost() contract,
+   * but in all the implementations so far they are the same.
+   * @return the number of vectors returned by this iterator
+   */
+  public abstract int size();
+
+  /**
+   * Return the score function used to compare these vectors
+   */
+  public abstract ScoreFunction scoreFunction();
+
+  /**
+   * Return the vector value for the current document ID.
+   * It is illegal to call this method when the iterator is not positioned: before advancing, or after failing to advance.
+   * The returned array may be shared across calls, re-used, and modified as the iterator advances.
+   * @return the vector value
+   */
+  public abstract float[] vectorValue() throws IOException;

Review comment:
       Thinking about this some more: why do we have the random access interface? It enables a vector consumer to build efficient data structures using vector ordinals as keys, allowing them to avoid an extra step of mapping from docid to vector ordinal and back. It's expected that such a consumer will maintain their own such mapping - the point is that we don't want to force them to constantly be mapping (by eg defining this API in terms of docids). And we don't want to provide a less-than-ideal mapping function, or spend extra effort to maintain a mapping that may not be used.
   
   We could try hiding this behind a more opaque class structure, but given that this interface needs to be accessed in o.a.l.index and o.a.l.codecs I think it would have to be public, and I'm not sure what the benefit would be.
   
   I think we can always *add* `ordinal(int docId)` and `docId(int ordinal)` methods as they become useful? I think that would be a good API, and nothing here prevents it from being added later, so I propose to commit this as is.




----------------------------------------------------------------
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.

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


[GitHub] [lucene-solr] msokolov commented on pull request #1930: LUCENE-9322: add VectorValues to new Lucene90 codec

Posted by GitBox <gi...@apache.org>.
msokolov commented on pull request #1930:
URL: https://github.com/apache/lucene-solr/pull/1930#issuecomment-707431740


   > One option could be 200-dimensional GloVe word vectors, available from http://ann-benchmarks.com/glove-200-angular.hdf5. I think these are trained on Twitter data.
   
   +1 I'm looking into adding GloVe data to luceneutil benchmarks, initially just to index and retrieve them, then I hope to add tasks for scoring lexical matches, and then for knn matching. I think some of the GloVe datasets are trained on wikipedia (plus other text) so should be suitable for use in our benchmarks, which are based on wikipedia text.
   
   I think for initial performance comparisons we can use our own tool; it wouldn't be as nicely controlled as running in the same framework, but if we are careful the results should be comparable. And it's good to know there is a reasonable path for integrating with ann-benchmark using py4j, and I hadn't realized there was a --batch option.


----------------------------------------------------------------
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.

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


[GitHub] [lucene-solr] msokolov commented on a change in pull request #1930: LUCENE-9322: add VectorValues to new Lucene90 codec

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #1930:
URL: https://github.com/apache/lucene-solr/pull/1930#discussion_r499893497



##########
File path: lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextVectorReader.java
##########
@@ -0,0 +1,304 @@
+/*
+ * 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.simpletext;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.codecs.VectorReader;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.VectorValues;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.store.BufferedChecksumIndexInput;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.StringHelper;
+
+import static org.apache.lucene.codecs.simpletext.SimpleTextVectorWriter.*;
+
+/**
+ * Reads vector values from a simple text format. All vectors are read up front and cached in RAM in order to support
+ * random access.
+ * <b>FOR RECREATIONAL USE ONLY</b>
+ * @lucene.experimental
+ */
+public class SimpleTextVectorReader extends VectorReader {
+
+  private static final BytesRef EMPTY = new BytesRef("");
+
+  private final SegmentReadState readState;
+  private final IndexInput dataIn;
+  private final BytesRefBuilder scratch = new BytesRefBuilder();
+  private final Map<String, FieldEntry> fieldEntries = new HashMap<>();
+
+  SimpleTextVectorReader(SegmentReadState readState) throws IOException {
+    this.readState = readState;
+    String metaFileName = IndexFileNames.segmentFileName(readState.segmentInfo.name, readState.segmentSuffix, SimpleTextVectorFormat.META_EXTENSION);
+    try (ChecksumIndexInput in = readState.directory.openChecksumInput(metaFileName, IOContext.DEFAULT)) {
+      int fieldNumber = readInt(in, FIELD_NUMBER);
+      while (fieldNumber != -1) {
+        String fieldName = readString(in, FIELD_NAME);
+        String scoreFunctionName = readString(in, SCORE_FUNCTION);
+        VectorValues.ScoreFunction scoreFunction = VectorValues.ScoreFunction.valueOf(scoreFunctionName);
+        long vectorDataOffset = readLong(in, VECTOR_DATA_OFFSET);
+        long vectorDataLength = readLong(in, VECTOR_DATA_LENGTH);
+        int dimension = readInt(in, VECTOR_DIMENSION);
+        int size = readInt(in, SIZE);
+        int[] docIds = new int[size];
+        for (int i = 0; i < size; i++) {
+          docIds[i] = readInt(in, EMPTY);
+        }
+        assert fieldEntries.containsKey(fieldName) == false;
+        fieldEntries.put(fieldName, new FieldEntry(dimension, scoreFunction, vectorDataOffset, vectorDataLength, docIds));
+        fieldNumber = readInt(in, FIELD_NUMBER);
+      }
+      SimpleTextUtil.checkFooter(in);
+    }
+
+    String vectorFileName = IndexFileNames.segmentFileName(readState.segmentInfo.name, readState.segmentSuffix, SimpleTextVectorFormat.VECTOR_EXTENSION);
+    dataIn = readState.directory.openInput(vectorFileName, IOContext.DEFAULT);
+  }
+
+  @Override
+  public VectorValues getVectorValues(String field) throws IOException {
+    FieldInfo info = readState.fieldInfos.fieldInfo(field);
+    if (info == null) {
+      throw new IllegalStateException("No vectors indexed for field=\"" + field + "\"");
+    }
+    int dimension = info.getVectorDimension();
+    if (dimension == 0) {
+      return VectorValues.EMPTY;
+    }
+    FieldEntry fieldEntry = fieldEntries.get(field);
+    if (fieldEntry == null) {
+      throw new IllegalStateException("No entry found for vector field=\"" + field + "\"");
+    }
+    if (dimension != fieldEntry.dimension) {
+      throw new IllegalStateException("Inconsistent vector dimension for field=\"" + field + "\"; " + dimension + " != " + fieldEntry.dimension);
+    }
+    IndexInput bytesSlice = dataIn.slice("vector-data", fieldEntry.vectorDataOffset, fieldEntry.vectorDataLength);
+    return new SimpleTextVectorValues(fieldEntry, bytesSlice);
+  }
+
+  @Override
+  public void checkIntegrity() throws IOException {
+    IndexInput clone = dataIn.clone();
+    clone.seek(0);
+
+    // checksum is fixed-width encoded with 20 bytes, plus 1 byte for newline (the space is included in SimpleTextUtil.CHECKSUM):
+    long footerStartPos = dataIn.length() - (SimpleTextUtil.CHECKSUM.length + 21);
+    ChecksumIndexInput input = new BufferedChecksumIndexInput(clone);
+    while (true) {
+      SimpleTextUtil.readLine(input, scratch);
+      if (input.getFilePointer() >= footerStartPos) {
+        // Make sure we landed at precisely the right location:
+        if (input.getFilePointer() != footerStartPos) {
+          throw new CorruptIndexException("SimpleText failure: footer does not start at expected position current=" + input.getFilePointer() + " vs expected=" + footerStartPos, input);
+        }
+        SimpleTextUtil.checkFooter(input);
+        break;
+      }
+    }
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return 0;
+  }
+
+  @Override
+  public void close() throws IOException {
+    dataIn.close();
+  }
+
+  private static class FieldEntry {
+
+    final int dimension;
+    final VectorValues.ScoreFunction scoreFunction;
+
+    final long vectorDataOffset;
+    final long vectorDataLength;
+    final int[] ordToDoc;
+
+    FieldEntry(int dimension, VectorValues.ScoreFunction scoreFunction,
+               long vectorDataOffset, long vectorDataLength, int[] ordToDoc) {
+      this.dimension = dimension;
+      this.scoreFunction = scoreFunction;
+      this.vectorDataOffset = vectorDataOffset;
+      this.vectorDataLength = vectorDataLength;
+      this.ordToDoc = ordToDoc;
+    }
+
+    int size() {
+      return ordToDoc.length;
+    }
+  }
+
+  private static class SimpleTextVectorValues extends VectorValues implements VectorValues.RandomAccess {
+
+    private final BytesRefBuilder scratch = new BytesRefBuilder();
+    private final FieldEntry entry;
+    private final IndexInput in;
+    private final BytesRef binaryValue;
+    private final float[][] values;
+
+    int curOrd;
+
+    SimpleTextVectorValues(FieldEntry entry, IndexInput in) throws IOException {
+      this.entry = entry;
+      this.in = in;
+      values = new float[entry.size()][entry.dimension];
+      binaryValue = new BytesRef(entry.dimension * Float.BYTES);
+      binaryValue.length = binaryValue.bytes.length;
+      curOrd = -1;
+      readAllVectors();
+    }
+
+    @Override
+    public int dimension() {
+      return entry.dimension;
+    }
+
+    @Override
+    public int size() {
+      return entry.size();
+    }
+
+    @Override
+    public ScoreFunction scoreFunction() {
+      return entry.scoreFunction;
+    }
+
+    @Override
+    public float[] vectorValue() {
+      return values[curOrd];
+    }
+
+    @Override
+    public BytesRef binaryValue() {

Review comment:
       Yeah, seems a bit weird from the API standpoint, yet (1) we primarily want to expose `float[] vectorValue` yet for efficiency when flushing and merging we don't care what the `float[]` value is and just want access to the underlying bytes. Maybe there is a way to provide some private interface for IndexWriter, but I didn't see how.




----------------------------------------------------------------
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.

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


[GitHub] [lucene-solr] mikemccand commented on a change in pull request #1930: LUCENE-9322: add VectorValues to new Lucene90 codec

Posted by GitBox <gi...@apache.org>.
mikemccand commented on a change in pull request #1930:
URL: https://github.com/apache/lucene-solr/pull/1930#discussion_r506561868



##########
File path: lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90VectorReader.java
##########
@@ -0,0 +1,344 @@
+/*
+ * 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.lucene90;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.FloatBuffer;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.VectorReader;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.VectorValues;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.RamUsageEstimator;
+
+/**
+ * Reads vectors from the index segments.
+ */

Review comment:
       Can we mark as `@lucene.experimental`?

##########
File path: lucene/core/src/java/org/apache/lucene/index/VectorValues.java
##########
@@ -0,0 +1,273 @@
+/*
+ * 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.index;
+
+import java.io.IOException;
+
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * This class provides access to per-document floating point vector values indexed as {@link
+ * org.apache.lucene.document.VectorField}.
+ */
+public abstract class VectorValues extends DocIdSetIterator {
+
+  /** The maximum length of a vector */
+  public static int MAX_DIMENSIONS = 1024;
+
+  /** Sole constructor */
+  protected VectorValues() {}
+
+  /**
+   * Return the dimension of the vectors
+   */
+  public abstract int dimension();
+
+  /**
+   * TODO: should we use cost() for this? We rely on its always being exactly the number
+   * of documents having a value for this field, which is not guaranteed by the cost() contract,
+   * but in all the implementations so far they are the same.
+   * @return the number of vectors returned by this iterator
+   */
+  public abstract int size();
+
+  /**
+   * Return the score function used to compare these vectors
+   */
+  public abstract ScoreFunction scoreFunction();
+
+  /**
+   * Return the vector value for the current document ID.
+   * It is illegal to call this method when the iterator is not positioned: before advancing, or after failing to advance.
+   * The returned array may be shared across calls, re-used, and modified as the iterator advances.
+   * @return the vector value
+   */
+  public abstract float[] vectorValue() throws IOException;
+
+  /**
+   * Return the binary encoded vector value for the current document ID. These are the bytes corresponding to the float array
+   * in IEEE 754 standard encoding, encoded using little-endian byte order.
+   * It is illegal to call this method when the iterator is not positioned: before advancing, or after failing to advance.
+   * The returned storage may be shared across calls, re-used and modified as the iterator advances.
+   * @return the binary value
+   */
+  public BytesRef binaryValue() throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  /**
+   * Return a random access interface over this iterator's vectors. Calling the RandomAccess methods will
+   * have no effect on the progress of the iteration or the values returned by this iterator. Successive calls
+   * will retrieve independent copies that do not overwrite each others' returned values.
+   */
+  public abstract RandomAccess randomAccess();
+
+  /**
+   * Provides random access to vectors by dense ordinal.
+   */
+  public interface RandomAccess {

Review comment:
       Also `@lucene.experimental`?

##########
File path: lucene/core/src/test/org/apache/lucene/index/TestVectorValues.java
##########
@@ -0,0 +1,650 @@
+/*
+ * 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.index;
+
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.document.VectorField;
+import org.apache.lucene.index.VectorValues.ScoreFunction;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.FSDirectory;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+/** Test Indexing/IndexWriter with vectors */
+public class TestVectorValues extends LuceneTestCase {
+
+  private IndexWriterConfig createIndexWriterConfig() {
+    IndexWriterConfig iwc = newIndexWriterConfig();
+    iwc.setCodec(Codec.forName("Lucene90"));
+    return iwc;
+  }
+
+  // Suddenly add vectors to an existing field:
+  public void testUpgradeFieldToVectors() throws Exception {
+    try (Directory dir = newDirectory()) {
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(newStringField("dim", "foo", Store.NO));
+        w.addDocument(doc);
+      }
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+        w.addDocument(doc);
+      }
+    }
+  }
+
+  public void testFieldConstructor() {
+    float[] v = new float[1];
+    VectorField field = new VectorField("f", v);
+    assertEquals(1, field.fieldType().vectorDimension());
+    assertEquals(ScoreFunction.EUCLIDEAN, field.fieldType().vectorScoreFunction());
+    assertSame(v, field.vectorValue());
+  }
+
+  public void testFieldConstructorExceptions() {
+    expectThrows(IllegalArgumentException.class, () -> new VectorField(null, new float[1]));
+    expectThrows(IllegalArgumentException.class, () -> new VectorField("f", null));
+    expectThrows(IllegalArgumentException.class, () -> new VectorField("f", new float[1], null));
+    expectThrows(IllegalArgumentException.class, () -> new VectorField("f", new float[0]));
+    expectThrows(IllegalArgumentException.class, () -> new VectorField("f", new float[VectorValues.MAX_DIMENSIONS + 1]));
+  }
+
+  public void testFieldSetValue() {
+    VectorField field = new VectorField("f", new float[1]);
+    float[] v1 = new float[1];
+    field.setVectorValue(v1);
+    assertSame(v1, field.vectorValue());
+    expectThrows(IllegalArgumentException.class, () -> field.setVectorValue(new float[2]));
+    expectThrows(NullPointerException.class, () -> field.setVectorValue(null));
+  }
+
+  // Illegal schema change tests:
+
+  public void testIllegalDimChangeTwoDocs() throws Exception {
+    try (Directory dir = newDirectory();
+         IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+      Document doc = new Document();
+      doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+      w.addDocument(doc);
+      if (random().nextBoolean()) {
+        // sometimes test with two segments
+        w.commit();
+      }
+
+      Document doc2 = new Document();
+      doc2.add(new VectorField("dim", new float[3], ScoreFunction.DOT_PRODUCT));
+      IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+          () -> w.addDocument(doc2));
+      assertEquals("cannot change vector dimension from 4 to 3 for field=\"dim\"", expected.getMessage());
+    }
+  }
+
+  public void testIllegalScoreFunctionChange() throws Exception {
+    try (Directory dir = newDirectory();
+         IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+      Document doc = new Document();
+      doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+      w.addDocument(doc);
+      if (random().nextBoolean()) {
+        // sometimes test with two segments
+        w.commit();
+      }
+
+      Document doc2 = new Document();
+      doc2.add(new VectorField("dim", new float[4], ScoreFunction.EUCLIDEAN));
+      IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+          () -> w.addDocument(doc2));
+      assertEquals("cannot change vector score function from DOT_PRODUCT to EUCLIDEAN for field=\"dim\"", expected.getMessage());
+    }
+  }
+
+  public void testIllegalDimChangeTwoWriters() throws Exception {
+    try (Directory dir = newDirectory()) {
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+        w.addDocument(doc);
+      }
+
+      try (IndexWriter w2 = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc2 = new Document();
+        doc2.add(new VectorField("dim", new float[1], ScoreFunction.DOT_PRODUCT));
+        IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+            () -> w2.addDocument(doc2));
+        assertEquals("cannot change vector dimension from 4 to 1 for field=\"dim\"", expected.getMessage());
+      }
+    }
+  }
+
+  public void testIllegalScoreFunctionChangeTwoWriters() throws Exception {
+    try (Directory dir = newDirectory()) {
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+        w.addDocument(doc);
+      }
+
+      try (IndexWriter w2 = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc2 = new Document();
+        doc2.add(new VectorField("dim", new float[4], ScoreFunction.EUCLIDEAN));
+        IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+            () -> w2.addDocument(doc2));
+        assertEquals("cannot change vector score function from DOT_PRODUCT to EUCLIDEAN for field=\"dim\"", expected.getMessage());
+      }
+    }
+  }
+
+  public void testIllegalDimChangeViaAddIndexesDirectory() throws Exception {
+    try (Directory dir = newDirectory();
+         Directory dir2 = newDirectory()) {
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+        w.addDocument(doc);
+      }
+      try (IndexWriter w2 = new IndexWriter(dir2, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[5], ScoreFunction.DOT_PRODUCT));
+        w2.addDocument(doc);
+        IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+            () -> w2.addIndexes(new Directory[]{dir}));
+        assertEquals("cannot change vector dimension from 5 to 4 for field=\"dim\"", expected.getMessage());
+      }
+    }
+  }
+
+  public void testIllegalScoreFunctionChangeViaAddIndexesDirectory() throws Exception {
+    try (Directory dir = newDirectory();
+         Directory dir2 = newDirectory()) {
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+        w.addDocument(doc);
+      }
+      try (IndexWriter w2 = new IndexWriter(dir2, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.EUCLIDEAN));
+        w2.addDocument(doc);
+        IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+            () -> w2.addIndexes(dir));
+        assertEquals("cannot change vector score function from EUCLIDEAN to DOT_PRODUCT for field=\"dim\"", expected.getMessage());
+      }
+    }
+  }
+
+  public void testIllegalDimChangeViaAddIndexesCodecReader() throws Exception {
+    try (Directory dir = newDirectory();
+         Directory dir2 = newDirectory()) {
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+        w.addDocument(doc);
+      }
+      try (IndexWriter w2 = new IndexWriter(dir2, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[5], ScoreFunction.DOT_PRODUCT));
+        w2.addDocument(doc);
+        try (DirectoryReader r = DirectoryReader.open(dir)) {
+          IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+              () -> w2.addIndexes(new CodecReader[]{(CodecReader) getOnlyLeafReader(r)}));
+          assertEquals("cannot change vector dimension from 5 to 4 for field=\"dim\"", expected.getMessage());
+        }
+      }
+    }
+  }
+
+  public void testIllegalScoreFunctionChangeViaAddIndexesCodecReader() throws Exception {
+    try (Directory dir = newDirectory();
+         Directory dir2 = newDirectory()) {
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+        w.addDocument(doc);
+      }
+      try (IndexWriter w2 = new IndexWriter(dir2, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.EUCLIDEAN));
+        w2.addDocument(doc);
+        try (DirectoryReader r = DirectoryReader.open(dir)) {
+          IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+              () -> w2.addIndexes(new CodecReader[]{(CodecReader) getOnlyLeafReader(r)}));
+          assertEquals("cannot change vector score function from EUCLIDEAN to DOT_PRODUCT for field=\"dim\"", expected.getMessage());
+        }
+      }
+    }
+  }
+
+  public void testIllegalDimChangeViaAddIndexesSlowCodecReader() throws Exception {
+    try (Directory dir = newDirectory();
+         Directory dir2 = newDirectory()) {
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+        w.addDocument(doc);
+      }
+      try (IndexWriter w2 = new IndexWriter(dir2, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[5], ScoreFunction.DOT_PRODUCT));
+        w2.addDocument(doc);
+        try (DirectoryReader r = DirectoryReader.open(dir)) {
+          IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+              () -> TestUtil.addIndexesSlowly(w2, r));
+          assertEquals("cannot change vector dimension from 5 to 4 for field=\"dim\"", expected.getMessage());
+        }
+      }
+    }
+  }
+
+  public void testIllegalScoreFunctionChangeViaAddIndexesSlowCodecReader() throws Exception {

Review comment:
       We have good coverage of illegal `addIndexes` usage, but do we have a test case of a successful usage, adding in an index with vectors to an index without, and vice/versa, and then both?

##########
File path: lucene/core/src/java/org/apache/lucene/index/VectorValues.java
##########
@@ -0,0 +1,273 @@
+/*
+ * 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.index;
+
+import java.io.IOException;
+
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * This class provides access to per-document floating point vector values indexed as {@link
+ * org.apache.lucene.document.VectorField}.
+ */
+public abstract class VectorValues extends DocIdSetIterator {
+
+  /** The maximum length of a vector */
+  public static int MAX_DIMENSIONS = 1024;
+
+  /** Sole constructor */
+  protected VectorValues() {}
+
+  /**
+   * Return the dimension of the vectors
+   */
+  public abstract int dimension();
+
+  /**
+   * TODO: should we use cost() for this? We rely on its always being exactly the number
+   * of documents having a value for this field, which is not guaranteed by the cost() contract,
+   * but in all the implementations so far they are the same.
+   * @return the number of vectors returned by this iterator
+   */
+  public abstract int size();
+
+  /**
+   * Return the score function used to compare these vectors
+   */
+  public abstract ScoreFunction scoreFunction();
+
+  /**
+   * Return the vector value for the current document ID.
+   * It is illegal to call this method when the iterator is not positioned: before advancing, or after failing to advance.
+   * The returned array may be shared across calls, re-used, and modified as the iterator advances.
+   * @return the vector value
+   */
+  public abstract float[] vectorValue() throws IOException;
+
+  /**
+   * Return the binary encoded vector value for the current document ID. These are the bytes corresponding to the float array
+   * in IEEE 754 standard encoding, encoded using little-endian byte order.
+   * It is illegal to call this method when the iterator is not positioned: before advancing, or after failing to advance.
+   * The returned storage may be shared across calls, re-used and modified as the iterator advances.
+   * @return the binary value
+   */
+  public BytesRef binaryValue() throws IOException {

Review comment:
       I wonder if we should reserve the right to change this binary format?  It feels like we are exposing implementation details of the underlying Codec storage?  E.g. what if Codec does lossy compression (one or two byte floats) in the future?

##########
File path: lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90VectorWriter.java
##########
@@ -0,0 +1,126 @@
+/*
+ * 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.lucene90;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.VectorWriter;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.index.VectorValues;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+/**
+ * Writes vector values and knn graphs to index segments.
+ */

Review comment:
       Can we mark as `@lucene.experimental`?

##########
File path: lucene/core/src/java/org/apache/lucene/index/VectorValues.java
##########
@@ -0,0 +1,273 @@
+/*
+ * 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.index;
+
+import java.io.IOException;
+
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * This class provides access to per-document floating point vector values indexed as {@link
+ * org.apache.lucene.document.VectorField}.
+ */
+public abstract class VectorValues extends DocIdSetIterator {
+
+  /** The maximum length of a vector */
+  public static int MAX_DIMENSIONS = 1024;
+
+  /** Sole constructor */
+  protected VectorValues() {}
+
+  /**
+   * Return the dimension of the vectors
+   */
+  public abstract int dimension();
+
+  /**
+   * TODO: should we use cost() for this? We rely on its always being exactly the number
+   * of documents having a value for this field, which is not guaranteed by the cost() contract,
+   * but in all the implementations so far they are the same.
+   * @return the number of vectors returned by this iterator
+   */
+  public abstract int size();
+
+  /**
+   * Return the score function used to compare these vectors
+   */
+  public abstract ScoreFunction scoreFunction();
+
+  /**
+   * Return the vector value for the current document ID.
+   * It is illegal to call this method when the iterator is not positioned: before advancing, or after failing to advance.
+   * The returned array may be shared across calls, re-used, and modified as the iterator advances.
+   * @return the vector value
+   */
+  public abstract float[] vectorValue() throws IOException;

Review comment:
       Hmm, is there an API somewhere to let me get the vector ordinal for the current Lucene `docId`?

##########
File path: lucene/core/src/java/org/apache/lucene/index/VectorValues.java
##########
@@ -0,0 +1,273 @@
+/*
+ * 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.index;
+
+import java.io.IOException;
+
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * This class provides access to per-document floating point vector values indexed as {@link
+ * org.apache.lucene.document.VectorField}.
+ */

Review comment:
       Can we mark as `@lucene.experimental`?

##########
File path: lucene/core/src/java/org/apache/lucene/index/VectorValuesWriter.java
##########
@@ -0,0 +1,322 @@
+/*
+ * 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.index;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.codecs.VectorWriter;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.Counter;
+import org.apache.lucene.util.RamUsageEstimator;
+
+/** Buffers up pending vector value(s) per doc, then flushes when segment flushes. */
+class VectorValuesWriter {
+
+  private final FieldInfo fieldInfo;
+  private final Counter iwBytesUsed;
+  private final List<float[]> vectors = new ArrayList<>();
+  private final DocsWithFieldSet docsWithField;
+
+  private int lastDocID = -1;
+
+  private long bytesUsed;
+
+  VectorValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
+    this.fieldInfo = fieldInfo;
+    this.iwBytesUsed = iwBytesUsed;
+    this.docsWithField = new DocsWithFieldSet();
+    this.bytesUsed = docsWithField.ramBytesUsed();
+    if (iwBytesUsed != null) {
+      iwBytesUsed.addAndGet(bytesUsed);
+    }
+  }
+
+  /**
+   * Adds a value for the given document. Only a single value may be added.
+   * @param docID the value is added to this document
+   * @param vectorValue the value to add
+   * @throws IllegalArgumentException if a value has already been added to the given document
+   */
+  public void addValue(int docID, float[] vectorValue) {
+    if (docID == lastDocID) {
+      throw new IllegalArgumentException("VectorValuesField \"" + fieldInfo.name + "\" appears more than once in this document (only one value is allowed per field)");
+    }
+    if (vectorValue.length != fieldInfo.getVectorDimension()) {
+      throw new IllegalArgumentException("Attempt to index a vector of dimension " + vectorValue.length +
+          " but \"" + fieldInfo.name + "\" has dimension " + fieldInfo.getVectorDimension());
+    }
+    assert docID > lastDocID;
+    docsWithField.add(docID);
+    vectors.add(ArrayUtil.copyOfSubArray(vectorValue, 0, vectorValue.length));
+    updateBytesUsed();
+    lastDocID = docID;
+  }
+
+  private void updateBytesUsed() {
+    final long newBytesUsed = docsWithField.ramBytesUsed()
+            + vectors.size() * (RamUsageEstimator.NUM_BYTES_OBJECT_REF + RamUsageEstimator.NUM_BYTES_ARRAY_HEADER)
+            + vectors.size() * vectors.get(0).length * Float.BYTES;
+    if (iwBytesUsed != null) {
+      iwBytesUsed.addAndGet(newBytesUsed - bytesUsed);
+    }
+    bytesUsed = newBytesUsed;
+  }
+
+  /**
+   * Flush this field's values to storage, sorting the values in accordance with sortMap
+   * @param sortMap specifies the order of documents being flushed, or null if they are to be flushed in docid order
+   * @param vectorWriter the Codec's vector writer that handles the actual encoding and I/O
+   * @throws IOException if there is an error writing the field and its values
+   */
+  public void flush(Sorter.DocMap sortMap, VectorWriter vectorWriter) throws IOException {
+    VectorValues vectorValues = new BufferedVectorValues(docsWithField, vectors, fieldInfo.getVectorDimension(), fieldInfo.getVectorScoreFunction());
+    if (sortMap != null) {
+      vectorWriter.writeField(fieldInfo, new SortingVectorValues(vectorValues, sortMap));
+    } else {
+      vectorWriter.writeField(fieldInfo, vectorValues);
+    }
+  }
+
+  private static class SortingVectorValues extends VectorValues {
+
+    private final VectorValues delegate;
+    private final VectorValues.RandomAccess randomAccess;
+    private final int[] docIdOffsets;
+    private final int[] ordMap;
+    private int docId = -1;
+
+    SortingVectorValues(VectorValues delegate, Sorter.DocMap sortMap) throws IOException {
+      this.delegate = delegate;
+      randomAccess = delegate.randomAccess();
+      docIdOffsets = new int[sortMap.size()];
+
+      int offset = 1; // 0 means no vector for this (field, document)
+      int docID;
+      while ((docID = delegate.nextDoc()) != NO_MORE_DOCS) {
+        int newDocID = sortMap.oldToNew(docID);
+        docIdOffsets[newDocID] = offset++;
+      }
+
+      // set up ordMap to map from new dense ordinal to old dense ordinal
+      ordMap = new int[offset - 1];
+      int ord = 0;
+      for (int docIdOffset : docIdOffsets) {
+        if (docIdOffset != 0) {
+          ordMap[ord++] = docIdOffset - 1;
+        }
+      }
+      assert ord == ordMap.length;
+    }
+
+    @Override
+    public int docID() {
+      return docId;
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      while (docId < docIdOffsets.length - 1) {
+        ++docId;
+        if (docIdOffsets[docId] != 0) {
+          return docId;
+        }
+      }
+      docId = NO_MORE_DOCS;
+      return docId;
+    }
+
+    @Override
+    public BytesRef binaryValue() throws IOException {
+      return randomAccess.binaryValue(docIdOffsets[docId] - 1);
+    }
+
+    @Override
+    public float[] vectorValue() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public int dimension() {
+      return delegate.dimension();
+    }
+
+    @Override
+    public int size() {
+      return delegate.size();
+    }
+
+    @Override
+    public ScoreFunction scoreFunction() {
+      return delegate.scoreFunction();
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public long cost() {
+      return size();
+    }
+
+    @Override
+    public RandomAccess randomAccess() {
+      RandomAccess ra = delegate.randomAccess();
+      return new RandomAccess() {
+
+        @Override
+        public int size() {
+          return delegate.size();
+        }
+
+        @Override
+        public int dimension() {
+          return delegate.dimension();
+        }
+
+        @Override
+        public ScoreFunction scoreFunction() {
+          return delegate.scoreFunction();
+        }
+
+        @Override
+        public float[] vectorValue(int targetOrd) throws IOException {
+          return ra.vectorValue(ordMap[targetOrd]);
+        }
+
+        @Override
+        public BytesRef binaryValue(int targetOrd) {
+          throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public TopDocs search(float[] target, int k, int fanout) {
+          throw new UnsupportedOperationException();
+        }
+      };
+    }
+  }
+
+  private static class BufferedVectorValues extends VectorValues implements VectorValues.RandomAccess {
+
+    final DocsWithFieldSet docsWithField;
+
+    // These are always the vectors of a VectorValuesWriter, which are copied when added to it
+    final List<float[]> vectors;
+    final VectorValues.ScoreFunction scoreFunction;
+    final int dimension;
+
+    final ByteBuffer buffer;
+    final BytesRef binaryValue;
+    final ByteBuffer raBuffer;
+    final BytesRef raBinaryValue;
+
+    DocIdSetIterator docsWithFieldIter;
+    int ord = -1;
+
+    BufferedVectorValues(DocsWithFieldSet docsWithField, List<float[]> vectorsArray, int dimension, VectorValues.ScoreFunction scoreFunction) {

Review comment:
       Hmm rename to `vectorsList` or just `vectors`?

##########
File path: lucene/core/src/java/org/apache/lucene/index/VectorValues.java
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.index;
+
+import java.io.IOException;
+
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Access to per-document vector value.
+ */
+public abstract class VectorValues extends DocIdSetIterator {
+
+  /** The maximum length of a vector */
+  public static int MAX_DIMENSIONS = 1024;
+
+  /** Sole constructor */
+  protected VectorValues() {}
+
+  /**
+   * Return the dimension of the vectors
+   */
+  public abstract int dimension();
+
+  /**
+   * TODO: should we use cost() for this? We rely on its always being exactly the number
+   * of documents having a value for this field, which is not guaranteed by the cost() contract,
+   * but in all the implementations so far they are the same.
+   * @return the number of vectors returned by this iterator
+   */
+  public abstract int size();
+
+  /**
+   * Return the score function used to compare these vectors
+   */
+  public abstract ScoreFunction scoreFunction();
+
+  /**
+   * Return the vector value for the current document ID.
+   * It is illegal to call this method after the iterator failed to advance.
+   * @return the vector value
+   */
+  public abstract float[] vectorValue() throws IOException;
+
+  /**
+   * Return the binary encoded vector value for the current document ID.
+   * It is illegal to call this method after the iterator failed to advance.
+   * @return the binary value
+   */
+  public BytesRef binaryValue() throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  /**
+   * Return a random access interface over this iterator's vectors.
+   */
+  public abstract RandomAccess randomAccess();
+
+  /**
+   * Provides random access to vectors by dense ordinal
+   */
+  public interface RandomAccess {
+
+    /**
+     * Return the vector value as a floating point array.
+     * @param targetOrd a valid ordinal, &ge; 0 and &lt; {@link #size()}.
+     */
+    float[] vectorValue(int targetOrd) throws IOException;
+
+    /**
+     * Return the vector value as a byte array; these are the bytes corresponding to the float array
+     * encoded using little-endian byte order.
+     * @param targetOrd a valid ordinal, &ge; 0 and &lt; {@link #size()}.
+     */
+    BytesRef binaryValue(int targetOrd) throws IOException;
+
+    /**
+     * Return the k nearest neighbor documents as determined by comparison of their vector values
+     * for this field, to the given vector, by the field's score function. If the score function is
+     * reversed, lower values indicate nearer vectors, otherwise higher scores indicate nearer
+     * vectors. Unlike relevance scores, vector scores may be negative.
+     * @param target the vector-valued query
+     * @param k      the number of docs to return
+     * @param fanout control the accuracy/speed tradeoff - larger values give better recall at higher cost
+     * @return the k nearest neighbor documents, along with their (scoreFunction-specific) scores.
+     */
+    TopDocs search(float[] target, int k, int fanout) throws IOException;
+  }
+
+  /**
+   * Score function. This is used during indexing and searching of the vectors to determine the nearest neighbors.
+   * Score values may be negative. By default high scores indicate nearer documents, unless the function is reversed.
+   */
+  public enum ScoreFunction {
+    /** No distance function is used. Note: {@link VectorValues.RandomAccess#search(float[], int, int)}

Review comment:
       Thanks, I added a comment.  Really we cannot/should not do that issue until just before the release, when we have the "final" vector format we intend to release as 9.0.0.

##########
File path: lucene/core/src/test/org/apache/lucene/index/TestVectorValues.java
##########
@@ -0,0 +1,650 @@
+/*
+ * 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.index;
+
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.document.VectorField;
+import org.apache.lucene.index.VectorValues.ScoreFunction;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.FSDirectory;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+/** Test Indexing/IndexWriter with vectors */
+public class TestVectorValues extends LuceneTestCase {
+
+  private IndexWriterConfig createIndexWriterConfig() {
+    IndexWriterConfig iwc = newIndexWriterConfig();
+    iwc.setCodec(Codec.forName("Lucene90"));
+    return iwc;
+  }
+
+  // Suddenly add vectors to an existing field:
+  public void testUpgradeFieldToVectors() throws Exception {
+    try (Directory dir = newDirectory()) {
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(newStringField("dim", "foo", Store.NO));
+        w.addDocument(doc);
+      }
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+        w.addDocument(doc);
+      }
+    }
+  }
+
+  public void testFieldConstructor() {
+    float[] v = new float[1];
+    VectorField field = new VectorField("f", v);
+    assertEquals(1, field.fieldType().vectorDimension());
+    assertEquals(ScoreFunction.EUCLIDEAN, field.fieldType().vectorScoreFunction());
+    assertSame(v, field.vectorValue());
+  }
+
+  public void testFieldConstructorExceptions() {
+    expectThrows(IllegalArgumentException.class, () -> new VectorField(null, new float[1]));
+    expectThrows(IllegalArgumentException.class, () -> new VectorField("f", null));
+    expectThrows(IllegalArgumentException.class, () -> new VectorField("f", new float[1], null));
+    expectThrows(IllegalArgumentException.class, () -> new VectorField("f", new float[0]));
+    expectThrows(IllegalArgumentException.class, () -> new VectorField("f", new float[VectorValues.MAX_DIMENSIONS + 1]));
+  }
+
+  public void testFieldSetValue() {
+    VectorField field = new VectorField("f", new float[1]);
+    float[] v1 = new float[1];
+    field.setVectorValue(v1);
+    assertSame(v1, field.vectorValue());
+    expectThrows(IllegalArgumentException.class, () -> field.setVectorValue(new float[2]));
+    expectThrows(NullPointerException.class, () -> field.setVectorValue(null));
+  }
+
+  // Illegal schema change tests:
+
+  public void testIllegalDimChangeTwoDocs() throws Exception {
+    try (Directory dir = newDirectory();
+         IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+      Document doc = new Document();
+      doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+      w.addDocument(doc);
+      if (random().nextBoolean()) {
+        // sometimes test with two segments
+        w.commit();
+      }
+
+      Document doc2 = new Document();
+      doc2.add(new VectorField("dim", new float[3], ScoreFunction.DOT_PRODUCT));
+      IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+          () -> w.addDocument(doc2));
+      assertEquals("cannot change vector dimension from 4 to 3 for field=\"dim\"", expected.getMessage());
+    }
+  }
+
+  public void testIllegalScoreFunctionChange() throws Exception {
+    try (Directory dir = newDirectory();
+         IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+      Document doc = new Document();
+      doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+      w.addDocument(doc);
+      if (random().nextBoolean()) {
+        // sometimes test with two segments
+        w.commit();
+      }
+
+      Document doc2 = new Document();
+      doc2.add(new VectorField("dim", new float[4], ScoreFunction.EUCLIDEAN));
+      IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+          () -> w.addDocument(doc2));
+      assertEquals("cannot change vector score function from DOT_PRODUCT to EUCLIDEAN for field=\"dim\"", expected.getMessage());
+    }
+  }
+
+  public void testIllegalDimChangeTwoWriters() throws Exception {
+    try (Directory dir = newDirectory()) {
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+        w.addDocument(doc);
+      }
+
+      try (IndexWriter w2 = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc2 = new Document();
+        doc2.add(new VectorField("dim", new float[1], ScoreFunction.DOT_PRODUCT));
+        IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+            () -> w2.addDocument(doc2));
+        assertEquals("cannot change vector dimension from 4 to 1 for field=\"dim\"", expected.getMessage());
+      }
+    }
+  }
+
+  public void testIllegalScoreFunctionChangeTwoWriters() throws Exception {
+    try (Directory dir = newDirectory()) {
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+        w.addDocument(doc);
+      }
+
+      try (IndexWriter w2 = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc2 = new Document();
+        doc2.add(new VectorField("dim", new float[4], ScoreFunction.EUCLIDEAN));
+        IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+            () -> w2.addDocument(doc2));
+        assertEquals("cannot change vector score function from DOT_PRODUCT to EUCLIDEAN for field=\"dim\"", expected.getMessage());
+      }
+    }
+  }
+
+  public void testIllegalDimChangeViaAddIndexesDirectory() throws Exception {
+    try (Directory dir = newDirectory();
+         Directory dir2 = newDirectory()) {
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+        w.addDocument(doc);
+      }
+      try (IndexWriter w2 = new IndexWriter(dir2, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[5], ScoreFunction.DOT_PRODUCT));
+        w2.addDocument(doc);
+        IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+            () -> w2.addIndexes(new Directory[]{dir}));
+        assertEquals("cannot change vector dimension from 5 to 4 for field=\"dim\"", expected.getMessage());
+      }
+    }
+  }
+
+  public void testIllegalScoreFunctionChangeViaAddIndexesDirectory() throws Exception {
+    try (Directory dir = newDirectory();
+         Directory dir2 = newDirectory()) {
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+        w.addDocument(doc);
+      }
+      try (IndexWriter w2 = new IndexWriter(dir2, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.EUCLIDEAN));
+        w2.addDocument(doc);
+        IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+            () -> w2.addIndexes(dir));
+        assertEquals("cannot change vector score function from EUCLIDEAN to DOT_PRODUCT for field=\"dim\"", expected.getMessage());
+      }
+    }
+  }
+
+  public void testIllegalDimChangeViaAddIndexesCodecReader() throws Exception {
+    try (Directory dir = newDirectory();
+         Directory dir2 = newDirectory()) {
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+        w.addDocument(doc);
+      }
+      try (IndexWriter w2 = new IndexWriter(dir2, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[5], ScoreFunction.DOT_PRODUCT));
+        w2.addDocument(doc);
+        try (DirectoryReader r = DirectoryReader.open(dir)) {
+          IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+              () -> w2.addIndexes(new CodecReader[]{(CodecReader) getOnlyLeafReader(r)}));
+          assertEquals("cannot change vector dimension from 5 to 4 for field=\"dim\"", expected.getMessage());
+        }
+      }
+    }
+  }
+
+  public void testIllegalScoreFunctionChangeViaAddIndexesCodecReader() throws Exception {
+    try (Directory dir = newDirectory();
+         Directory dir2 = newDirectory()) {
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+        w.addDocument(doc);
+      }
+      try (IndexWriter w2 = new IndexWriter(dir2, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.EUCLIDEAN));
+        w2.addDocument(doc);
+        try (DirectoryReader r = DirectoryReader.open(dir)) {
+          IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+              () -> w2.addIndexes(new CodecReader[]{(CodecReader) getOnlyLeafReader(r)}));
+          assertEquals("cannot change vector score function from EUCLIDEAN to DOT_PRODUCT for field=\"dim\"", expected.getMessage());
+        }
+      }
+    }
+  }
+
+  public void testIllegalDimChangeViaAddIndexesSlowCodecReader() throws Exception {
+    try (Directory dir = newDirectory();
+         Directory dir2 = newDirectory()) {
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+        w.addDocument(doc);
+      }
+      try (IndexWriter w2 = new IndexWriter(dir2, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[5], ScoreFunction.DOT_PRODUCT));
+        w2.addDocument(doc);
+        try (DirectoryReader r = DirectoryReader.open(dir)) {
+          IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+              () -> TestUtil.addIndexesSlowly(w2, r));
+          assertEquals("cannot change vector dimension from 5 to 4 for field=\"dim\"", expected.getMessage());
+        }
+      }
+    }
+  }
+
+  public void testIllegalScoreFunctionChangeViaAddIndexesSlowCodecReader() throws Exception {
+    try (Directory dir = newDirectory();
+         Directory dir2 = newDirectory()) {
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+        w.addDocument(doc);
+      }
+      try (IndexWriter w2 = new IndexWriter(dir2, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.EUCLIDEAN));
+        w2.addDocument(doc);
+        try (DirectoryReader r = DirectoryReader.open(dir)) {
+          IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+              () -> TestUtil.addIndexesSlowly(w2, r));
+          assertEquals("cannot change vector score function from EUCLIDEAN to DOT_PRODUCT for field=\"dim\"", expected.getMessage());
+        }
+      }
+    }
+  }
+
+  public void testIllegalMultipleValues() throws Exception {
+    try (Directory dir = newDirectory();
+         IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+      Document doc = new Document();
+      doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+      doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+      IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+          () -> w.addDocument(doc));
+      assertEquals("VectorValuesField \"dim\" appears more than once in this document (only one value is allowed per field)",
+          expected.getMessage());
+    }
+  }
+
+  public void testIllegalDimensionTooLarge() throws Exception {
+    try (Directory dir = newDirectory();
+         IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+      Document doc = new Document();
+      expectThrows(IllegalArgumentException.class,
+          () -> doc.add(new VectorField("dim", new float[VectorValues.MAX_DIMENSIONS + 1], ScoreFunction.DOT_PRODUCT)));
+
+      Document doc2 = new Document();
+      doc2.add(new VectorField("dim", new float[1], ScoreFunction.EUCLIDEAN));
+      w.addDocument(doc2);
+    }
+  }
+
+  public void testIllegalEmptyVector() throws Exception {

Review comment:
       Thank you for all the corner-case illegal tests!




----------------------------------------------------------------
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.

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


[GitHub] [lucene-solr] mikemccand commented on a change in pull request #1930: LUCENE-9322: add VectorValues to new Lucene90 codec

Posted by GitBox <gi...@apache.org>.
mikemccand commented on a change in pull request #1930:
URL: https://github.com/apache/lucene-solr/pull/1930#discussion_r499232347



##########
File path: .dir-locals.el
##########
@@ -0,0 +1,3 @@
+;; set up Lucene style for emacs
+((java-mode . ((c-basic-offset . 2))))

Review comment:
       Woot!

##########
File path: lucene/core/src/java/org/apache/lucene/codecs/VectorWriter.java
##########
@@ -0,0 +1,253 @@
+/*
+ * 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;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.lucene.index.DocIDMerger;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.MergeState;
+import org.apache.lucene.index.VectorValues;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.util.BytesRef;
+
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+/**
+ * Writes vectors to an index.
+ */
+public abstract class VectorWriter implements Closeable {
+
+  /** Sole constructor */
+  protected VectorWriter() {}
+
+  /** Write all values contained in the provided reader */
+  public abstract void writeField(FieldInfo fieldInfo, VectorValues values) throws IOException;
+
+  /** Called once at the end before close */
+  public abstract void finish() throws IOException;
+
+  /** Merge the vector values from multiple segments, for all fields */
+  public void merge(MergeState mergeState) throws IOException {
+    for (VectorReader reader : mergeState.vectorReaders) {
+      if (reader != null) {
+        reader.checkIntegrity();
+      }
+    }
+    for (FieldInfo fieldInfo : mergeState.mergeFieldInfos) {
+      if (fieldInfo.hasVectorValues()) {
+        mergeVectors(fieldInfo, mergeState);
+      }
+    }
+    finish();
+  }
+
+  private void mergeVectors(FieldInfo mergeFieldInfo, final MergeState mergeState) throws IOException {
+    if (mergeState.infoStream.isEnabled("VV")) {
+      mergeState.infoStream.message("VV", "merging " + mergeState.segmentInfo);
+    }
+    List<VectorValuesSub> subs = new ArrayList<>();
+    int dimension = -1;
+    VectorValues.ScoreFunction scoreFunction = null;
+    for (int i = 0; i < mergeState.vectorReaders.length; i++) {
+      VectorReader vectorReader = mergeState.vectorReaders[i];
+      if (vectorReader != null) {
+        if (mergeFieldInfo != null && mergeFieldInfo.hasVectorValues()) {
+          int segmentDimension = mergeFieldInfo.getVectorDimension();
+          VectorValues.ScoreFunction segmentScoreFunction = mergeFieldInfo.getVectorScoreFunction();
+          if (dimension == -1) {
+            dimension = segmentDimension;
+            scoreFunction = mergeFieldInfo.getVectorScoreFunction();
+          } else if (dimension != segmentDimension) {
+            throw new IllegalStateException("Varying dimensions for vector-valued field " + mergeFieldInfo.name
+                + ": " + dimension + "!=" + segmentDimension);
+          } else if (scoreFunction != segmentScoreFunction) {

Review comment:
       Does `IndexWriter` also catch if the user tries to change either dimension or score function on indexing a new document?

##########
File path: lucene/core/src/java/org/apache/lucene/index/VectorValuesWriter.java
##########
@@ -0,0 +1,272 @@
+/*
+ * 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.index;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.codecs.VectorWriter;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.Counter;
+
+/** Buffers up pending vector value(s) per doc, then flushes when segment flushes. */
+public class VectorValuesWriter {
+
+  private final FieldInfo fieldInfo;
+  private final Counter iwBytesUsed;
+  private final List<float[]> vectors = new ArrayList<>();
+  private final DocsWithFieldSet docsWithField;
+
+  private int lastDocID = -1;
+
+  private long bytesUsed;
+
+  VectorValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
+    this.fieldInfo = fieldInfo;
+    this.iwBytesUsed = iwBytesUsed;
+    this.docsWithField = new DocsWithFieldSet();
+    this.bytesUsed = docsWithField.ramBytesUsed();
+    if (iwBytesUsed != null) {
+      iwBytesUsed.addAndGet(bytesUsed);
+    }
+  }
+
+  /**
+   * Adds a value for the given document. Only a single value may be added.
+   * @param docID the value is added to this document
+   * @param vectorValue the value to add
+   * @throws IllegalArgumentException if a value has already been added to the given document
+   */
+  public void addValue(int docID, float[] vectorValue) {
+    if (docID == lastDocID) {
+      throw new IllegalArgumentException("VectorValuesField \"" + fieldInfo.name + "\" appears more than once in this document (only one value is allowed per field)");
+    }
+    assert docID > lastDocID;
+    docsWithField.add(docID);
+    vectors.add(ArrayUtil.copyOfSubArray(vectorValue, 0, vectorValue.length));

Review comment:
       Is something above us confirming `vectorValue.length` matches the length in `fieldInfo`?

##########
File path: lucene/core/src/java/org/apache/lucene/index/VectorValues.java
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.index;
+
+import java.io.IOException;
+
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Access to per-document vector value.
+ */
+public abstract class VectorValues extends DocIdSetIterator {
+
+  /** The maximum length of a vector */
+  public static int MAX_DIMENSIONS = 1024;
+
+  /** Sole constructor */
+  protected VectorValues() {}
+
+  /**
+   * Return the dimension of the vectors
+   */
+  public abstract int dimension();
+
+  /**
+   * TODO: should we use cost() for this? We rely on its always being exactly the number
+   * of documents having a value for this field, which is not guaranteed by the cost() contract,
+   * but in all the implementations so far they are the same.
+   * @return the number of vectors returned by this iterator
+   */
+  public abstract int size();
+
+  /**
+   * Return the score function used to compare these vectors
+   */
+  public abstract ScoreFunction scoreFunction();
+
+  /**
+   * Return the vector value for the current document ID.
+   * It is illegal to call this method after the iterator failed to advance.
+   * @return the vector value
+   */
+  public abstract float[] vectorValue() throws IOException;
+
+  /**
+   * Return the binary encoded vector value for the current document ID.
+   * It is illegal to call this method after the iterator failed to advance.
+   * @return the binary value
+   */
+  public BytesRef binaryValue() throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  /**
+   * Return a random access interface over this iterator's vectors.
+   */
+  public abstract RandomAccess randomAccess();
+
+  /**
+   * Provides random access to vectors by dense ordinal
+   */
+  public interface RandomAccess {
+
+    /**
+     * Return the vector value as a floating point array.
+     * @param targetOrd a valid ordinal, &ge; 0 and &lt; {@link #size()}.
+     */
+    float[] vectorValue(int targetOrd) throws IOException;
+
+    /**
+     * Return the vector value as a byte array; these are the bytes corresponding to the float array
+     * encoded using little-endian byte order.
+     * @param targetOrd a valid ordinal, &ge; 0 and &lt; {@link #size()}.
+     */
+    BytesRef binaryValue(int targetOrd) throws IOException;
+
+    /**
+     * Return the k nearest neighbor documents as determined by comparison of their vector values
+     * for this field, to the given vector, by the field's score function. If the score function is
+     * reversed, lower values indicate nearer vectors, otherwise higher scores indicate nearer
+     * vectors. Unlike relevance scores, vector scores may be negative.
+     * @param target the vector-valued query
+     * @param k      the number of docs to return
+     * @param fanout control the accuracy/speed tradeoff - larger values give better recall at higher cost
+     * @return the k nearest neighbor documents, along with their (scoreFunction-specific) scores.
+     */
+    TopDocs search(float[] target, int k, int fanout) throws IOException;
+  }
+
+  /**
+   * Score function. This is used during indexing and searching of the vectors to determine the nearest neighbors.
+   * Score values may be negative. By default high scores indicate nearer documents, unless the function is reversed.
+   */
+  public enum ScoreFunction {
+    /** No distance function is used. Note: {@link VectorValues.RandomAccess#search(float[], int, int)}
+     * is not supported for fields specifying this score function. */
+    NONE(0),
+
+    /** Euclidean distance */
+    EUCLIDEAN(1, true) {
+      @Override
+      public float score(float[] v1, float[] v2) {
+        assert v1.length == v2.length;
+        float squareSum = 0.0f;
+        int dim = v1.length;
+        for (int i = 0; i < dim; i++) {
+          float diff = v1[i] - v2[i];
+          squareSum += diff * diff;
+        }
+        return squareSum;
+      }
+    },
+
+    /** dot product - note, may be negative; larger values are better */
+    DOT_PRODUCT(2) {
+      @Override
+      public float score(float[] a, float[] b) {
+        float res = 0f;
+        /*
+         * If length of vector is larger than 8, we use unrolled dot product to accelerate the
+         * calculation.
+         */
+        int i;
+        for (i = 0; i < a.length % 8; i++) {
+            res += b[i] * a[i];
+        }
+        if (a.length < 8) {
+            return res;
+        }
+        float s0 = 0f;
+        float s1 = 0f;
+        float s2 = 0f;
+        float s3 = 0f;
+        float s4 = 0f;
+        float s5 = 0f;
+        float s6 = 0f;
+        float s7 = 0f;
+        for (; i + 7 < a.length; i += 8) {
+            s0 += b[i] * a[i];
+            s1 += b[i + 1] * a[i + 1];
+            s2 += b[i + 2] * a[i + 2];
+            s3 += b[i + 3] * a[i + 3];
+            s4 += b[i + 4] * a[i + 4];
+            s5 += b[i + 5] * a[i + 5];
+            s6 += b[i + 6] * a[i + 6];
+            s7 += b[i + 7] * a[i + 7];
+        }
+        res += s0 + s1 + s2 + s3 + s4 + s5 + s6 + s7;
+        return res;
+      }
+    };
+    
+    /** ID for each enum value; this is persisted to the index and cannot be changed after indexing. */
+    final public int id;
+
+    /** If reversed, smaller values are better */
+    final public boolean reversed;
+
+    ScoreFunction(int id, boolean reversed) {
+      this.id = id;
+      this.reversed = reversed;
+    }
+
+    ScoreFunction(int id) {
+      this(id, false);
+    }
+
+    /**
+     * Calculates the score between the specified two vectors.
+     */
+    public float score(float[] v1, float[] v2) {
+      throw new UnsupportedOperationException();
+    }
+
+    /**
+     * Returns the distance function that is specified by the id.
+     */
+    public static ScoreFunction fromId(int id) {
+      for (ScoreFunction d : ScoreFunction.values()) {
+        if (d.id == id) {
+          return d;
+        }
+      }
+      throw new IllegalArgumentException("no such distance function with id " + id);
+    }
+  }
+
+   /**
+   * Calculates a similarity score between the two vectors with specified function.
+   */
+  public static float compare(float[] v1, float[] v2, ScoreFunction scoreFunction) {
+    assert v1.length == v2.length : "attempt to compare vectors of lengths: " + v1.length + " " + v2.length;
+    return scoreFunction.score(v1, v2);
+  }
+
+  /**
+   * Represents the lack of vector values. It is returned by providers that do not
+   * support VectorValues.
+   */
+  public static VectorValues EMPTY = new VectorValues() {
+
+    @Override
+    public int size() {
+      return 0;
+    }
+
+    @Override
+    public int dimension() {
+      return 0;
+    }
+
+    @Override
+    public ScoreFunction scoreFunction() {
+      return ScoreFunction.NONE;
+    }
+
+    @Override
+    public float[] vectorValue() {
+      throw new IndexOutOfBoundsException("Attempt to get vectors from EMPTY values");
+    }
+
+    @Override
+    public RandomAccess randomAccess() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public int docID() {
+      return -1;

Review comment:
       `throw IllegalStateException` instead?

##########
File path: .gitignore
##########
@@ -37,3 +37,5 @@ build/
 
 # Ignore the generated local settings file.
 gradle.properties
+*~
+.#*

Review comment:
       Ha!  More echos of emacs ...

##########
File path: lucene/core/src/java/org/apache/lucene/codecs/VectorWriter.java
##########
@@ -0,0 +1,253 @@
+/*
+ * 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;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.lucene.index.DocIDMerger;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.MergeState;
+import org.apache.lucene.index.VectorValues;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.util.BytesRef;
+
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+/**
+ * Writes vectors to an index.
+ */
+public abstract class VectorWriter implements Closeable {
+
+  /** Sole constructor */
+  protected VectorWriter() {}
+
+  /** Write all values contained in the provided reader */
+  public abstract void writeField(FieldInfo fieldInfo, VectorValues values) throws IOException;
+
+  /** Called once at the end before close */
+  public abstract void finish() throws IOException;
+
+  /** Merge the vector values from multiple segments, for all fields */
+  public void merge(MergeState mergeState) throws IOException {
+    for (VectorReader reader : mergeState.vectorReaders) {
+      if (reader != null) {

Review comment:
       `reader` might be `null` when a given segment had no vectors, right?  Maybe `assert mergeState.fieldInfos[seg].hasVectorValues() == false` in that case?

##########
File path: lucene/codecs/src/test/org/apache/lucene/codecs/uniformsplit/sharedterms/TestSTBlockReader.java
##########
@@ -203,6 +204,8 @@ private static FieldInfo mockFieldInfo(String fieldName, int number) {
         0,
         0,
         0,
+        0,
+         VectorValues.ScoreFunction.NONE,

Review comment:
       Hmm fix indent?  It is indented one space too much?

##########
File path: lucene/core/src/java/org/apache/lucene/codecs/VectorWriter.java
##########
@@ -0,0 +1,253 @@
+/*
+ * 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;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.lucene.index.DocIDMerger;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.MergeState;
+import org.apache.lucene.index.VectorValues;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.util.BytesRef;
+
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+/**
+ * Writes vectors to an index.
+ */
+public abstract class VectorWriter implements Closeable {
+
+  /** Sole constructor */
+  protected VectorWriter() {}
+
+  /** Write all values contained in the provided reader */
+  public abstract void writeField(FieldInfo fieldInfo, VectorValues values) throws IOException;
+
+  /** Called once at the end before close */
+  public abstract void finish() throws IOException;
+
+  /** Merge the vector values from multiple segments, for all fields */
+  public void merge(MergeState mergeState) throws IOException {
+    for (VectorReader reader : mergeState.vectorReaders) {
+      if (reader != null) {
+        reader.checkIntegrity();
+      }
+    }
+    for (FieldInfo fieldInfo : mergeState.mergeFieldInfos) {
+      if (fieldInfo.hasVectorValues()) {
+        mergeVectors(fieldInfo, mergeState);
+      }
+    }
+    finish();
+  }
+
+  private void mergeVectors(FieldInfo mergeFieldInfo, final MergeState mergeState) throws IOException {
+    if (mergeState.infoStream.isEnabled("VV")) {
+      mergeState.infoStream.message("VV", "merging " + mergeState.segmentInfo);
+    }
+    List<VectorValuesSub> subs = new ArrayList<>();
+    int dimension = -1;
+    VectorValues.ScoreFunction scoreFunction = null;
+    for (int i = 0; i < mergeState.vectorReaders.length; i++) {
+      VectorReader vectorReader = mergeState.vectorReaders[i];
+      if (vectorReader != null) {
+        if (mergeFieldInfo != null && mergeFieldInfo.hasVectorValues()) {
+          int segmentDimension = mergeFieldInfo.getVectorDimension();
+          VectorValues.ScoreFunction segmentScoreFunction = mergeFieldInfo.getVectorScoreFunction();
+          if (dimension == -1) {
+            dimension = segmentDimension;
+            scoreFunction = mergeFieldInfo.getVectorScoreFunction();
+          } else if (dimension != segmentDimension) {
+            throw new IllegalStateException("Varying dimensions for vector-valued field " + mergeFieldInfo.name
+                + ": " + dimension + "!=" + segmentDimension);
+          } else if (scoreFunction != segmentScoreFunction) {
+            throw new IllegalStateException("Varying score functions for vector-valued field " + mergeFieldInfo.name
+                + ": " + scoreFunction + "!=" + segmentScoreFunction);
+          }
+          VectorValues values = vectorReader.getVectorValues(mergeFieldInfo.name);
+          if (values != null) {
+            subs.add(new VectorValuesSub(i, mergeState.docMaps[i], values));
+          }
+        }
+      }
+    }
+    // Create a new VectorValues by iterating over the sub vectors, mapping the resulting
+    // docids using docMaps in the mergeState.
+    writeField(mergeFieldInfo, new VectorValuesMerger(subs, mergeState));
+    if (mergeState.infoStream.isEnabled("VV")) {
+      mergeState.infoStream.message("VV", "merge done " + mergeState.segmentInfo);
+    }
+  }
+
+  /** Tracks state of one sub-reader that we are merging */
+  private static class VectorValuesSub extends DocIDMerger.Sub {
+
+    final MergeState.DocMap docMap;
+    final VectorValues values;
+    final int segmentIndex;
+    int count;
+
+    VectorValuesSub(int segmentIndex, MergeState.DocMap docMap, VectorValues values) {
+      super(docMap);
+      this.values = values;
+      this.segmentIndex = segmentIndex;
+      this.docMap = docMap;
+      assert values.docID() == -1;
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      int docId = values.nextDoc();
+      if (docId != NO_MORE_DOCS) {
+        // Note: this does count deleted docs  since they are present in the to-be-merged segment

Review comment:
       Extra space between `docs` and `since`?

##########
File path: lucene/core/src/java/org/apache/lucene/codecs/VectorFormat.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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;
+
+import java.io.IOException;
+
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.index.VectorValues;
+
+/**
+ * Encodes/decodes per-document vector and any associated indexing structures required to support nearest-neighbor search
+ */
+public abstract class VectorFormat {
+
+  /** Sole constructor */
+  protected VectorFormat() {}
+
+  /**
+   * Returns a {@link VectorWriter} to write the vectors to the index.
+   */
+  public abstract VectorWriter fieldsWriter(SegmentWriteState state) throws IOException;
+
+  /**
+   * Returns a {@link VectorReader} to read the vectors from the index.
+   */
+  public abstract VectorReader fieldsReader(SegmentReadState state) throws IOException;
+
+  /**
+   * EMPTY throws an exception when written. It acts as a sentinel indicating a Codec that does not support vectors.
+   */
+  public static VectorFormat EMPTY = new VectorFormat() {

Review comment:
       `final`?

##########
File path: lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextVectorReader.java
##########
@@ -0,0 +1,304 @@
+/*
+ * 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.simpletext;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.codecs.VectorReader;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.VectorValues;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.store.BufferedChecksumIndexInput;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.StringHelper;
+
+import static org.apache.lucene.codecs.simpletext.SimpleTextVectorWriter.*;
+
+/**
+ * Reads vector values from a simple text format. All vectors are read up front and cached in RAM in order to support
+ * random access.
+ * <b>FOR RECREATIONAL USE ONLY</b>
+ * @lucene.experimental
+ */
+public class SimpleTextVectorReader extends VectorReader {
+
+  private static final BytesRef EMPTY = new BytesRef("");
+
+  private final SegmentReadState readState;
+  private final IndexInput dataIn;
+  private final BytesRefBuilder scratch = new BytesRefBuilder();
+  private final Map<String, FieldEntry> fieldEntries = new HashMap<>();
+
+  SimpleTextVectorReader(SegmentReadState readState) throws IOException {
+    this.readState = readState;
+    String metaFileName = IndexFileNames.segmentFileName(readState.segmentInfo.name, readState.segmentSuffix, SimpleTextVectorFormat.META_EXTENSION);
+    try (ChecksumIndexInput in = readState.directory.openChecksumInput(metaFileName, IOContext.DEFAULT)) {
+      int fieldNumber = readInt(in, FIELD_NUMBER);
+      while (fieldNumber != -1) {
+        String fieldName = readString(in, FIELD_NAME);
+        String scoreFunctionName = readString(in, SCORE_FUNCTION);
+        VectorValues.ScoreFunction scoreFunction = VectorValues.ScoreFunction.valueOf(scoreFunctionName);
+        long vectorDataOffset = readLong(in, VECTOR_DATA_OFFSET);
+        long vectorDataLength = readLong(in, VECTOR_DATA_LENGTH);
+        int dimension = readInt(in, VECTOR_DIMENSION);
+        int size = readInt(in, SIZE);
+        int[] docIds = new int[size];
+        for (int i = 0; i < size; i++) {
+          docIds[i] = readInt(in, EMPTY);
+        }
+        assert fieldEntries.containsKey(fieldName) == false;
+        fieldEntries.put(fieldName, new FieldEntry(dimension, scoreFunction, vectorDataOffset, vectorDataLength, docIds));
+        fieldNumber = readInt(in, FIELD_NUMBER);
+      }
+      SimpleTextUtil.checkFooter(in);
+    }
+
+    String vectorFileName = IndexFileNames.segmentFileName(readState.segmentInfo.name, readState.segmentSuffix, SimpleTextVectorFormat.VECTOR_EXTENSION);
+    dataIn = readState.directory.openInput(vectorFileName, IOContext.DEFAULT);
+  }
+
+  @Override
+  public VectorValues getVectorValues(String field) throws IOException {
+    FieldInfo info = readState.fieldInfos.fieldInfo(field);
+    if (info == null) {
+      throw new IllegalStateException("No vectors indexed for field=\"" + field + "\"");
+    }
+    int dimension = info.getVectorDimension();
+    if (dimension == 0) {
+      return VectorValues.EMPTY;
+    }
+    FieldEntry fieldEntry = fieldEntries.get(field);
+    if (fieldEntry == null) {
+      throw new IllegalStateException("No entry found for vector field=\"" + field + "\"");
+    }
+    if (dimension != fieldEntry.dimension) {
+      throw new IllegalStateException("Inconsistent vector dimension for field=\"" + field + "\"; " + dimension + " != " + fieldEntry.dimension);
+    }
+    IndexInput bytesSlice = dataIn.slice("vector-data", fieldEntry.vectorDataOffset, fieldEntry.vectorDataLength);
+    return new SimpleTextVectorValues(fieldEntry, bytesSlice);
+  }
+
+  @Override
+  public void checkIntegrity() throws IOException {
+    IndexInput clone = dataIn.clone();
+    clone.seek(0);
+
+    // checksum is fixed-width encoded with 20 bytes, plus 1 byte for newline (the space is included in SimpleTextUtil.CHECKSUM):
+    long footerStartPos = dataIn.length() - (SimpleTextUtil.CHECKSUM.length + 21);
+    ChecksumIndexInput input = new BufferedChecksumIndexInput(clone);
+    while (true) {
+      SimpleTextUtil.readLine(input, scratch);
+      if (input.getFilePointer() >= footerStartPos) {
+        // Make sure we landed at precisely the right location:
+        if (input.getFilePointer() != footerStartPos) {
+          throw new CorruptIndexException("SimpleText failure: footer does not start at expected position current=" + input.getFilePointer() + " vs expected=" + footerStartPos, input);
+        }
+        SimpleTextUtil.checkFooter(input);
+        break;
+      }
+    }
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return 0;
+  }
+
+  @Override
+  public void close() throws IOException {
+    dataIn.close();
+  }
+
+  private static class FieldEntry {
+
+    final int dimension;
+    final VectorValues.ScoreFunction scoreFunction;
+
+    final long vectorDataOffset;
+    final long vectorDataLength;
+    final int[] ordToDoc;
+
+    FieldEntry(int dimension, VectorValues.ScoreFunction scoreFunction,
+               long vectorDataOffset, long vectorDataLength, int[] ordToDoc) {
+      this.dimension = dimension;
+      this.scoreFunction = scoreFunction;
+      this.vectorDataOffset = vectorDataOffset;
+      this.vectorDataLength = vectorDataLength;
+      this.ordToDoc = ordToDoc;
+    }
+
+    int size() {
+      return ordToDoc.length;
+    }
+  }
+
+  private static class SimpleTextVectorValues extends VectorValues implements VectorValues.RandomAccess {
+
+    private final BytesRefBuilder scratch = new BytesRefBuilder();
+    private final FieldEntry entry;
+    private final IndexInput in;
+    private final BytesRef binaryValue;
+    private final float[][] values;
+
+    int curOrd;
+
+    SimpleTextVectorValues(FieldEntry entry, IndexInput in) throws IOException {
+      this.entry = entry;
+      this.in = in;
+      values = new float[entry.size()][entry.dimension];
+      binaryValue = new BytesRef(entry.dimension * Float.BYTES);
+      binaryValue.length = binaryValue.bytes.length;
+      curOrd = -1;
+      readAllVectors();
+    }
+
+    @Override
+    public int dimension() {
+      return entry.dimension;
+    }
+
+    @Override
+    public int size() {
+      return entry.size();
+    }
+
+    @Override
+    public ScoreFunction scoreFunction() {
+      return entry.scoreFunction;
+    }
+
+    @Override
+    public float[] vectorValue() {
+      return values[curOrd];
+    }
+
+    @Override
+    public BytesRef binaryValue() {
+      ByteBuffer.wrap(binaryValue.bytes).asFloatBuffer().get(values[curOrd]);
+      return binaryValue;
+    }
+
+    @Override
+    public RandomAccess randomAccess() {
+      return this;
+    }
+
+    @Override
+    public int docID() {
+      if (curOrd == -1) {
+        return -1;
+      }
+      return entry.ordToDoc[curOrd];
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      if (++curOrd < entry.size()) {
+        return docID();
+      }
+      return NO_MORE_DOCS;
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      return slowAdvance(target);
+    }
+
+    @Override
+    public long cost() {
+      return size();
+    }
+
+    private void readAllVectors() throws IOException {

Review comment:
       `SimpleText` loads all vectors into RAM at index-open time, but this is for simplicity?  I.e. the default Codec will leave the vectors on disk?
   
   Edit: yes, default codec is a random-access off-heap implementation.

##########
File path: lucene/core/src/java/org/apache/lucene/index/VectorValues.java
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.index;
+
+import java.io.IOException;
+
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Access to per-document vector value.
+ */
+public abstract class VectorValues extends DocIdSetIterator {
+
+  /** The maximum length of a vector */
+  public static int MAX_DIMENSIONS = 1024;
+
+  /** Sole constructor */
+  protected VectorValues() {}
+
+  /**
+   * Return the dimension of the vectors
+   */
+  public abstract int dimension();
+
+  /**
+   * TODO: should we use cost() for this? We rely on its always being exactly the number
+   * of documents having a value for this field, which is not guaranteed by the cost() contract,
+   * but in all the implementations so far they are the same.
+   * @return the number of vectors returned by this iterator
+   */
+  public abstract int size();
+
+  /**
+   * Return the score function used to compare these vectors
+   */
+  public abstract ScoreFunction scoreFunction();
+
+  /**
+   * Return the vector value for the current document ID.
+   * It is illegal to call this method after the iterator failed to advance.

Review comment:
       ... or before the iterator was advanced?

##########
File path: lucene/core/src/java/org/apache/lucene/index/VectorValues.java
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.index;
+
+import java.io.IOException;
+
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Access to per-document vector value.
+ */
+public abstract class VectorValues extends DocIdSetIterator {
+
+  /** The maximum length of a vector */
+  public static int MAX_DIMENSIONS = 1024;
+
+  /** Sole constructor */
+  protected VectorValues() {}
+
+  /**
+   * Return the dimension of the vectors
+   */
+  public abstract int dimension();
+
+  /**
+   * TODO: should we use cost() for this? We rely on its always being exactly the number
+   * of documents having a value for this field, which is not guaranteed by the cost() contract,
+   * but in all the implementations so far they are the same.
+   * @return the number of vectors returned by this iterator
+   */
+  public abstract int size();
+
+  /**
+   * Return the score function used to compare these vectors
+   */
+  public abstract ScoreFunction scoreFunction();
+
+  /**
+   * Return the vector value for the current document ID.
+   * It is illegal to call this method after the iterator failed to advance.
+   * @return the vector value
+   */
+  public abstract float[] vectorValue() throws IOException;
+
+  /**
+   * Return the binary encoded vector value for the current document ID.
+   * It is illegal to call this method after the iterator failed to advance.

Review comment:
       ... or before the iterator was advanced?

##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -374,7 +377,25 @@ private FieldNormStatus() {
       /** Total number of fields with points. */
       public int totalValueFields;
       
-      /** Exception thrown during doc values test (null on success) */
+      /** Exception thrown during point values test (null on success) */

Review comment:
       Woops :)

##########
File path: lucene/core/src/java/org/apache/lucene/index/VectorValuesWriter.java
##########
@@ -0,0 +1,272 @@
+/*
+ * 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.index;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.codecs.VectorWriter;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.Counter;
+
+/** Buffers up pending vector value(s) per doc, then flushes when segment flushes. */
+public class VectorValuesWriter {
+
+  private final FieldInfo fieldInfo;
+  private final Counter iwBytesUsed;
+  private final List<float[]> vectors = new ArrayList<>();
+  private final DocsWithFieldSet docsWithField;
+
+  private int lastDocID = -1;
+
+  private long bytesUsed;
+
+  VectorValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
+    this.fieldInfo = fieldInfo;
+    this.iwBytesUsed = iwBytesUsed;
+    this.docsWithField = new DocsWithFieldSet();
+    this.bytesUsed = docsWithField.ramBytesUsed();
+    if (iwBytesUsed != null) {
+      iwBytesUsed.addAndGet(bytesUsed);
+    }
+  }
+
+  /**
+   * Adds a value for the given document. Only a single value may be added.
+   * @param docID the value is added to this document
+   * @param vectorValue the value to add
+   * @throws IllegalArgumentException if a value has already been added to the given document
+   */
+  public void addValue(int docID, float[] vectorValue) {
+    if (docID == lastDocID) {
+      throw new IllegalArgumentException("VectorValuesField \"" + fieldInfo.name + "\" appears more than once in this document (only one value is allowed per field)");
+    }
+    assert docID > lastDocID;
+    docsWithField.add(docID);
+    vectors.add(ArrayUtil.copyOfSubArray(vectorValue, 0, vectorValue.length));
+    updateBytesUsed();
+    lastDocID = docID;
+  }
+
+  private void updateBytesUsed() {
+    final long newBytesUsed = docsWithField.ramBytesUsed()
+            + vectors.size() * 5 // pointer plus array overhead for each array??
+            + vectors.size() * vectors.get(0).length * Float.BYTES;
+    if (iwBytesUsed != null) {
+      iwBytesUsed.addAndGet(newBytesUsed - bytesUsed);
+    }
+    bytesUsed = newBytesUsed;
+  }
+
+  /**
+   * Flush this field's values to storage, sorting the values in accordance with sortMap
+   * @param sortMap specifies the order of documents being flushed, or null if they are to be flushed in docid order
+   * @param vectorWriter the Codec's vector writer that handles the actual encoding and I/O
+   * @throws IOException if there is an error writing the field and its values
+   */
+  public void flush(Sorter.DocMap sortMap, VectorWriter vectorWriter) throws IOException {
+    VectorValues vectorValues = new BufferedVectorValues(docsWithField, vectors, fieldInfo.getVectorDimension(), fieldInfo.getVectorScoreFunction());
+    if (sortMap != null) {
+      vectorWriter.writeField(fieldInfo, new SortingVectorValues(vectorValues, sortMap));
+    } else {
+      vectorWriter.writeField(fieldInfo, vectorValues);
+    }
+  }
+
+  private static class SortingVectorValues extends VectorValues {
+
+    private final VectorValues delegate;
+    private final VectorValues.RandomAccess randomAccess;
+    private final int[] offsets;
+    private int docId = -1;
+
+    SortingVectorValues(VectorValues delegate, Sorter.DocMap sortMap) throws IOException {
+      this.delegate = delegate;
+      randomAccess = delegate.randomAccess();
+      offsets = new int[sortMap.size()];
+      int offset = 1; // 0 means no values for this document
+      int docID;
+      while ((docID = delegate.nextDoc()) != NO_MORE_DOCS) {
+        int newDocID = sortMap.oldToNew(docID);
+        offsets[newDocID] = offset++;
+      }
+    }
+
+    @Override
+    public int docID() {
+      return docId;
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      while (docId < offsets.length - 1) {
+        ++docId;
+        if (offsets[docId] != 0) {
+          return docId;
+        }
+      }
+      docId = NO_MORE_DOCS;
+      return docId;
+    }
+
+    @Override
+    public BytesRef binaryValue() throws IOException {
+      int oldOffset = offsets[docId] - 1;
+      return randomAccess.binaryValue(oldOffset);
+    }
+
+    @Override
+    public float[] vectorValue() {
+      throw new UnsupportedOperationException();

Review comment:
       Hmm, we are able to do this because the `binaryValue` does work, and our caller here is writing binary values so it is using `binaryValues`?

##########
File path: lucene/core/src/java/org/apache/lucene/index/VectorValues.java
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.index;
+
+import java.io.IOException;
+
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Access to per-document vector value.
+ */
+public abstract class VectorValues extends DocIdSetIterator {
+
+  /** The maximum length of a vector */
+  public static int MAX_DIMENSIONS = 1024;
+
+  /** Sole constructor */
+  protected VectorValues() {}
+
+  /**
+   * Return the dimension of the vectors
+   */
+  public abstract int dimension();
+
+  /**
+   * TODO: should we use cost() for this? We rely on its always being exactly the number
+   * of documents having a value for this field, which is not guaranteed by the cost() contract,
+   * but in all the implementations so far they are the same.
+   * @return the number of vectors returned by this iterator
+   */
+  public abstract int size();
+
+  /**
+   * Return the score function used to compare these vectors
+   */
+  public abstract ScoreFunction scoreFunction();
+
+  /**
+   * Return the vector value for the current document ID.
+   * It is illegal to call this method after the iterator failed to advance.
+   * @return the vector value
+   */
+  public abstract float[] vectorValue() throws IOException;
+
+  /**
+   * Return the binary encoded vector value for the current document ID.
+   * It is illegal to call this method after the iterator failed to advance.
+   * @return the binary value
+   */
+  public BytesRef binaryValue() throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  /**
+   * Return a random access interface over this iterator's vectors.
+   */
+  public abstract RandomAccess randomAccess();
+
+  /**
+   * Provides random access to vectors by dense ordinal
+   */
+  public interface RandomAccess {
+
+    /**
+     * Return the vector value as a floating point array.
+     * @param targetOrd a valid ordinal, &ge; 0 and &lt; {@link #size()}.
+     */
+    float[] vectorValue(int targetOrd) throws IOException;
+
+    /**
+     * Return the vector value as a byte array; these are the bytes corresponding to the float array
+     * encoded using little-endian byte order.
+     * @param targetOrd a valid ordinal, &ge; 0 and &lt; {@link #size()}.
+     */
+    BytesRef binaryValue(int targetOrd) throws IOException;
+
+    /**
+     * Return the k nearest neighbor documents as determined by comparison of their vector values
+     * for this field, to the given vector, by the field's score function. If the score function is
+     * reversed, lower values indicate nearer vectors, otherwise higher scores indicate nearer
+     * vectors. Unlike relevance scores, vector scores may be negative.
+     * @param target the vector-valued query
+     * @param k      the number of docs to return
+     * @param fanout control the accuracy/speed tradeoff - larger values give better recall at higher cost
+     * @return the k nearest neighbor documents, along with their (scoreFunction-specific) scores.
+     */
+    TopDocs search(float[] target, int k, int fanout) throws IOException;
+  }
+
+  /**
+   * Score function. This is used during indexing and searching of the vectors to determine the nearest neighbors.
+   * Score values may be negative. By default high scores indicate nearer documents, unless the function is reversed.
+   */
+  public enum ScoreFunction {
+    /** No distance function is used. Note: {@link VectorValues.RandomAccess#search(float[], int, int)}
+     * is not supported for fields specifying this score function. */
+    NONE(0),
+
+    /** Euclidean distance */
+    EUCLIDEAN(1, true) {
+      @Override
+      public float score(float[] v1, float[] v2) {
+        assert v1.length == v2.length;
+        float squareSum = 0.0f;
+        int dim = v1.length;
+        for (int i = 0; i < dim; i++) {
+          float diff = v1[i] - v2[i];
+          squareSum += diff * diff;
+        }
+        return squareSum;
+      }
+    },
+
+    /** dot product - note, may be negative; larger values are better */
+    DOT_PRODUCT(2) {
+      @Override
+      public float score(float[] a, float[] b) {
+        float res = 0f;
+        /*
+         * If length of vector is larger than 8, we use unrolled dot product to accelerate the
+         * calculation.
+         */
+        int i;
+        for (i = 0; i < a.length % 8; i++) {
+            res += b[i] * a[i];
+        }
+        if (a.length < 8) {
+            return res;
+        }
+        float s0 = 0f;
+        float s1 = 0f;
+        float s2 = 0f;
+        float s3 = 0f;
+        float s4 = 0f;
+        float s5 = 0f;
+        float s6 = 0f;
+        float s7 = 0f;
+        for (; i + 7 < a.length; i += 8) {
+            s0 += b[i] * a[i];
+            s1 += b[i + 1] * a[i + 1];
+            s2 += b[i + 2] * a[i + 2];
+            s3 += b[i + 3] * a[i + 3];
+            s4 += b[i + 4] * a[i + 4];
+            s5 += b[i + 5] * a[i + 5];
+            s6 += b[i + 6] * a[i + 6];
+            s7 += b[i + 7] * a[i + 7];
+        }
+        res += s0 + s1 + s2 + s3 + s4 + s5 + s6 + s7;
+        return res;
+      }
+    };
+    
+    /** ID for each enum value; this is persisted to the index and cannot be changed after indexing. */
+    final public int id;
+
+    /** If reversed, smaller values are better */
+    final public boolean reversed;
+
+    ScoreFunction(int id, boolean reversed) {
+      this.id = id;
+      this.reversed = reversed;
+    }
+
+    ScoreFunction(int id) {
+      this(id, false);
+    }
+
+    /**
+     * Calculates the score between the specified two vectors.
+     */
+    public float score(float[] v1, float[] v2) {
+      throw new UnsupportedOperationException();
+    }
+
+    /**
+     * Returns the distance function that is specified by the id.
+     */
+    public static ScoreFunction fromId(int id) {
+      for (ScoreFunction d : ScoreFunction.values()) {
+        if (d.id == id) {
+          return d;
+        }
+      }
+      throw new IllegalArgumentException("no such distance function with id " + id);
+    }
+  }
+
+   /**
+   * Calculates a similarity score between the two vectors with specified function.
+   */
+  public static float compare(float[] v1, float[] v2, ScoreFunction scoreFunction) {
+    assert v1.length == v2.length : "attempt to compare vectors of lengths: " + v1.length + " " + v2.length;
+    return scoreFunction.score(v1, v2);
+  }
+
+  /**
+   * Represents the lack of vector values. It is returned by providers that do not
+   * support VectorValues.
+   */
+  public static VectorValues EMPTY = new VectorValues() {
+
+    @Override
+    public int size() {
+      return 0;
+    }
+
+    @Override
+    public int dimension() {
+      return 0;
+    }
+
+    @Override
+    public ScoreFunction scoreFunction() {
+      return ScoreFunction.NONE;
+    }
+
+    @Override
+    public float[] vectorValue() {
+      throw new IndexOutOfBoundsException("Attempt to get vectors from EMPTY values");
+    }
+
+    @Override
+    public RandomAccess randomAccess() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public int docID() {
+      return -1;
+    }
+
+    @Override
+    public int nextDoc() {
+      return -1;

Review comment:
       Hmm `return NO_MORE_DOCS`?

##########
File path: lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java
##########
@@ -233,15 +194,51 @@ private int mergeFields() throws IOException {
    * Merge the TermVectors from each of the segments into the new one.
    * @throws IOException if there is a low-level IO error
    */
-  private int mergeVectors() throws IOException {
+  private int mergeTermVectors() throws IOException {

Review comment:
       Ha!

##########
File path: lucene/core/src/java/org/apache/lucene/index/VectorValues.java
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.index;
+
+import java.io.IOException;
+
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Access to per-document vector value.
+ */
+public abstract class VectorValues extends DocIdSetIterator {
+
+  /** The maximum length of a vector */
+  public static int MAX_DIMENSIONS = 1024;
+
+  /** Sole constructor */
+  protected VectorValues() {}
+
+  /**
+   * Return the dimension of the vectors
+   */
+  public abstract int dimension();
+
+  /**
+   * TODO: should we use cost() for this? We rely on its always being exactly the number
+   * of documents having a value for this field, which is not guaranteed by the cost() contract,
+   * but in all the implementations so far they are the same.
+   * @return the number of vectors returned by this iterator
+   */
+  public abstract int size();
+
+  /**
+   * Return the score function used to compare these vectors
+   */
+  public abstract ScoreFunction scoreFunction();
+
+  /**
+   * Return the vector value for the current document ID.
+   * It is illegal to call this method after the iterator failed to advance.
+   * @return the vector value
+   */
+  public abstract float[] vectorValue() throws IOException;
+
+  /**
+   * Return the binary encoded vector value for the current document ID.
+   * It is illegal to call this method after the iterator failed to advance.
+   * @return the binary value
+   */
+  public BytesRef binaryValue() throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  /**
+   * Return a random access interface over this iterator's vectors.
+   */
+  public abstract RandomAccess randomAccess();
+
+  /**
+   * Provides random access to vectors by dense ordinal
+   */
+  public interface RandomAccess {
+
+    /**
+     * Return the vector value as a floating point array.
+     * @param targetOrd a valid ordinal, &ge; 0 and &lt; {@link #size()}.
+     */
+    float[] vectorValue(int targetOrd) throws IOException;
+
+    /**
+     * Return the vector value as a byte array; these are the bytes corresponding to the float array
+     * encoded using little-endian byte order.
+     * @param targetOrd a valid ordinal, &ge; 0 and &lt; {@link #size()}.
+     */
+    BytesRef binaryValue(int targetOrd) throws IOException;
+
+    /**
+     * Return the k nearest neighbor documents as determined by comparison of their vector values
+     * for this field, to the given vector, by the field's score function. If the score function is
+     * reversed, lower values indicate nearer vectors, otherwise higher scores indicate nearer
+     * vectors. Unlike relevance scores, vector scores may be negative.
+     * @param target the vector-valued query
+     * @param k      the number of docs to return
+     * @param fanout control the accuracy/speed tradeoff - larger values give better recall at higher cost
+     * @return the k nearest neighbor documents, along with their (scoreFunction-specific) scores.
+     */
+    TopDocs search(float[] target, int k, int fanout) throws IOException;
+  }
+
+  /**
+   * Score function. This is used during indexing and searching of the vectors to determine the nearest neighbors.
+   * Score values may be negative. By default high scores indicate nearer documents, unless the function is reversed.
+   */
+  public enum ScoreFunction {
+    /** No distance function is used. Note: {@link VectorValues.RandomAccess#search(float[], int, int)}
+     * is not supported for fields specifying this score function. */
+    NONE(0),
+
+    /** Euclidean distance */
+    EUCLIDEAN(1, true) {
+      @Override
+      public float score(float[] v1, float[] v2) {
+        assert v1.length == v2.length;
+        float squareSum = 0.0f;
+        int dim = v1.length;
+        for (int i = 0; i < dim; i++) {
+          float diff = v1[i] - v2[i];
+          squareSum += diff * diff;
+        }
+        return squareSum;
+      }
+    },
+
+    /** dot product - note, may be negative; larger values are better */
+    DOT_PRODUCT(2) {
+      @Override
+      public float score(float[] a, float[] b) {
+        float res = 0f;
+        /*
+         * If length of vector is larger than 8, we use unrolled dot product to accelerate the
+         * calculation.
+         */
+        int i;
+        for (i = 0; i < a.length % 8; i++) {
+            res += b[i] * a[i];
+        }
+        if (a.length < 8) {
+            return res;
+        }
+        float s0 = 0f;
+        float s1 = 0f;
+        float s2 = 0f;
+        float s3 = 0f;
+        float s4 = 0f;
+        float s5 = 0f;
+        float s6 = 0f;
+        float s7 = 0f;
+        for (; i + 7 < a.length; i += 8) {
+            s0 += b[i] * a[i];
+            s1 += b[i + 1] * a[i + 1];
+            s2 += b[i + 2] * a[i + 2];
+            s3 += b[i + 3] * a[i + 3];
+            s4 += b[i + 4] * a[i + 4];
+            s5 += b[i + 5] * a[i + 5];
+            s6 += b[i + 6] * a[i + 6];
+            s7 += b[i + 7] * a[i + 7];
+        }
+        res += s0 + s1 + s2 + s3 + s4 + s5 + s6 + s7;
+        return res;
+      }
+    };
+    
+    /** ID for each enum value; this is persisted to the index and cannot be changed after indexing. */
+    final public int id;

Review comment:
       Hmm instead of our own `id`, should we just use `ordinal`, which `enum` already supports?  It would mean we must not change the order of `enum` values once released...

##########
File path: lucene/core/src/java/org/apache/lucene/index/VectorValuesWriter.java
##########
@@ -0,0 +1,272 @@
+/*
+ * 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.index;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.codecs.VectorWriter;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.Counter;
+
+/** Buffers up pending vector value(s) per doc, then flushes when segment flushes. */
+public class VectorValuesWriter {
+
+  private final FieldInfo fieldInfo;
+  private final Counter iwBytesUsed;
+  private final List<float[]> vectors = new ArrayList<>();
+  private final DocsWithFieldSet docsWithField;
+
+  private int lastDocID = -1;
+
+  private long bytesUsed;
+
+  VectorValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
+    this.fieldInfo = fieldInfo;
+    this.iwBytesUsed = iwBytesUsed;
+    this.docsWithField = new DocsWithFieldSet();
+    this.bytesUsed = docsWithField.ramBytesUsed();
+    if (iwBytesUsed != null) {
+      iwBytesUsed.addAndGet(bytesUsed);
+    }
+  }
+
+  /**
+   * Adds a value for the given document. Only a single value may be added.
+   * @param docID the value is added to this document
+   * @param vectorValue the value to add
+   * @throws IllegalArgumentException if a value has already been added to the given document
+   */
+  public void addValue(int docID, float[] vectorValue) {
+    if (docID == lastDocID) {
+      throw new IllegalArgumentException("VectorValuesField \"" + fieldInfo.name + "\" appears more than once in this document (only one value is allowed per field)");
+    }
+    assert docID > lastDocID;
+    docsWithField.add(docID);
+    vectors.add(ArrayUtil.copyOfSubArray(vectorValue, 0, vectorValue.length));
+    updateBytesUsed();
+    lastDocID = docID;
+  }
+
+  private void updateBytesUsed() {
+    final long newBytesUsed = docsWithField.ramBytesUsed()
+            + vectors.size() * 5 // pointer plus array overhead for each array??
+            + vectors.size() * vectors.get(0).length * Float.BYTES;
+    if (iwBytesUsed != null) {
+      iwBytesUsed.addAndGet(newBytesUsed - bytesUsed);
+    }
+    bytesUsed = newBytesUsed;
+  }
+
+  /**
+   * Flush this field's values to storage, sorting the values in accordance with sortMap
+   * @param sortMap specifies the order of documents being flushed, or null if they are to be flushed in docid order
+   * @param vectorWriter the Codec's vector writer that handles the actual encoding and I/O
+   * @throws IOException if there is an error writing the field and its values
+   */
+  public void flush(Sorter.DocMap sortMap, VectorWriter vectorWriter) throws IOException {
+    VectorValues vectorValues = new BufferedVectorValues(docsWithField, vectors, fieldInfo.getVectorDimension(), fieldInfo.getVectorScoreFunction());
+    if (sortMap != null) {
+      vectorWriter.writeField(fieldInfo, new SortingVectorValues(vectorValues, sortMap));
+    } else {
+      vectorWriter.writeField(fieldInfo, vectorValues);
+    }
+  }
+
+  private static class SortingVectorValues extends VectorValues {
+
+    private final VectorValues delegate;
+    private final VectorValues.RandomAccess randomAccess;
+    private final int[] offsets;
+    private int docId = -1;
+
+    SortingVectorValues(VectorValues delegate, Sorter.DocMap sortMap) throws IOException {
+      this.delegate = delegate;
+      randomAccess = delegate.randomAccess();
+      offsets = new int[sortMap.size()];
+      int offset = 1; // 0 means no values for this document

Review comment:
       Maybe `... means no vector for this field x document`?  (`values` is confusing because it seems to imply there could be more than one value per field x document).

##########
File path: lucene/core/src/java/org/apache/lucene/index/VectorValuesWriter.java
##########
@@ -0,0 +1,272 @@
+/*
+ * 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.index;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.codecs.VectorWriter;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.Counter;
+
+/** Buffers up pending vector value(s) per doc, then flushes when segment flushes. */
+public class VectorValuesWriter {

Review comment:
       Hmm why does this need to be `public`?  It is a utility class for `IndexWriter` to privately use to write vectors when flushing a new segment?

##########
File path: lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextVectorReader.java
##########
@@ -0,0 +1,304 @@
+/*
+ * 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.simpletext;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.codecs.VectorReader;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.VectorValues;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.store.BufferedChecksumIndexInput;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.StringHelper;
+
+import static org.apache.lucene.codecs.simpletext.SimpleTextVectorWriter.*;
+
+/**
+ * Reads vector values from a simple text format. All vectors are read up front and cached in RAM in order to support
+ * random access.
+ * <b>FOR RECREATIONAL USE ONLY</b>
+ * @lucene.experimental
+ */
+public class SimpleTextVectorReader extends VectorReader {
+
+  private static final BytesRef EMPTY = new BytesRef("");
+
+  private final SegmentReadState readState;
+  private final IndexInput dataIn;
+  private final BytesRefBuilder scratch = new BytesRefBuilder();
+  private final Map<String, FieldEntry> fieldEntries = new HashMap<>();
+
+  SimpleTextVectorReader(SegmentReadState readState) throws IOException {
+    this.readState = readState;
+    String metaFileName = IndexFileNames.segmentFileName(readState.segmentInfo.name, readState.segmentSuffix, SimpleTextVectorFormat.META_EXTENSION);
+    try (ChecksumIndexInput in = readState.directory.openChecksumInput(metaFileName, IOContext.DEFAULT)) {
+      int fieldNumber = readInt(in, FIELD_NUMBER);
+      while (fieldNumber != -1) {
+        String fieldName = readString(in, FIELD_NAME);
+        String scoreFunctionName = readString(in, SCORE_FUNCTION);
+        VectorValues.ScoreFunction scoreFunction = VectorValues.ScoreFunction.valueOf(scoreFunctionName);
+        long vectorDataOffset = readLong(in, VECTOR_DATA_OFFSET);
+        long vectorDataLength = readLong(in, VECTOR_DATA_LENGTH);
+        int dimension = readInt(in, VECTOR_DIMENSION);
+        int size = readInt(in, SIZE);
+        int[] docIds = new int[size];
+        for (int i = 0; i < size; i++) {
+          docIds[i] = readInt(in, EMPTY);
+        }
+        assert fieldEntries.containsKey(fieldName) == false;
+        fieldEntries.put(fieldName, new FieldEntry(dimension, scoreFunction, vectorDataOffset, vectorDataLength, docIds));
+        fieldNumber = readInt(in, FIELD_NUMBER);
+      }
+      SimpleTextUtil.checkFooter(in);
+    }
+
+    String vectorFileName = IndexFileNames.segmentFileName(readState.segmentInfo.name, readState.segmentSuffix, SimpleTextVectorFormat.VECTOR_EXTENSION);
+    dataIn = readState.directory.openInput(vectorFileName, IOContext.DEFAULT);
+  }
+
+  @Override
+  public VectorValues getVectorValues(String field) throws IOException {
+    FieldInfo info = readState.fieldInfos.fieldInfo(field);
+    if (info == null) {
+      throw new IllegalStateException("No vectors indexed for field=\"" + field + "\"");
+    }
+    int dimension = info.getVectorDimension();
+    if (dimension == 0) {
+      return VectorValues.EMPTY;
+    }
+    FieldEntry fieldEntry = fieldEntries.get(field);
+    if (fieldEntry == null) {
+      throw new IllegalStateException("No entry found for vector field=\"" + field + "\"");
+    }
+    if (dimension != fieldEntry.dimension) {
+      throw new IllegalStateException("Inconsistent vector dimension for field=\"" + field + "\"; " + dimension + " != " + fieldEntry.dimension);
+    }
+    IndexInput bytesSlice = dataIn.slice("vector-data", fieldEntry.vectorDataOffset, fieldEntry.vectorDataLength);
+    return new SimpleTextVectorValues(fieldEntry, bytesSlice);
+  }
+
+  @Override
+  public void checkIntegrity() throws IOException {
+    IndexInput clone = dataIn.clone();
+    clone.seek(0);
+
+    // checksum is fixed-width encoded with 20 bytes, plus 1 byte for newline (the space is included in SimpleTextUtil.CHECKSUM):
+    long footerStartPos = dataIn.length() - (SimpleTextUtil.CHECKSUM.length + 21);
+    ChecksumIndexInput input = new BufferedChecksumIndexInput(clone);
+    while (true) {
+      SimpleTextUtil.readLine(input, scratch);
+      if (input.getFilePointer() >= footerStartPos) {
+        // Make sure we landed at precisely the right location:
+        if (input.getFilePointer() != footerStartPos) {
+          throw new CorruptIndexException("SimpleText failure: footer does not start at expected position current=" + input.getFilePointer() + " vs expected=" + footerStartPos, input);
+        }
+        SimpleTextUtil.checkFooter(input);
+        break;
+      }
+    }
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return 0;
+  }
+
+  @Override
+  public void close() throws IOException {
+    dataIn.close();
+  }
+
+  private static class FieldEntry {
+
+    final int dimension;
+    final VectorValues.ScoreFunction scoreFunction;
+
+    final long vectorDataOffset;
+    final long vectorDataLength;
+    final int[] ordToDoc;
+
+    FieldEntry(int dimension, VectorValues.ScoreFunction scoreFunction,
+               long vectorDataOffset, long vectorDataLength, int[] ordToDoc) {
+      this.dimension = dimension;
+      this.scoreFunction = scoreFunction;
+      this.vectorDataOffset = vectorDataOffset;
+      this.vectorDataLength = vectorDataLength;
+      this.ordToDoc = ordToDoc;
+    }
+
+    int size() {
+      return ordToDoc.length;
+    }
+  }
+
+  private static class SimpleTextVectorValues extends VectorValues implements VectorValues.RandomAccess {
+
+    private final BytesRefBuilder scratch = new BytesRefBuilder();
+    private final FieldEntry entry;
+    private final IndexInput in;
+    private final BytesRef binaryValue;
+    private final float[][] values;
+
+    int curOrd;
+
+    SimpleTextVectorValues(FieldEntry entry, IndexInput in) throws IOException {
+      this.entry = entry;
+      this.in = in;
+      values = new float[entry.size()][entry.dimension];
+      binaryValue = new BytesRef(entry.dimension * Float.BYTES);
+      binaryValue.length = binaryValue.bytes.length;
+      curOrd = -1;
+      readAllVectors();
+    }
+
+    @Override
+    public int dimension() {
+      return entry.dimension;
+    }
+
+    @Override
+    public int size() {
+      return entry.size();
+    }
+
+    @Override
+    public ScoreFunction scoreFunction() {
+      return entry.scoreFunction;
+    }
+
+    @Override
+    public float[] vectorValue() {
+      return values[curOrd];
+    }
+
+    @Override
+    public BytesRef binaryValue() {

Review comment:
       Hmm, why do expose both `binaryValue` and `vectorValue` for each vector?

##########
File path: lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90VectorReader.java
##########
@@ -0,0 +1,326 @@
+/*
+ * 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.lucene90;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.FloatBuffer;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.VectorReader;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.VectorValues;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.RamUsageEstimator;
+
+/**
+ * Reads vectors from the index segments.
+ */
+public final class Lucene90VectorReader extends VectorReader {
+
+  private final FieldInfos fieldInfos;
+  private final Map<String, FieldEntry> fields = new HashMap<>();
+  private final IndexInput vectorData;
+  private final int maxDoc;
+
+  Lucene90VectorReader(SegmentReadState state) throws IOException {
+    this.fieldInfos = state.fieldInfos;
+    this.maxDoc = state.segmentInfo.maxDoc();
+
+    String metaFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Lucene90VectorFormat.META_EXTENSION);
+    int versionMeta = -1;
+    try (ChecksumIndexInput meta = state.directory.openChecksumInput(metaFileName, state.context)) {
+      Throwable priorE = null;
+      try {
+        versionMeta = CodecUtil.checkIndexHeader(meta,
+            Lucene90VectorFormat.META_CODEC_NAME,
+            Lucene90VectorFormat.VERSION_START,
+            Lucene90VectorFormat.VERSION_CURRENT,
+            state.segmentInfo.getId(),
+            state.segmentSuffix);
+        readFields(meta, state.fieldInfos);
+      } catch (Throwable exception) {
+        priorE = exception;
+      } finally {
+        CodecUtil.checkFooter(meta, priorE);
+      }
+    }
+
+    boolean success = false;
+
+    String vectorDataFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Lucene90VectorFormat.VECTOR_DATA_EXTENSION);
+    this.vectorData = state.directory.openInput(vectorDataFileName, state.context);
+    try {
+      int versionVectorData = CodecUtil.checkIndexHeader(vectorData,
+          Lucene90VectorFormat.VECTOR_DATA_CODEC_NAME,
+          Lucene90VectorFormat.VERSION_START,
+          Lucene90VectorFormat.VERSION_CURRENT,
+          state.segmentInfo.getId(),
+          state.segmentSuffix);
+      if (versionMeta != versionVectorData) {
+        throw new CorruptIndexException("Format versions mismatch: meta=" + versionMeta + ", vector data=" + versionVectorData, vectorData);
+      }
+      CodecUtil.retrieveChecksum(vectorData);
+
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(this.vectorData);
+      }
+    }
+  }
+
+  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);
+      }
+      VectorValues.ScoreFunction scoreFunction = VectorValues.ScoreFunction.fromId(meta.readInt());
+      long vectorDataOffset = meta.readVLong();
+      long vectorDataLength = meta.readVLong();
+      int dimension = meta.readInt();
+      int size = meta.readInt();
+      int[] ordToDoc = new int[size];
+      for (int i = 0; i < size; i++) {
+        int doc = meta.readVInt();
+        ordToDoc[i] = doc;
+      }
+      FieldEntry fieldEntry = new FieldEntry(dimension, scoreFunction, maxDoc, vectorDataOffset, vectorDataLength,
+                                              ordToDoc);
+      fields.put(info.name, fieldEntry);
+    }
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    long totalBytes = RamUsageEstimator.shallowSizeOfInstance(Lucene90VectorReader.class);
+    totalBytes += RamUsageEstimator.sizeOfMap(fields, RamUsageEstimator.shallowSizeOfInstance(FieldEntry.class));
+    for (FieldEntry entry : fields.values()) {
+      totalBytes += RamUsageEstimator.sizeOf(entry.ordToDoc);
+    }
+    return totalBytes;
+  }
+
+  @Override
+  public void checkIntegrity() throws IOException {
+    CodecUtil.checksumEntireFile(vectorData);
+  }
+
+  @Override
+  public VectorValues getVectorValues(String field) throws IOException {
+    FieldInfo info = fieldInfos.fieldInfo(field);
+    if (info == null) {
+      return null;
+    }
+    int dimension = info.getVectorDimension();
+    if (dimension == 0) {
+      return VectorValues.EMPTY;
+    }
+    FieldEntry fieldEntry = fields.get(field);
+    if (fieldEntry == null) {
+      // There is a FieldInfo, but no vectors. Should we have deleted the FieldInfo?
+      return null;
+    }
+    if (dimension != fieldEntry.dimension) {
+      throw new IllegalStateException("Inconsistent vector dimension for field=\"" + field + "\"; " + dimension + " != " + fieldEntry.dimension);
+    }
+    long numBytes = (long) fieldEntry.size() * dimension * Float.BYTES;
+    if (numBytes != fieldEntry.vectorDataLength) {
+      throw new IllegalStateException("Vector data length " + fieldEntry.vectorDataLength +
+          " not matching size=" + fieldEntry.size() + " * dim=" + dimension + " * 4 = " +
+          numBytes);
+    }
+    IndexInput bytesSlice = vectorData.slice("vector-data", fieldEntry.vectorDataOffset, fieldEntry.vectorDataLength);
+    return new OffHeapVectorValues(fieldEntry, bytesSlice);
+  }
+
+  @Override
+  public void close() throws IOException {
+    vectorData.close();
+  }
+
+  private static class FieldEntry {
+
+    final int dimension;
+    final VectorValues.ScoreFunction scoreFunction;
+    final int maxDoc;
+
+    final long vectorDataOffset;
+    final long vectorDataLength;
+    final int[] ordToDoc;
+
+    FieldEntry(int dimension, VectorValues.ScoreFunction scoreFunction, int maxDoc,
+               long vectorDataOffset, long vectorDataLength, int[] ordToDoc) {
+      this.dimension = dimension;
+      this.scoreFunction = scoreFunction;
+      this.maxDoc = maxDoc;
+      this.vectorDataOffset = vectorDataOffset;
+      this.vectorDataLength = vectorDataLength;
+      this.ordToDoc = ordToDoc;
+    }
+
+    int size() {
+      return ordToDoc.length;
+    }
+  }
+
+  /** Read the vector values from the index input. This supports both iterated and random access. */
+  private final static class OffHeapVectorValues extends VectorValues {
+
+    final FieldEntry fieldEntry;
+    final IndexInput dataIn;
+
+    final BytesRef binaryValue;
+    final ByteBuffer byteBuffer;
+    final FloatBuffer floatBuffer;
+    final int byteSize;
+    final float[] value;
+
+    int ord = -1;
+    int doc = -1;
+
+    OffHeapVectorValues(FieldEntry fieldEntry, IndexInput dataIn) {
+      this.fieldEntry = fieldEntry;
+      this.dataIn = dataIn;
+      byteSize = Float.BYTES * fieldEntry.dimension;
+      byteBuffer = ByteBuffer.allocate(byteSize);
+      floatBuffer = byteBuffer.asFloatBuffer();
+      value = new float[fieldEntry.dimension];
+      binaryValue = new BytesRef(byteBuffer.array(), byteBuffer.arrayOffset(), byteSize);
+    }
+
+    @Override
+    public int dimension() {
+      return fieldEntry.dimension;
+    }
+
+    @Override
+    public int size() {
+      return fieldEntry.size();
+    }
+
+    @Override
+    public ScoreFunction scoreFunction() {
+      return fieldEntry.scoreFunction;
+    }
+
+    @Override
+    public float[] vectorValue() throws IOException {
+      dataIn.seek(ord * byteSize);
+      dataIn.readBytes(byteBuffer.array(), byteBuffer.arrayOffset(), byteSize);

Review comment:
       Maybe just call `binaryValue();` here?

##########
File path: lucene/core/src/java/org/apache/lucene/index/VectorValues.java
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.index;
+
+import java.io.IOException;
+
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Access to per-document vector value.
+ */
+public abstract class VectorValues extends DocIdSetIterator {
+
+  /** The maximum length of a vector */
+  public static int MAX_DIMENSIONS = 1024;
+
+  /** Sole constructor */
+  protected VectorValues() {}
+
+  /**
+   * Return the dimension of the vectors
+   */
+  public abstract int dimension();
+
+  /**
+   * TODO: should we use cost() for this? We rely on its always being exactly the number
+   * of documents having a value for this field, which is not guaranteed by the cost() contract,
+   * but in all the implementations so far they are the same.
+   * @return the number of vectors returned by this iterator
+   */
+  public abstract int size();
+
+  /**
+   * Return the score function used to compare these vectors
+   */
+  public abstract ScoreFunction scoreFunction();
+
+  /**
+   * Return the vector value for the current document ID.
+   * It is illegal to call this method after the iterator failed to advance.
+   * @return the vector value
+   */
+  public abstract float[] vectorValue() throws IOException;
+
+  /**
+   * Return the binary encoded vector value for the current document ID.
+   * It is illegal to call this method after the iterator failed to advance.
+   * @return the binary value
+   */
+  public BytesRef binaryValue() throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  /**
+   * Return a random access interface over this iterator's vectors.
+   */
+  public abstract RandomAccess randomAccess();
+
+  /**
+   * Provides random access to vectors by dense ordinal
+   */
+  public interface RandomAccess {
+
+    /**
+     * Return the vector value as a floating point array.
+     * @param targetOrd a valid ordinal, &ge; 0 and &lt; {@link #size()}.
+     */
+    float[] vectorValue(int targetOrd) throws IOException;
+
+    /**
+     * Return the vector value as a byte array; these are the bytes corresponding to the float array
+     * encoded using little-endian byte order.
+     * @param targetOrd a valid ordinal, &ge; 0 and &lt; {@link #size()}.
+     */
+    BytesRef binaryValue(int targetOrd) throws IOException;
+
+    /**
+     * Return the k nearest neighbor documents as determined by comparison of their vector values
+     * for this field, to the given vector, by the field's score function. If the score function is
+     * reversed, lower values indicate nearer vectors, otherwise higher scores indicate nearer
+     * vectors. Unlike relevance scores, vector scores may be negative.
+     * @param target the vector-valued query
+     * @param k      the number of docs to return
+     * @param fanout control the accuracy/speed tradeoff - larger values give better recall at higher cost
+     * @return the k nearest neighbor documents, along with their (scoreFunction-specific) scores.
+     */
+    TopDocs search(float[] target, int k, int fanout) throws IOException;
+  }
+
+  /**
+   * Score function. This is used during indexing and searching of the vectors to determine the nearest neighbors.
+   * Score values may be negative. By default high scores indicate nearer documents, unless the function is reversed.
+   */
+  public enum ScoreFunction {
+    /** No distance function is used. Note: {@link VectorValues.RandomAccess#search(float[], int, int)}

Review comment:
       Maybe re-emphasize the javadoc you added for `id` below?
   
   I.e. add comment here that the `id` must not be changed!  On adding a new `ScoreFunction` in the future, it must go to the next `id`.  Else we break back compat of existing Lucene indices which store these int `id`s.
   
   Could you open a follow-on issue to add back compat indices for vectors into `TestBackwardsCompatibility`, as blocker for 9.0 release?  We cannot do this issue now (the format may change between now and 9.0) but let's not forget to do it just before release!

##########
File path: lucene/core/src/java/org/apache/lucene/index/VectorValues.java
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.index;
+
+import java.io.IOException;
+
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Access to per-document vector value.
+ */
+public abstract class VectorValues extends DocIdSetIterator {
+
+  /** The maximum length of a vector */
+  public static int MAX_DIMENSIONS = 1024;
+
+  /** Sole constructor */
+  protected VectorValues() {}
+
+  /**
+   * Return the dimension of the vectors
+   */
+  public abstract int dimension();
+
+  /**
+   * TODO: should we use cost() for this? We rely on its always being exactly the number
+   * of documents having a value for this field, which is not guaranteed by the cost() contract,
+   * but in all the implementations so far they are the same.
+   * @return the number of vectors returned by this iterator
+   */
+  public abstract int size();
+
+  /**
+   * Return the score function used to compare these vectors
+   */
+  public abstract ScoreFunction scoreFunction();
+
+  /**
+   * Return the vector value for the current document ID.
+   * It is illegal to call this method after the iterator failed to advance.
+   * @return the vector value
+   */
+  public abstract float[] vectorValue() throws IOException;
+
+  /**
+   * Return the binary encoded vector value for the current document ID.
+   * It is illegal to call this method after the iterator failed to advance.
+   * @return the binary value
+   */
+  public BytesRef binaryValue() throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  /**
+   * Return a random access interface over this iterator's vectors.
+   */
+  public abstract RandomAccess randomAccess();
+
+  /**
+   * Provides random access to vectors by dense ordinal
+   */
+  public interface RandomAccess {
+
+    /**
+     * Return the vector value as a floating point array.
+     * @param targetOrd a valid ordinal, &ge; 0 and &lt; {@link #size()}.
+     */
+    float[] vectorValue(int targetOrd) throws IOException;

Review comment:
       Maybe state in the javadoc that the returned `float[]` may be reused across calls?

##########
File path: lucene/core/src/java/org/apache/lucene/index/VectorValues.java
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.index;
+
+import java.io.IOException;
+
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Access to per-document vector value.
+ */
+public abstract class VectorValues extends DocIdSetIterator {
+
+  /** The maximum length of a vector */
+  public static int MAX_DIMENSIONS = 1024;
+
+  /** Sole constructor */
+  protected VectorValues() {}
+
+  /**
+   * Return the dimension of the vectors
+   */
+  public abstract int dimension();
+
+  /**
+   * TODO: should we use cost() for this? We rely on its always being exactly the number
+   * of documents having a value for this field, which is not guaranteed by the cost() contract,
+   * but in all the implementations so far they are the same.
+   * @return the number of vectors returned by this iterator
+   */
+  public abstract int size();
+
+  /**
+   * Return the score function used to compare these vectors
+   */
+  public abstract ScoreFunction scoreFunction();
+
+  /**
+   * Return the vector value for the current document ID.
+   * It is illegal to call this method after the iterator failed to advance.
+   * @return the vector value
+   */
+  public abstract float[] vectorValue() throws IOException;
+
+  /**
+   * Return the binary encoded vector value for the current document ID.
+   * It is illegal to call this method after the iterator failed to advance.
+   * @return the binary value
+   */
+  public BytesRef binaryValue() throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  /**
+   * Return a random access interface over this iterator's vectors.
+   */
+  public abstract RandomAccess randomAccess();
+
+  /**
+   * Provides random access to vectors by dense ordinal
+   */
+  public interface RandomAccess {
+
+    /**
+     * Return the vector value as a floating point array.
+     * @param targetOrd a valid ordinal, &ge; 0 and &lt; {@link #size()}.
+     */
+    float[] vectorValue(int targetOrd) throws IOException;
+
+    /**
+     * Return the vector value as a byte array; these are the bytes corresponding to the float array
+     * encoded using little-endian byte order.

Review comment:
       And `IEEE 754` standard encoding?
   
   Maybe state in the javadoc that the returned `BytesRef` may be reused by this method across calls?

##########
File path: lucene/core/src/test/org/apache/lucene/index/TestVectorValues.java
##########
@@ -0,0 +1,606 @@
+/*
+ * 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.index;
+
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.document.VectorField;
+import org.apache.lucene.index.VectorValues.ScoreFunction;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.FSDirectory;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+/** Test Indexing/IndexWriter with vectors */
+public class TestVectorValues extends LuceneTestCase {
+
+  private IndexWriterConfig createIndexWriterConfig() {
+    IndexWriterConfig iwc = newIndexWriterConfig();
+    iwc.setCodec(Codec.forName("Lucene90"));
+    return iwc;
+  }
+
+  // Suddenly add vectors to an existing field:
+  public void testUpgradeFieldToVectors() throws Exception {
+    try (Directory dir = newDirectory()) {
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(newStringField("dim", "foo", Store.NO));
+        w.addDocument(doc);
+      }
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+        w.addDocument(doc);
+      }
+    }
+  }
+
+  public void testFieldConstructor() {
+    float[] v = new float[1];
+    VectorField field = new VectorField("f", v);
+    assertEquals(1, field.fieldType().vectorDimension());
+    assertEquals(ScoreFunction.EUCLIDEAN, field.fieldType().vectorScoreFunction());
+    assertSame(v, field.vectorValue());
+  }
+
+  public void testFieldConstructorExceptions() {
+    expectThrows(IllegalArgumentException.class, () -> new VectorField(null, new float[1]));
+    expectThrows(IllegalArgumentException.class, () -> new VectorField("f", null));
+    expectThrows(IllegalArgumentException.class, () -> new VectorField("f", new float[1], null));
+    expectThrows(IllegalArgumentException.class, () -> new VectorField("f", new float[0]));
+    expectThrows(IllegalArgumentException.class, () -> new VectorField("f", new float[VectorValues.MAX_DIMENSIONS + 1]));
+  }
+
+  public void testFieldSetValue() {
+    VectorField field = new VectorField("f", new float[1]);
+    float[] v1 = new float[1];
+    field.setVectorValue(v1);
+    assertSame(v1, field.vectorValue());
+    expectThrows(IllegalArgumentException.class, () -> field.setVectorValue(new float[2]));
+    expectThrows(NullPointerException.class, () -> field.setVectorValue(null));
+  }
+
+  // Illegal schema change tests:
+
+  public void testIllegalDimChangeTwoDocs() throws Exception {
+    try (Directory dir = newDirectory();
+         IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+      Document doc = new Document();
+      doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+      w.addDocument(doc);
+      if (random().nextBoolean()) {
+        // sometimes test with two segments
+        w.commit();
+      }
+
+      Document doc2 = new Document();
+      doc2.add(new VectorField("dim", new float[3], ScoreFunction.DOT_PRODUCT));
+      IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+          () -> w.addDocument(doc2));
+      assertEquals("cannot change vector dimension from 4 to 3 for field=\"dim\"", expected.getMessage());
+    }
+  }
+
+  public void testIllegalScoreFunctionChange() throws Exception {
+    try (Directory dir = newDirectory();
+         IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+      Document doc = new Document();
+      doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+      w.addDocument(doc);
+      if (random().nextBoolean()) {
+        // sometimes test with two segments
+        w.commit();
+      }
+
+      Document doc2 = new Document();
+      doc2.add(new VectorField("dim", new float[4], ScoreFunction.EUCLIDEAN));
+      IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+          () -> w.addDocument(doc2));
+      assertEquals("cannot change vector score function from DOT_PRODUCT to EUCLIDEAN for field=\"dim\"", expected.getMessage());
+    }
+  }
+
+  public void testIllegalDimChangeTwoWriters() throws Exception {
+    try (Directory dir = newDirectory()) {
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+        w.addDocument(doc);
+      }
+
+      try (IndexWriter w2 = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc2 = new Document();
+        doc2.add(new VectorField("dim", new float[1], ScoreFunction.DOT_PRODUCT));
+        IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+            () -> w2.addDocument(doc2));
+        assertEquals("cannot change vector dimension from 4 to 1 for field=\"dim\"", expected.getMessage());
+      }
+    }
+  }
+
+  public void testIllegalScoreFunctionChangeTwoWriters() throws Exception {
+    try (Directory dir = newDirectory()) {
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+        w.addDocument(doc);
+      }
+
+      try (IndexWriter w2 = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc2 = new Document();
+        doc2.add(new VectorField("dim", new float[4], ScoreFunction.EUCLIDEAN));
+        IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+            () -> w2.addDocument(doc2));
+        assertEquals("cannot change vector score function from DOT_PRODUCT to EUCLIDEAN for field=\"dim\"", expected.getMessage());
+      }
+    }
+  }
+
+  public void testIllegalDimChangeViaAddIndexesDirectory() throws Exception {
+    try (Directory dir = newDirectory();
+         Directory dir2 = newDirectory()) {
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+        w.addDocument(doc);
+      }
+      try (IndexWriter w2 = new IndexWriter(dir2, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[5], ScoreFunction.DOT_PRODUCT));
+        w2.addDocument(doc);
+        IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+            () -> w2.addIndexes(new Directory[]{dir}));
+        assertEquals("cannot change vector dimension from 5 to 4 for field=\"dim\"", expected.getMessage());
+      }
+    }
+  }
+
+  public void testIllegalScoreFunctionChangeViaAddIndexesDirectory() throws Exception {
+    try (Directory dir = newDirectory();
+         Directory dir2 = newDirectory()) {
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+        w.addDocument(doc);
+      }
+      try (IndexWriter w2 = new IndexWriter(dir2, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.EUCLIDEAN));
+        w2.addDocument(doc);
+        IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+            () -> w2.addIndexes(dir));
+        assertEquals("cannot change vector score function from EUCLIDEAN to DOT_PRODUCT for field=\"dim\"", expected.getMessage());
+      }
+    }
+  }
+
+  public void testIllegalDimChangeViaAddIndexesCodecReader() throws Exception {
+    try (Directory dir = newDirectory();
+         Directory dir2 = newDirectory()) {
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+        w.addDocument(doc);
+      }
+      try (IndexWriter w2 = new IndexWriter(dir2, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[5], ScoreFunction.DOT_PRODUCT));
+        w2.addDocument(doc);
+        try (DirectoryReader r = DirectoryReader.open(dir)) {
+          IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+              () -> w2.addIndexes(new CodecReader[]{(CodecReader) getOnlyLeafReader(r)}));
+          assertEquals("cannot change vector dimension from 5 to 4 for field=\"dim\"", expected.getMessage());
+        }
+      }
+    }
+  }
+
+  public void testIllegalScoreFunctionChangeViaAddIndexesCodecReader() throws Exception {
+    try (Directory dir = newDirectory();
+         Directory dir2 = newDirectory()) {
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+        w.addDocument(doc);
+      }
+      try (IndexWriter w2 = new IndexWriter(dir2, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.EUCLIDEAN));
+        w2.addDocument(doc);
+        try (DirectoryReader r = DirectoryReader.open(dir)) {
+          IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+              () -> w2.addIndexes(new CodecReader[]{(CodecReader) getOnlyLeafReader(r)}));
+          assertEquals("cannot change vector score function from EUCLIDEAN to DOT_PRODUCT for field=\"dim\"", expected.getMessage());
+        }
+      }
+    }
+  }
+
+  public void testIllegalDimChangeViaAddIndexesSlowCodecReader() throws Exception {
+    try (Directory dir = newDirectory();
+         Directory dir2 = newDirectory()) {
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+        w.addDocument(doc);
+      }
+      try (IndexWriter w2 = new IndexWriter(dir2, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[5], ScoreFunction.DOT_PRODUCT));
+        w2.addDocument(doc);
+        try (DirectoryReader r = DirectoryReader.open(dir)) {
+          IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+              () -> TestUtil.addIndexesSlowly(w2, r));
+          assertEquals("cannot change vector dimension from 5 to 4 for field=\"dim\"", expected.getMessage());
+        }
+      }
+    }
+  }
+
+  public void testIllegalScoreFunctionChangeViaAddIndexesSlowCodecReader() throws Exception {
+    try (Directory dir = newDirectory();
+         Directory dir2 = newDirectory()) {
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+        w.addDocument(doc);
+      }
+      try (IndexWriter w2 = new IndexWriter(dir2, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.EUCLIDEAN));
+        w2.addDocument(doc);
+        try (DirectoryReader r = DirectoryReader.open(dir)) {
+          IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+              () -> TestUtil.addIndexesSlowly(w2, r));
+          assertEquals("cannot change vector score function from EUCLIDEAN to DOT_PRODUCT for field=\"dim\"", expected.getMessage());
+        }
+      }
+    }
+  }
+
+  public void testIllegalMultipleValues() throws Exception {
+    try (Directory dir = newDirectory();
+         IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+      Document doc = new Document();
+      doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+      doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+      IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+          () -> w.addDocument(doc));
+      assertEquals("VectorValuesField \"dim\" appears more than once in this document (only one value is allowed per field)",
+          expected.getMessage());
+    }
+  }
+
+  public void testIllegalDimensionTooLarge() throws Exception {
+    try (Directory dir = newDirectory();
+         IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+      Document doc = new Document();
+      expectThrows(IllegalArgumentException.class,
+          () -> doc.add(new VectorField("dim", new float[VectorValues.MAX_DIMENSIONS + 1], ScoreFunction.DOT_PRODUCT)));
+
+      Document doc2 = new Document();
+      doc2.add(new VectorField("dim", new float[1], ScoreFunction.EUCLIDEAN));
+      w.addDocument(doc2);
+    }
+  }
+
+  public void testIllegalEmptyVector() throws Exception {
+    try (Directory dir = newDirectory();
+         IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+      Document doc = new Document();
+      Exception e = expectThrows(IllegalArgumentException.class,
+          () -> doc.add(new VectorField("dim", new float[0], ScoreFunction.NONE)));
+      assertEquals("cannot index an empty vector", e.getMessage());
+
+      Document doc2 = new Document();
+      doc2.add(new VectorField("dim", new float[1], ScoreFunction.NONE));
+      w.addDocument(doc2);
+    }
+  }
+
+  // Write vectors, one segment with default codec, another with SimpleText, then forceMerge
+  public void testDifferentCodecs1() throws Exception {
+    try (Directory dir = newDirectory()) {
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+        w.addDocument(doc);
+      }
+      IndexWriterConfig iwc = newIndexWriterConfig();
+      iwc.setCodec(Codec.forName("SimpleText"));
+      try (IndexWriter w = new IndexWriter(dir, iwc)) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+        w.addDocument(doc);
+        w.forceMerge(1);
+      }
+    }
+  }
+
+  // Write vectors, one segment with with SimpleText, another with default codec, then forceMerge
+  public void testDifferentCodecs2() throws Exception {
+    IndexWriterConfig iwc = newIndexWriterConfig();
+    iwc.setCodec(Codec.forName("SimpleText"));
+    try (Directory dir = newDirectory()) {
+      try (IndexWriter w = new IndexWriter(dir, iwc)) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+        w.addDocument(doc);
+      }
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+        w.addDocument(doc);
+        w.forceMerge(1);
+      }
+    }
+  }
+
+  public void testInvalidVectorFieldUsage() {
+    VectorField field = new VectorField("field", new float[2], ScoreFunction.NONE);
+
+    expectThrows(IllegalArgumentException.class, () -> field.setIntValue(14));
+
+    expectThrows(IllegalArgumentException.class, () -> field.setVectorValue(new float[1]));
+
+    assertNull(field.numericValue());
+  }
+
+  public void testDeleteAllVectorDocs() throws Exception {
+    try (Directory dir = newDirectory();
+         IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+      Document doc = new Document();
+      doc.add(new StringField("id", "0", Store.NO));
+      doc.add(new VectorField("v", new float[]{2, 3, 5}, ScoreFunction.DOT_PRODUCT));
+      w.addDocument(doc);
+      w.addDocument(new Document());
+      w.commit();
+
+      try (DirectoryReader r = w.getReader()) {
+        assertNotNull(r.leaves().get(0).reader().getVectorValues("v"));
+      }
+      w.deleteDocuments(new Term("id", "0"));
+      w.forceMerge(1);
+      try (DirectoryReader r = w.getReader()) {
+        assertNull(r.leaves().get(0).reader().getVectorValues("v"));
+      }
+    }
+  }
+
+  public void testVectorFieldMissingFromOneSegment() throws Exception {
+    try (Directory dir = FSDirectory.open(createTempDir());
+         IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+      Document doc = new Document();
+      doc.add(new StringField("id", "0", Store.NO));
+      doc.add(new VectorField("v0", new float[]{2, 3, 5}, ScoreFunction.DOT_PRODUCT));
+      w.addDocument(doc);
+      w.commit();
+
+      doc = new Document();
+      doc.add(new VectorField("v1", new float[]{2, 3, 5}, ScoreFunction.DOT_PRODUCT));
+      w.addDocument(doc);
+      w.forceMerge(1);
+    }
+  }
+
+  public void testSparseVectors() throws Exception {
+    int numDocs = atLeast(1000);
+    int numFields = TestUtil.nextInt(random(), 1, 10);
+    int[] fieldDocCounts = new int[numFields];
+    float[] fieldTotals= new float[numFields];
+    int[] fieldDims = new int[numFields];
+    ScoreFunction[] fieldScoreFunctions = new ScoreFunction[numFields];
+    for (int i = 0; i < numFields; i++) {
+      fieldDims[i] = random().nextInt(20) + 1;
+      fieldScoreFunctions[i] = ScoreFunction.fromId(random().nextInt(ScoreFunction.values().length));
+    }
+    try (Directory dir = newDirectory();
+         RandomIndexWriter w = new RandomIndexWriter(random(), dir, createIndexWriterConfig())) {
+      for (int i = 0; i < numDocs; i++) {
+        Document doc = new Document();
+        for (int field = 0; field < numFields; field++) {
+          String fieldName = "int" + field;
+          if (random().nextInt(100) == 17) {
+            float[] v = randomVector(fieldDims[field]);
+            doc.add(new VectorField(fieldName, v, fieldScoreFunctions[field]));
+            fieldDocCounts[field]++;
+            fieldTotals[field] += v[0];
+          }
+        }
+        w.addDocument(doc);
+      }
+
+      try (IndexReader r = w.getReader()) {
+        for (int field = 0; field < numFields; field++) {
+          int docCount = 0;
+          float checksum = 0;
+          String fieldName = "int" + field;
+          for (LeafReaderContext ctx : r.leaves()) {
+            VectorValues vectors = ctx.reader().getVectorValues(fieldName);
+            if (vectors != null) {
+              docCount += vectors.size();
+              while (vectors.nextDoc() != NO_MORE_DOCS) {
+                checksum += vectors.vectorValue()[0];
+              }
+            }
+          }
+          assertEquals(fieldDocCounts[field], docCount);
+          assertEquals(fieldTotals[field], checksum, 1e-5);
+        }
+      }
+    }
+  }
+
+  public void testIndexedValueNotAliased() throws Exception {
+    // We copy indexed values (as for BinaryDocValues) so the input float[] can be reused across
+    // calls to IndexWriter.addDocument.
+    String fieldName = "field";
+    float[] v = { 0 };
+    try (Directory dir = newDirectory();
+         IndexWriter iw = new IndexWriter(dir, createIndexWriterConfig())) {
+      Document doc1 = new Document();
+      doc1.add(new VectorField(fieldName, v, VectorValues.ScoreFunction.EUCLIDEAN));
+      v[0] = 1;
+      Document doc2 = new Document();
+      doc2.add(new VectorField(fieldName, v, VectorValues.ScoreFunction.EUCLIDEAN));
+      iw.addDocument(doc1);
+      iw.addDocument(doc2);
+      v[0] = 2;
+      Document doc3 = new Document();
+      doc3.add(new VectorField(fieldName, v, VectorValues.ScoreFunction.EUCLIDEAN));
+      iw.addDocument(doc3);
+      try (IndexReader reader = iw.getReader()) {
+        LeafReader r = reader.leaves().get(0).reader();
+        VectorValues vectorValues = r.getVectorValues(fieldName);
+        vectorValues.nextDoc();
+        assertEquals(1, vectorValues.vectorValue()[0], 0);
+        vectorValues.nextDoc();
+        assertEquals(1, vectorValues.vectorValue()[0], 0);
+        vectorValues.nextDoc();
+        assertEquals(2, vectorValues.vectorValue()[0], 0);
+      }
+    }
+  }
+
+  /**
+   * Index random vectors, sometimes skipping documents, sometimes deleting a document,
+   * sometimes merging, sometimes sorting the index,
+   * and verify that the expected values can be read back consistently.
+   */
+  public void testRandom() throws Exception {

Review comment:
       Yay!

##########
File path: lucene/core/src/java/org/apache/lucene/document/FieldType.java
##########
@@ -351,6 +356,27 @@ public int pointNumBytes() {
     return dimensionNumBytes;
   }
 
+  void setVectorDimensionsAndScoreFunction(int numDimensions, VectorValues.ScoreFunction distFunc) {
+    if (numDimensions <= 0) {
+      throw new IllegalArgumentException("vector numDimensions must be > 0; got " + numDimensions);
+    }
+    if (numDimensions > VectorValues.MAX_DIMENSIONS) {
+      throw new IllegalArgumentException("vector numDimensions must be <= VectorValues.MAX_DIMENSIONS (=" + VectorValues.MAX_DIMENSIONS + "); got " + numDimensions);
+    }
+    this.vectorDimension = numDimensions;

Review comment:
       Should we verify that the caller is not attempting to change a previously set `vectorDimension` and `vectorScoreFunction`?  E.g. init `vectorDimension = -1` and then confirm it is either being set for the first time, or, the incoming values match what was already set?
   
   Edit: oh, nevermind, I see you are doing those checks in `FieldInfo.java`, and we should NOT do these checks here, since caller is free to tweak the `FieldType` up until they `.freeze()` it.  Hmm, should we add `checkIfFrozen()` at the top of this method?  (Hmm and also to `.setDimension(..)` for points?).

##########
File path: lucene/core/src/java/org/apache/lucene/index/VectorValuesWriter.java
##########
@@ -0,0 +1,272 @@
+/*
+ * 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.index;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.codecs.VectorWriter;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.Counter;
+
+/** Buffers up pending vector value(s) per doc, then flushes when segment flushes. */
+public class VectorValuesWriter {
+
+  private final FieldInfo fieldInfo;
+  private final Counter iwBytesUsed;
+  private final List<float[]> vectors = new ArrayList<>();
+  private final DocsWithFieldSet docsWithField;
+
+  private int lastDocID = -1;
+
+  private long bytesUsed;
+
+  VectorValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
+    this.fieldInfo = fieldInfo;
+    this.iwBytesUsed = iwBytesUsed;
+    this.docsWithField = new DocsWithFieldSet();
+    this.bytesUsed = docsWithField.ramBytesUsed();
+    if (iwBytesUsed != null) {
+      iwBytesUsed.addAndGet(bytesUsed);
+    }
+  }
+
+  /**
+   * Adds a value for the given document. Only a single value may be added.
+   * @param docID the value is added to this document
+   * @param vectorValue the value to add
+   * @throws IllegalArgumentException if a value has already been added to the given document
+   */
+  public void addValue(int docID, float[] vectorValue) {
+    if (docID == lastDocID) {
+      throw new IllegalArgumentException("VectorValuesField \"" + fieldInfo.name + "\" appears more than once in this document (only one value is allowed per field)");
+    }
+    assert docID > lastDocID;
+    docsWithField.add(docID);
+    vectors.add(ArrayUtil.copyOfSubArray(vectorValue, 0, vectorValue.length));
+    updateBytesUsed();
+    lastDocID = docID;
+  }
+
+  private void updateBytesUsed() {
+    final long newBytesUsed = docsWithField.ramBytesUsed()
+            + vectors.size() * 5 // pointer plus array overhead for each array??

Review comment:
       Instead of `5` I think you can use `RamUsageEstimator.NUM_BYTES_OBJECT_REF + RamUsageEstimator.NUM_BYTES_ARRAY_HEADER`?

##########
File path: lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90VectorReader.java
##########
@@ -0,0 +1,326 @@
+/*
+ * 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.lucene90;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.FloatBuffer;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.VectorReader;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.VectorValues;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.RamUsageEstimator;
+
+/**
+ * Reads vectors from the index segments.
+ */
+public final class Lucene90VectorReader extends VectorReader {
+
+  private final FieldInfos fieldInfos;
+  private final Map<String, FieldEntry> fields = new HashMap<>();
+  private final IndexInput vectorData;
+  private final int maxDoc;
+
+  Lucene90VectorReader(SegmentReadState state) throws IOException {
+    this.fieldInfos = state.fieldInfos;
+    this.maxDoc = state.segmentInfo.maxDoc();
+
+    String metaFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Lucene90VectorFormat.META_EXTENSION);
+    int versionMeta = -1;
+    try (ChecksumIndexInput meta = state.directory.openChecksumInput(metaFileName, state.context)) {
+      Throwable priorE = null;
+      try {
+        versionMeta = CodecUtil.checkIndexHeader(meta,
+            Lucene90VectorFormat.META_CODEC_NAME,
+            Lucene90VectorFormat.VERSION_START,
+            Lucene90VectorFormat.VERSION_CURRENT,
+            state.segmentInfo.getId(),
+            state.segmentSuffix);
+        readFields(meta, state.fieldInfos);
+      } catch (Throwable exception) {
+        priorE = exception;
+      } finally {
+        CodecUtil.checkFooter(meta, priorE);
+      }
+    }
+
+    boolean success = false;
+
+    String vectorDataFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Lucene90VectorFormat.VECTOR_DATA_EXTENSION);
+    this.vectorData = state.directory.openInput(vectorDataFileName, state.context);
+    try {
+      int versionVectorData = CodecUtil.checkIndexHeader(vectorData,
+          Lucene90VectorFormat.VECTOR_DATA_CODEC_NAME,
+          Lucene90VectorFormat.VERSION_START,
+          Lucene90VectorFormat.VERSION_CURRENT,
+          state.segmentInfo.getId(),
+          state.segmentSuffix);
+      if (versionMeta != versionVectorData) {
+        throw new CorruptIndexException("Format versions mismatch: meta=" + versionMeta + ", vector data=" + versionVectorData, vectorData);
+      }
+      CodecUtil.retrieveChecksum(vectorData);
+
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(this.vectorData);
+      }
+    }
+  }
+
+  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);
+      }
+      VectorValues.ScoreFunction scoreFunction = VectorValues.ScoreFunction.fromId(meta.readInt());
+      long vectorDataOffset = meta.readVLong();
+      long vectorDataLength = meta.readVLong();
+      int dimension = meta.readInt();
+      int size = meta.readInt();
+      int[] ordToDoc = new int[size];
+      for (int i = 0; i < size; i++) {
+        int doc = meta.readVInt();
+        ordToDoc[i] = doc;
+      }
+      FieldEntry fieldEntry = new FieldEntry(dimension, scoreFunction, maxDoc, vectorDataOffset, vectorDataLength,
+                                              ordToDoc);
+      fields.put(info.name, fieldEntry);
+    }
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    long totalBytes = RamUsageEstimator.shallowSizeOfInstance(Lucene90VectorReader.class);
+    totalBytes += RamUsageEstimator.sizeOfMap(fields, RamUsageEstimator.shallowSizeOfInstance(FieldEntry.class));
+    for (FieldEntry entry : fields.values()) {
+      totalBytes += RamUsageEstimator.sizeOf(entry.ordToDoc);
+    }
+    return totalBytes;
+  }
+
+  @Override
+  public void checkIntegrity() throws IOException {
+    CodecUtil.checksumEntireFile(vectorData);
+  }
+
+  @Override
+  public VectorValues getVectorValues(String field) throws IOException {
+    FieldInfo info = fieldInfos.fieldInfo(field);
+    if (info == null) {
+      return null;
+    }
+    int dimension = info.getVectorDimension();
+    if (dimension == 0) {
+      return VectorValues.EMPTY;
+    }
+    FieldEntry fieldEntry = fields.get(field);
+    if (fieldEntry == null) {
+      // There is a FieldInfo, but no vectors. Should we have deleted the FieldInfo?
+      return null;
+    }
+    if (dimension != fieldEntry.dimension) {
+      throw new IllegalStateException("Inconsistent vector dimension for field=\"" + field + "\"; " + dimension + " != " + fieldEntry.dimension);
+    }
+    long numBytes = (long) fieldEntry.size() * dimension * Float.BYTES;
+    if (numBytes != fieldEntry.vectorDataLength) {
+      throw new IllegalStateException("Vector data length " + fieldEntry.vectorDataLength +
+          " not matching size=" + fieldEntry.size() + " * dim=" + dimension + " * 4 = " +
+          numBytes);
+    }
+    IndexInput bytesSlice = vectorData.slice("vector-data", fieldEntry.vectorDataOffset, fieldEntry.vectorDataLength);
+    return new OffHeapVectorValues(fieldEntry, bytesSlice);
+  }
+
+  @Override
+  public void close() throws IOException {
+    vectorData.close();
+  }
+
+  private static class FieldEntry {
+
+    final int dimension;
+    final VectorValues.ScoreFunction scoreFunction;
+    final int maxDoc;
+
+    final long vectorDataOffset;
+    final long vectorDataLength;
+    final int[] ordToDoc;
+
+    FieldEntry(int dimension, VectorValues.ScoreFunction scoreFunction, int maxDoc,
+               long vectorDataOffset, long vectorDataLength, int[] ordToDoc) {
+      this.dimension = dimension;
+      this.scoreFunction = scoreFunction;
+      this.maxDoc = maxDoc;
+      this.vectorDataOffset = vectorDataOffset;
+      this.vectorDataLength = vectorDataLength;
+      this.ordToDoc = ordToDoc;
+    }
+
+    int size() {
+      return ordToDoc.length;
+    }
+  }
+
+  /** Read the vector values from the index input. This supports both iterated and random access. */
+  private final static class OffHeapVectorValues extends VectorValues {
+
+    final FieldEntry fieldEntry;
+    final IndexInput dataIn;
+
+    final BytesRef binaryValue;
+    final ByteBuffer byteBuffer;
+    final FloatBuffer floatBuffer;
+    final int byteSize;
+    final float[] value;
+
+    int ord = -1;
+    int doc = -1;
+
+    OffHeapVectorValues(FieldEntry fieldEntry, IndexInput dataIn) {
+      this.fieldEntry = fieldEntry;
+      this.dataIn = dataIn;
+      byteSize = Float.BYTES * fieldEntry.dimension;
+      byteBuffer = ByteBuffer.allocate(byteSize);
+      floatBuffer = byteBuffer.asFloatBuffer();
+      value = new float[fieldEntry.dimension];
+      binaryValue = new BytesRef(byteBuffer.array(), byteBuffer.arrayOffset(), byteSize);
+    }
+
+    @Override
+    public int dimension() {
+      return fieldEntry.dimension;
+    }
+
+    @Override
+    public int size() {
+      return fieldEntry.size();
+    }
+
+    @Override
+    public ScoreFunction scoreFunction() {
+      return fieldEntry.scoreFunction;
+    }
+
+    @Override
+    public float[] vectorValue() throws IOException {
+      dataIn.seek(ord * byteSize);
+      dataIn.readBytes(byteBuffer.array(), byteBuffer.arrayOffset(), byteSize);
+      floatBuffer.position(0);
+      floatBuffer.get(value, 0, fieldEntry.dimension);
+      return value;
+    }
+
+    @Override
+    public BytesRef binaryValue() throws IOException {
+      dataIn.seek(ord * byteSize);
+      dataIn.readBytes(byteBuffer.array(), byteBuffer.arrayOffset(), byteSize);
+      return binaryValue;
+    }
+
+    @Override
+    public int docID() {
+      return doc;
+    }
+
+    @Override
+    public int nextDoc() {
+      if (++ord >= size()) {
+        doc = NO_MORE_DOCS;
+      } else {
+        doc = fieldEntry.ordToDoc[ord];
+      }
+      return doc;
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      // We could do better by log-binary search in ordToDoc, but this is never used

Review comment:
       Both ord and docs are ascending?
   
   I suppose with this `ord` approach, we could e.g. dedup vectors?  So if the same vector appears across multiple docs, we only store it once?  And then ords/docs would not always be ascending.

##########
File path: lucene/core/src/java/org/apache/lucene/index/VectorValues.java
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.index;
+
+import java.io.IOException;
+
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Access to per-document vector value.
+ */
+public abstract class VectorValues extends DocIdSetIterator {
+
+  /** The maximum length of a vector */
+  public static int MAX_DIMENSIONS = 1024;
+
+  /** Sole constructor */
+  protected VectorValues() {}
+
+  /**
+   * Return the dimension of the vectors
+   */
+  public abstract int dimension();
+
+  /**
+   * TODO: should we use cost() for this? We rely on its always being exactly the number
+   * of documents having a value for this field, which is not guaranteed by the cost() contract,
+   * but in all the implementations so far they are the same.
+   * @return the number of vectors returned by this iterator
+   */
+  public abstract int size();
+
+  /**
+   * Return the score function used to compare these vectors
+   */
+  public abstract ScoreFunction scoreFunction();
+
+  /**
+   * Return the vector value for the current document ID.
+   * It is illegal to call this method after the iterator failed to advance.
+   * @return the vector value
+   */
+  public abstract float[] vectorValue() throws IOException;
+
+  /**
+   * Return the binary encoded vector value for the current document ID.
+   * It is illegal to call this method after the iterator failed to advance.
+   * @return the binary value
+   */
+  public BytesRef binaryValue() throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  /**
+   * Return a random access interface over this iterator's vectors.
+   */
+  public abstract RandomAccess randomAccess();
+
+  /**
+   * Provides random access to vectors by dense ordinal
+   */
+  public interface RandomAccess {
+
+    /**
+     * Return the vector value as a floating point array.
+     * @param targetOrd a valid ordinal, &ge; 0 and &lt; {@link #size()}.
+     */
+    float[] vectorValue(int targetOrd) throws IOException;
+
+    /**
+     * Return the vector value as a byte array; these are the bytes corresponding to the float array
+     * encoded using little-endian byte order.
+     * @param targetOrd a valid ordinal, &ge; 0 and &lt; {@link #size()}.
+     */
+    BytesRef binaryValue(int targetOrd) throws IOException;
+
+    /**
+     * Return the k nearest neighbor documents as determined by comparison of their vector values
+     * for this field, to the given vector, by the field's score function. If the score function is
+     * reversed, lower values indicate nearer vectors, otherwise higher scores indicate nearer
+     * vectors. Unlike relevance scores, vector scores may be negative.
+     * @param target the vector-valued query
+     * @param k      the number of docs to return
+     * @param fanout control the accuracy/speed tradeoff - larger values give better recall at higher cost
+     * @return the k nearest neighbor documents, along with their (scoreFunction-specific) scores.
+     */
+    TopDocs search(float[] target, int k, int fanout) throws IOException;
+  }
+
+  /**
+   * Score function. This is used during indexing and searching of the vectors to determine the nearest neighbors.
+   * Score values may be negative. By default high scores indicate nearer documents, unless the function is reversed.
+   */
+  public enum ScoreFunction {
+    /** No distance function is used. Note: {@link VectorValues.RandomAccess#search(float[], int, int)}
+     * is not supported for fields specifying this score function. */
+    NONE(0),
+
+    /** Euclidean distance */
+    EUCLIDEAN(1, true) {
+      @Override
+      public float score(float[] v1, float[] v2) {
+        assert v1.length == v2.length;
+        float squareSum = 0.0f;
+        int dim = v1.length;
+        for (int i = 0; i < dim; i++) {
+          float diff = v1[i] - v2[i];
+          squareSum += diff * diff;
+        }
+        return squareSum;
+      }
+    },
+
+    /** dot product - note, may be negative; larger values are better */
+    DOT_PRODUCT(2) {
+      @Override
+      public float score(float[] a, float[] b) {
+        float res = 0f;
+        /*
+         * If length of vector is larger than 8, we use unrolled dot product to accelerate the
+         * calculation.
+         */
+        int i;
+        for (i = 0; i < a.length % 8; i++) {
+            res += b[i] * a[i];
+        }
+        if (a.length < 8) {
+            return res;
+        }
+        float s0 = 0f;
+        float s1 = 0f;
+        float s2 = 0f;
+        float s3 = 0f;
+        float s4 = 0f;
+        float s5 = 0f;
+        float s6 = 0f;
+        float s7 = 0f;
+        for (; i + 7 < a.length; i += 8) {
+            s0 += b[i] * a[i];
+            s1 += b[i + 1] * a[i + 1];
+            s2 += b[i + 2] * a[i + 2];
+            s3 += b[i + 3] * a[i + 3];
+            s4 += b[i + 4] * a[i + 4];
+            s5 += b[i + 5] * a[i + 5];
+            s6 += b[i + 6] * a[i + 6];
+            s7 += b[i + 7] * a[i + 7];
+        }
+        res += s0 + s1 + s2 + s3 + s4 + s5 + s6 + s7;
+        return res;
+      }
+    };
+    
+    /** ID for each enum value; this is persisted to the index and cannot be changed after indexing. */
+    final public int id;
+
+    /** If reversed, smaller values are better */
+    final public boolean reversed;
+
+    ScoreFunction(int id, boolean reversed) {
+      this.id = id;
+      this.reversed = reversed;
+    }
+
+    ScoreFunction(int id) {
+      this(id, false);
+    }
+
+    /**
+     * Calculates the score between the specified two vectors.
+     */
+    public float score(float[] v1, float[] v2) {
+      throw new UnsupportedOperationException();
+    }
+
+    /**
+     * Returns the distance function that is specified by the id.
+     */
+    public static ScoreFunction fromId(int id) {
+      for (ScoreFunction d : ScoreFunction.values()) {
+        if (d.id == id) {
+          return d;
+        }
+      }
+      throw new IllegalArgumentException("no such distance function with id " + id);
+    }
+  }
+
+   /**
+   * Calculates a similarity score between the two vectors with specified function.
+   */
+  public static float compare(float[] v1, float[] v2, ScoreFunction scoreFunction) {
+    assert v1.length == v2.length : "attempt to compare vectors of lengths: " + v1.length + " " + v2.length;
+    return scoreFunction.score(v1, v2);
+  }
+
+  /**
+   * Represents the lack of vector values. It is returned by providers that do not
+   * support VectorValues.
+   */
+  public static VectorValues EMPTY = new VectorValues() {
+
+    @Override
+    public int size() {
+      return 0;
+    }
+
+    @Override
+    public int dimension() {
+      return 0;
+    }
+
+    @Override
+    public ScoreFunction scoreFunction() {
+      return ScoreFunction.NONE;
+    }
+
+    @Override
+    public float[] vectorValue() {
+      throw new IndexOutOfBoundsException("Attempt to get vectors from EMPTY values");

Review comment:
       Maybe `IllegalStateException` instead?  I.e. caller tried to get the vector value without first advancing successfully...

##########
File path: lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java
##########
@@ -98,15 +99,8 @@ MergeState merge() throws IOException {
       throw new IllegalStateException("Merge would result in 0 document segment");
     }
     mergeFieldInfos();
-    long t0 = 0;
-    if (mergeState.infoStream.isEnabled("SM")) {
-      t0 = System.nanoTime();
-    }
-    int numMerged = mergeFields();
-    if (mergeState.infoStream.isEnabled("SM")) {
-      long t1 = System.nanoTime();
-      mergeState.infoStream.message("SM", ((t1-t0)/1000000) + " msec to merge stored fields [" + numMerged + " docs]");
-    }
+
+    int numMerged = mergeWithLogging(this::mergeFields, "stored fields");

Review comment:
       Whoa, cool!  Thank you for factoring out this new `mergeWithLogging`...

##########
File path: lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosFormat.java
##########
@@ -67,6 +68,8 @@
   static final BytesRef DATA_DIM_COUNT  =  new BytesRef("  data dimensional count ");
   static final BytesRef INDEX_DIM_COUNT =  new BytesRef("  index dimensional count ");
   static final BytesRef DIM_NUM_BYTES   =  new BytesRef("  dimensional num bytes ");
+  static final BytesRef VECTOR_NUM_DIMS =  new BytesRef("  vector number of dimensions ");

Review comment:
       Thank you for adding `SimpleText` support!  This is an awesome tool for debugging ... it lets you see the whole index in a simple human readable (but incredibly inefficient for the machines!!!) text readable form.

##########
File path: lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene84/Lucene84Codec.java
##########
@@ -136,6 +137,11 @@ public PointsFormat pointsFormat() {
     return new Lucene60PointsFormat();
   }
 
+  @Override
+  public VectorFormat vectorFormat() {
+    return null;

Review comment:
       Why not also `VectorFormat.EMPTY` here?




----------------------------------------------------------------
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.

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


[GitHub] [lucene-solr] msokolov commented on pull request #1930: LUCENE-9322: add VectorValues to new Lucene90 codec

Posted by GitBox <gi...@apache.org>.
msokolov commented on pull request #1930:
URL: https://github.com/apache/lucene-solr/pull/1930#issuecomment-712104682


   @jtibshirani I am in sympathy with your points; the scoring implementations don't really belong in VectorValues which is more of an abstract interface otherwise, and random-access API should really be an internal detail of the HNSW implementation. I did fiddle with ideas for hiding the RandomAccess API but see my comments about visibility in the thread above; I'm not yet sure what the best way is. Probably it will be easiest to address these as part of the HNSW patch, as you say, but if you have specific ideas, please feel free to post a PR!


----------------------------------------------------------------
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.

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


[GitHub] [lucene-solr] jtibshirani edited a comment on pull request #1930: LUCENE-9322: add VectorValues to new Lucene90 codec

Posted by GitBox <gi...@apache.org>.
jtibshirani edited a comment on pull request #1930:
URL: https://github.com/apache/lucene-solr/pull/1930#issuecomment-706813811


   > Hmm I tried to get that benchmarking suite to run and it requires some major Python-fu.
   
   I managed to get this working a few months ago while experimenting with a clustering-based approach: https://github.com/jtibshirani/ann-benchmarks/pull/2. It indeed involved a lot of set-up -- I can try to get it working again and post results. Going forward, I think it will be helpful to use ann-benchmarks to compare recall and QPS against the ANN reference implementations.


----------------------------------------------------------------
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.

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


[GitHub] [lucene-solr] msokolov commented on a change in pull request #1930: LUCENE-9322: add VectorValues to new Lucene90 codec

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #1930:
URL: https://github.com/apache/lucene-solr/pull/1930#discussion_r501828351



##########
File path: lucene/core/src/java/org/apache/lucene/index/VectorValues.java
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.index;
+
+import java.io.IOException;
+
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Access to per-document vector value.
+ */
+public abstract class VectorValues extends DocIdSetIterator {
+
+  /** The maximum length of a vector */
+  public static int MAX_DIMENSIONS = 1024;
+
+  /** Sole constructor */
+  protected VectorValues() {}
+
+  /**
+   * Return the dimension of the vectors
+   */
+  public abstract int dimension();
+
+  /**
+   * TODO: should we use cost() for this? We rely on its always being exactly the number
+   * of documents having a value for this field, which is not guaranteed by the cost() contract,
+   * but in all the implementations so far they are the same.
+   * @return the number of vectors returned by this iterator
+   */
+  public abstract int size();
+
+  /**
+   * Return the score function used to compare these vectors
+   */
+  public abstract ScoreFunction scoreFunction();
+
+  /**
+   * Return the vector value for the current document ID.
+   * It is illegal to call this method after the iterator failed to advance.
+   * @return the vector value
+   */
+  public abstract float[] vectorValue() throws IOException;
+
+  /**
+   * Return the binary encoded vector value for the current document ID.
+   * It is illegal to call this method after the iterator failed to advance.
+   * @return the binary value
+   */
+  public BytesRef binaryValue() throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  /**
+   * Return a random access interface over this iterator's vectors.
+   */
+  public abstract RandomAccess randomAccess();
+
+  /**
+   * Provides random access to vectors by dense ordinal
+   */
+  public interface RandomAccess {
+
+    /**
+     * Return the vector value as a floating point array.
+     * @param targetOrd a valid ordinal, &ge; 0 and &lt; {@link #size()}.
+     */
+    float[] vectorValue(int targetOrd) throws IOException;
+
+    /**
+     * Return the vector value as a byte array; these are the bytes corresponding to the float array
+     * encoded using little-endian byte order.
+     * @param targetOrd a valid ordinal, &ge; 0 and &lt; {@link #size()}.
+     */
+    BytesRef binaryValue(int targetOrd) throws IOException;
+
+    /**
+     * Return the k nearest neighbor documents as determined by comparison of their vector values
+     * for this field, to the given vector, by the field's score function. If the score function is
+     * reversed, lower values indicate nearer vectors, otherwise higher scores indicate nearer
+     * vectors. Unlike relevance scores, vector scores may be negative.
+     * @param target the vector-valued query
+     * @param k      the number of docs to return
+     * @param fanout control the accuracy/speed tradeoff - larger values give better recall at higher cost
+     * @return the k nearest neighbor documents, along with their (scoreFunction-specific) scores.
+     */
+    TopDocs search(float[] target, int k, int fanout) throws IOException;
+  }
+
+  /**
+   * Score function. This is used during indexing and searching of the vectors to determine the nearest neighbors.
+   * Score values may be negative. By default high scores indicate nearer documents, unless the function is reversed.
+   */
+  public enum ScoreFunction {
+    /** No distance function is used. Note: {@link VectorValues.RandomAccess#search(float[], int, int)}

Review comment:
       OK I opened LUCENE-9573. I have to admit I don't fully understand the timing constraints/dependencies here. Maybe you could comment on that issue?  Re: the ids I opted to move to using the enum ordinal as you suggested later. I can't see how that restricts us in any meaningful way. Perhaps we add a back-compat test to verify that the enum ordinals don't change




----------------------------------------------------------------
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.

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


[GitHub] [lucene-solr] msokolov commented on a change in pull request #1930: LUCENE-9322: add VectorValues to new Lucene90 codec

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #1930:
URL: https://github.com/apache/lucene-solr/pull/1930#discussion_r506677596



##########
File path: lucene/core/src/java/org/apache/lucene/index/VectorValues.java
##########
@@ -0,0 +1,273 @@
+/*
+ * 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.index;
+
+import java.io.IOException;
+
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * This class provides access to per-document floating point vector values indexed as {@link
+ * org.apache.lucene.document.VectorField}.
+ */
+public abstract class VectorValues extends DocIdSetIterator {
+
+  /** The maximum length of a vector */
+  public static int MAX_DIMENSIONS = 1024;
+
+  /** Sole constructor */
+  protected VectorValues() {}
+
+  /**
+   * Return the dimension of the vectors
+   */
+  public abstract int dimension();
+
+  /**
+   * TODO: should we use cost() for this? We rely on its always being exactly the number
+   * of documents having a value for this field, which is not guaranteed by the cost() contract,
+   * but in all the implementations so far they are the same.
+   * @return the number of vectors returned by this iterator
+   */
+  public abstract int size();
+
+  /**
+   * Return the score function used to compare these vectors
+   */
+  public abstract ScoreFunction scoreFunction();
+
+  /**
+   * Return the vector value for the current document ID.
+   * It is illegal to call this method when the iterator is not positioned: before advancing, or after failing to advance.
+   * The returned array may be shared across calls, re-used, and modified as the iterator advances.
+   * @return the vector value
+   */
+  public abstract float[] vectorValue() throws IOException;

Review comment:
       If you are iterating, you can keep track by counting. On the other hand, you wouldn't need to since the only purpose for the ordinal is to retrieve the vector that you already have. So I think getting the ordinal for  the *current* docId is not really all that helpful.
   
   In theory one might want to get an ordinal for some arbitrary docId. I guess the API is sort of incomplete without it - it offers random access, but only by an opaque ordinal. You can of course iterate over all the docs and build your own map, but that is kind of unhelpful.
   
   However supporting this comes at some additional cost. It's not required to support knnSearch since the way that works is to search for the best ordinal and then map that to a docid, but internally we do not maintain any docid->ordinal mapping. We can get an answer using binary search in the ordToDoc map, but I wonder if we should expose that. WDYT?




----------------------------------------------------------------
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.

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


[GitHub] [lucene-solr] msokolov edited a comment on pull request #1930: LUCENE-9322: add VectorValues to new Lucene90 codec

Posted by GitBox <gi...@apache.org>.
msokolov edited a comment on pull request #1930:
URL: https://github.com/apache/lucene-solr/pull/1930#issuecomment-703872279


   > Thank you for ... the tests catching mis-use where user tries to change dimension or scoring function in an existing field.
   
   Thanks to @mocobeta for those; I was able to carry that forward from her earlier patch
   
   > I see you implemented the two score functions, but are they ever exercised in tests
   
   True - this was extracted from a bigger change including usage of those methods as part of KNN search, but they deserve their own unit tests - I'll add.
   
   > I would love to see a "Vector Overview" javadoc somewhere ...
   
   Yes - I'll add to the VectorValues/VectorField class javadocs I think that's the most natural/visible place.
   
   > I am curious how the basic vector usage performs -- just indexing one vector field, and retrieving it at search time. We can (separately) enable luceneutil to support testing vectors, somehow. But I wonder where we'll get semi-realistic vectors derived from Wikipedia content 
   
   Agreed that benchmarking is needed. I think we can use http://ann-benchmarks.com/ as a guide for some standardized test vectors. They won't be related to wikipedia? If we get to wanting that, we could also make use of something like https://fasttext.cc/docs/en/pretrained-vectors.html that is trained on ngrams taken from Wikipedia (for many languages)? I don't know how suited it is, just found in a google search. For that, we'd have to compute document/query vectors based on an ngram-vector dictionary. I think a simple thing is to sum all the ngram-vectors for all the ngrams in a document / query


----------------------------------------------------------------
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.

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


[GitHub] [lucene-solr] msokolov commented on pull request #1930: LUCENE-9322: add VectorValues to new Lucene90 codec

Posted by GitBox <gi...@apache.org>.
msokolov commented on pull request #1930:
URL: https://github.com/apache/lucene-solr/pull/1930#issuecomment-710028939


   So .. this has been out for 17 days now - I'd like to commit in the next day or so in order to enable moving forward with the matching piece of this, adding some benchmarking to luceneutil that depends on it, etc. @mikemccand do you want to have another look? - I think I've addressed your comments.


----------------------------------------------------------------
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.

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


[GitHub] [lucene-solr] msokolov commented on pull request #1930: LUCENE-9322: add VectorValues to new Lucene90 codec

Posted by GitBox <gi...@apache.org>.
msokolov commented on pull request #1930:
URL: https://github.com/apache/lucene-solr/pull/1930#issuecomment-711171187


   I squashed, resolved conflicts and pushed using command line


----------------------------------------------------------------
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.

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


[GitHub] [lucene-solr] jtibshirani edited a comment on pull request #1930: LUCENE-9322: add VectorValues to new Lucene90 codec

Posted by GitBox <gi...@apache.org>.
jtibshirani edited a comment on pull request #1930:
URL: https://github.com/apache/lucene-solr/pull/1930#issuecomment-707404485


   >  It seems to expect your algorithm to be delivered as an in-process extension to Python, which works OK for a native code library, but I'm not sure how we'd present Lucene to it. We don't want to have to call through a network API?
   
   I ended up using `py4j` to call out to Lucene, which sets up a 'gateway server' and passes data between the Python + Java processes through a socket. I found there to be a significant overhead from converting between Python <-> Java, but this can be largely mitigated by making sure to use 'batch mode' (the `--batch` option), which allows all query vectors to be passed to Lucene at once. Amortizing the overhead this way, I was able to get consistent + informative results. Let me know if you're interested in trying the py4j option and I can post set-up steps. I found it helpful while developing but it's quite tricky and maybe shouldn't be the main way to track performance right now (as you mentioned) !
   
   A note that it's possible to use vector data from ann-benchmarks without integrating with the framework. The datasets are listed [here](https://github.com/erikbern/ann-benchmarks/blob/master/ann_benchmarks/datasets.py#L396) and made available on the website in hdf5 format. One option could be 200-dimensional GloVe word vectors, available from `http://ann-benchmarks.com/glove-200-angular.hdf5`. I think these are trained on Twitter data.


----------------------------------------------------------------
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.

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


[GitHub] [lucene-solr] msokolov commented on a change in pull request #1930: LUCENE-9322: add VectorValues to new Lucene90 codec

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #1930:
URL: https://github.com/apache/lucene-solr/pull/1930#discussion_r499862666



##########
File path: lucene/core/src/java/org/apache/lucene/document/FieldType.java
##########
@@ -351,6 +356,27 @@ public int pointNumBytes() {
     return dimensionNumBytes;
   }
 
+  void setVectorDimensionsAndScoreFunction(int numDimensions, VectorValues.ScoreFunction distFunc) {
+    if (numDimensions <= 0) {
+      throw new IllegalArgumentException("vector numDimensions must be > 0; got " + numDimensions);
+    }
+    if (numDimensions > VectorValues.MAX_DIMENSIONS) {
+      throw new IllegalArgumentException("vector numDimensions must be <= VectorValues.MAX_DIMENSIONS (=" + VectorValues.MAX_DIMENSIONS + "); got " + numDimensions);
+    }
+    this.vectorDimension = numDimensions;

Review comment:
       hmm I noticed we do not do this for Points. I think that in practice we do not expect users to create these FieldTypes - they are created implicitly when adding values using VectorField constructors - we create a new Type for every Field unless the user cleverly creates a generic Field with a float[] as its `fieldsData` and an appropriate FieldType. Efficient usage would seem to be: create a VectorField, add it to a doc, update its value and repeat. Anyway I'll add the checking




----------------------------------------------------------------
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.

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


[GitHub] [lucene-solr] msokolov commented on a change in pull request #1930: LUCENE-9322: add VectorValues to new Lucene90 codec

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #1930:
URL: https://github.com/apache/lucene-solr/pull/1930#discussion_r506623857



##########
File path: lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90VectorReader.java
##########
@@ -0,0 +1,344 @@
+/*
+ * 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.lucene90;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.FloatBuffer;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.VectorReader;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.VectorValues;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.RamUsageEstimator;
+
+/**
+ * Reads vectors from the index segments.
+ */

Review comment:
       ok; tagging experimental here and the other classes you mentioned. All the public classes I guess? Should we also tag o.a.l.codecs.VectorReader/Writer and o.a.l.document/VectorField




----------------------------------------------------------------
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.

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


[GitHub] [lucene-solr] muse-dev[bot] commented on a change in pull request #1930: LUCENE-9322: add VectorValues to new Lucene90 codec

Posted by GitBox <gi...@apache.org>.
muse-dev[bot] commented on a change in pull request #1930:
URL: https://github.com/apache/lucene-solr/pull/1930#discussion_r501886824



##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexingChain.java
##########
@@ -562,6 +614,12 @@ private int processField(int docID, IndexableField field, long fieldGen, int fie
       }
       indexPoint(docID, fp, field);
     }
+    if (fieldType.vectorDimension() != 0) {
+      if (fp == null) {
+        fp = getOrAddField(fieldName, fieldType, false);
+      }
+      indexVector(docID, fp, field);

Review comment:
       *NULL_DEREFERENCE:*  object `fp` last assigned on line 619 could be null and is dereferenced by call to `indexVector(...)` at line 621.




----------------------------------------------------------------
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.

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


[GitHub] [lucene-solr] msokolov commented on pull request #1930: LUCENE-9322: add VectorValues to new Lucene90 codec

Posted by GitBox <gi...@apache.org>.
msokolov commented on pull request #1930:
URL: https://github.com/apache/lucene-solr/pull/1930#issuecomment-706185147


   > Agreed that benchmarking is needed. I think we can use http://ann-benchmarks.com/ as a guide for some standardized test vectors.
   
   Hmm I tried to get that benchmarking suite to run and it requires some major Python-fu. That package relies on docker, scipy, scikit-learn, h5py, matplotlib, and these in turn rely on a lot of native libraries, and all the versions have to be just right. I didn't have the right versions in my package manager's repos, so I had to install from source, and was never able to get the right combination, so I finally just gave up on that approach. Maybe someday we can use it to compare the performance of this solution with SOA native libraries, but not today!
   
   I'll try having a look at the Wikipedia-derived vectors to see if we can at least develop our own internal benchmarks in luceneutil.


----------------------------------------------------------------
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.

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


[GitHub] [lucene-solr] jtibshirani commented on pull request #1930: LUCENE-9322: add VectorValues to new Lucene90 codec

Posted by GitBox <gi...@apache.org>.
jtibshirani commented on pull request #1930:
URL: https://github.com/apache/lucene-solr/pull/1930#issuecomment-711759394


   @msokolov I did a belated review and had some high-level thoughts. Hopefully post-commit reviews are still helpful in giving ideas for follow-ups.
   
   > It seems messy to have the ScoringFunction implementation in the main VectorValues interface API file. 
   
   It seems like we’re using `ScoreFunction` for two separate use cases:
   1. Configuring the ANN indexing + search strategy (to be introduced later). For example, HNSW needs to know up-front what similarity measure to use when building the graph.
   2. Providing convenience methods for users that want to incorporate vectors into scoring, outside of ANN.
   
   We could think about separating out these cases. Maybe `ScoreFunction` could stick closely to case 1 and avoid exposing score implementations. This fits well with its role of specifying the ANN index strategy, which might encompass other ANN approaches in the future like `KMEANS_EUCLIDEAN`. And for use case 2, we could introduce static utility methods like `dotProduct(float[] a, float[] b)` ? The ANN implementations would be free to make use of these methods if they prove helpful. If we go with this design, perhaps `ScoreFunction` would be renamed to something like `SearchStrategy`.
   
   Separately, I wonder if we need to expose a random access interface on the public `VectorValues`. When using vectors for scoring, iterator access seems sufficient. I’m having trouble thinking of a use for random access outside of ANN, and some ANN implementations don’t even need it -- for example most k-means and LSH approaches only need forward iteration. Would it make sense to keep the `VectorValues` surface area smaller and only provide iterator access? Random access could still be supported at a lower level, in a place that's available to HNSW. I’m not sure exactly how this would look, perhaps it'd be clearer once we see how HNSW will be integrated.


----------------------------------------------------------------
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.

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


[GitHub] [lucene-solr] msokolov commented on a change in pull request #1930: LUCENE-9322: add VectorValues to new Lucene90 codec

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #1930:
URL: https://github.com/apache/lucene-solr/pull/1930#discussion_r499891879



##########
File path: lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90VectorReader.java
##########
@@ -0,0 +1,326 @@
+/*
+ * 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.lucene90;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.FloatBuffer;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.VectorReader;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.VectorValues;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.RamUsageEstimator;
+
+/**
+ * Reads vectors from the index segments.
+ */
+public final class Lucene90VectorReader extends VectorReader {
+
+  private final FieldInfos fieldInfos;
+  private final Map<String, FieldEntry> fields = new HashMap<>();
+  private final IndexInput vectorData;
+  private final int maxDoc;
+
+  Lucene90VectorReader(SegmentReadState state) throws IOException {
+    this.fieldInfos = state.fieldInfos;
+    this.maxDoc = state.segmentInfo.maxDoc();
+
+    String metaFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Lucene90VectorFormat.META_EXTENSION);
+    int versionMeta = -1;
+    try (ChecksumIndexInput meta = state.directory.openChecksumInput(metaFileName, state.context)) {
+      Throwable priorE = null;
+      try {
+        versionMeta = CodecUtil.checkIndexHeader(meta,
+            Lucene90VectorFormat.META_CODEC_NAME,
+            Lucene90VectorFormat.VERSION_START,
+            Lucene90VectorFormat.VERSION_CURRENT,
+            state.segmentInfo.getId(),
+            state.segmentSuffix);
+        readFields(meta, state.fieldInfos);
+      } catch (Throwable exception) {
+        priorE = exception;
+      } finally {
+        CodecUtil.checkFooter(meta, priorE);
+      }
+    }
+
+    boolean success = false;
+
+    String vectorDataFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Lucene90VectorFormat.VECTOR_DATA_EXTENSION);
+    this.vectorData = state.directory.openInput(vectorDataFileName, state.context);
+    try {
+      int versionVectorData = CodecUtil.checkIndexHeader(vectorData,
+          Lucene90VectorFormat.VECTOR_DATA_CODEC_NAME,
+          Lucene90VectorFormat.VERSION_START,
+          Lucene90VectorFormat.VERSION_CURRENT,
+          state.segmentInfo.getId(),
+          state.segmentSuffix);
+      if (versionMeta != versionVectorData) {
+        throw new CorruptIndexException("Format versions mismatch: meta=" + versionMeta + ", vector data=" + versionVectorData, vectorData);
+      }
+      CodecUtil.retrieveChecksum(vectorData);
+
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(this.vectorData);
+      }
+    }
+  }
+
+  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);
+      }
+      VectorValues.ScoreFunction scoreFunction = VectorValues.ScoreFunction.fromId(meta.readInt());
+      long vectorDataOffset = meta.readVLong();
+      long vectorDataLength = meta.readVLong();
+      int dimension = meta.readInt();
+      int size = meta.readInt();
+      int[] ordToDoc = new int[size];
+      for (int i = 0; i < size; i++) {
+        int doc = meta.readVInt();
+        ordToDoc[i] = doc;
+      }
+      FieldEntry fieldEntry = new FieldEntry(dimension, scoreFunction, maxDoc, vectorDataOffset, vectorDataLength,
+                                              ordToDoc);
+      fields.put(info.name, fieldEntry);
+    }
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    long totalBytes = RamUsageEstimator.shallowSizeOfInstance(Lucene90VectorReader.class);
+    totalBytes += RamUsageEstimator.sizeOfMap(fields, RamUsageEstimator.shallowSizeOfInstance(FieldEntry.class));
+    for (FieldEntry entry : fields.values()) {
+      totalBytes += RamUsageEstimator.sizeOf(entry.ordToDoc);
+    }
+    return totalBytes;
+  }
+
+  @Override
+  public void checkIntegrity() throws IOException {
+    CodecUtil.checksumEntireFile(vectorData);
+  }
+
+  @Override
+  public VectorValues getVectorValues(String field) throws IOException {
+    FieldInfo info = fieldInfos.fieldInfo(field);
+    if (info == null) {
+      return null;
+    }
+    int dimension = info.getVectorDimension();
+    if (dimension == 0) {
+      return VectorValues.EMPTY;
+    }
+    FieldEntry fieldEntry = fields.get(field);
+    if (fieldEntry == null) {
+      // There is a FieldInfo, but no vectors. Should we have deleted the FieldInfo?
+      return null;
+    }
+    if (dimension != fieldEntry.dimension) {
+      throw new IllegalStateException("Inconsistent vector dimension for field=\"" + field + "\"; " + dimension + " != " + fieldEntry.dimension);
+    }
+    long numBytes = (long) fieldEntry.size() * dimension * Float.BYTES;
+    if (numBytes != fieldEntry.vectorDataLength) {
+      throw new IllegalStateException("Vector data length " + fieldEntry.vectorDataLength +
+          " not matching size=" + fieldEntry.size() + " * dim=" + dimension + " * 4 = " +
+          numBytes);
+    }
+    IndexInput bytesSlice = vectorData.slice("vector-data", fieldEntry.vectorDataOffset, fieldEntry.vectorDataLength);
+    return new OffHeapVectorValues(fieldEntry, bytesSlice);
+  }
+
+  @Override
+  public void close() throws IOException {
+    vectorData.close();
+  }
+
+  private static class FieldEntry {
+
+    final int dimension;
+    final VectorValues.ScoreFunction scoreFunction;
+    final int maxDoc;
+
+    final long vectorDataOffset;
+    final long vectorDataLength;
+    final int[] ordToDoc;
+
+    FieldEntry(int dimension, VectorValues.ScoreFunction scoreFunction, int maxDoc,
+               long vectorDataOffset, long vectorDataLength, int[] ordToDoc) {
+      this.dimension = dimension;
+      this.scoreFunction = scoreFunction;
+      this.maxDoc = maxDoc;
+      this.vectorDataOffset = vectorDataOffset;
+      this.vectorDataLength = vectorDataLength;
+      this.ordToDoc = ordToDoc;
+    }
+
+    int size() {
+      return ordToDoc.length;
+    }
+  }
+
+  /** Read the vector values from the index input. This supports both iterated and random access. */
+  private final static class OffHeapVectorValues extends VectorValues {
+
+    final FieldEntry fieldEntry;
+    final IndexInput dataIn;
+
+    final BytesRef binaryValue;
+    final ByteBuffer byteBuffer;
+    final FloatBuffer floatBuffer;
+    final int byteSize;
+    final float[] value;
+
+    int ord = -1;
+    int doc = -1;
+
+    OffHeapVectorValues(FieldEntry fieldEntry, IndexInput dataIn) {
+      this.fieldEntry = fieldEntry;
+      this.dataIn = dataIn;
+      byteSize = Float.BYTES * fieldEntry.dimension;
+      byteBuffer = ByteBuffer.allocate(byteSize);
+      floatBuffer = byteBuffer.asFloatBuffer();
+      value = new float[fieldEntry.dimension];
+      binaryValue = new BytesRef(byteBuffer.array(), byteBuffer.arrayOffset(), byteSize);
+    }
+
+    @Override
+    public int dimension() {
+      return fieldEntry.dimension;
+    }
+
+    @Override
+    public int size() {
+      return fieldEntry.size();
+    }
+
+    @Override
+    public ScoreFunction scoreFunction() {
+      return fieldEntry.scoreFunction;
+    }
+
+    @Override
+    public float[] vectorValue() throws IOException {
+      dataIn.seek(ord * byteSize);
+      dataIn.readBytes(byteBuffer.array(), byteBuffer.arrayOffset(), byteSize);
+      floatBuffer.position(0);
+      floatBuffer.get(value, 0, fieldEntry.dimension);
+      return value;
+    }
+
+    @Override
+    public BytesRef binaryValue() throws IOException {
+      dataIn.seek(ord * byteSize);
+      dataIn.readBytes(byteBuffer.array(), byteBuffer.arrayOffset(), byteSize);
+      return binaryValue;
+    }
+
+    @Override
+    public int docID() {
+      return doc;
+    }
+
+    @Override
+    public int nextDoc() {
+      if (++ord >= size()) {
+        doc = NO_MORE_DOCS;
+      } else {
+        doc = fieldEntry.ordToDoc[ord];
+      }
+      return doc;
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      // We could do better by log-binary search in ordToDoc, but this is never used

Review comment:
       Yes, the doc-> ord relation is monotonic, ie order-preserving s.t. `doc1 > doc2 iff ord(doc1) > ord(doc2)`. We use this fact in the implementation of random-access over a sorted index while merging so I don't think we're free to share vectors like that (with this implementation). I also plan to make use of that invariant when building NSW graphs - we want to store the ords in sorted order, and if we generate the graph while indexing, then they will naturally be added in order (since their order corresponds to the docid order) with no need to sort them at any point.
   
   I tend to think it's unlikely that you would often see duplicated floating-point vectors aside from zeros, although of course YMMV. I think we will want to allow for lossy compression using byte-per-dimension rather than float-per-dimension, but I am hard-pressed to see what kind of lossless compression we could provide. Well I see there is stuff out there: https://computing.llnl.gov/projects/floating-point-compression , but I expect it would require some additional index structure to enable random access




----------------------------------------------------------------
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.

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


[GitHub] [lucene-solr] jtibshirani commented on pull request #1930: LUCENE-9322: add VectorValues to new Lucene90 codec

Posted by GitBox <gi...@apache.org>.
jtibshirani commented on pull request #1930:
URL: https://github.com/apache/lucene-solr/pull/1930#issuecomment-707404485


   >  It seems to expect your algorithm to be delivered as an in-process extension to Python, which works OK for a native code library, but I'm not sure how we'd present Lucene to it. We don't want to have to call through a network API?
   
   I ended up using `py4j` to call out to Lucene, which sets up a 'gateway server' and passes data between the Python + Java processes through a socket. I did find there to be a significant overhead just from converting between Python <-> Java, but can be largely mitigated by making sure to use 'batch mode' (the `--batch` option), which allows all query vectors to be passed to Lucene at once. Amortizing the overhead this way, I was able to get consistent + informative results. Let me know if you're interested in trying the py4j option and I can post set-up steps. I found it helpful while developing but it's quite tricky and maybe shouldn't be the main way to track performance right now (as you mentioned) !
   
   A note that it's possible to use vector data from ann-benchmarks without integrating with the framework. The datasets are listed [here](https://github.com/erikbern/ann-benchmarks/blob/master/ann_benchmarks/datasets.py#L396) and made available on the website in hdf5 format. One option could be 200-dimensional GloVe word vectors, available from `http://ann-benchmarks.com/glove-200-angular.hdf5`. I think these are trained on Twitter data.


----------------------------------------------------------------
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.

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


[GitHub] [lucene-solr] msokolov commented on a change in pull request #1930: LUCENE-9322: add VectorValues to new Lucene90 codec

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #1930:
URL: https://github.com/apache/lucene-solr/pull/1930#discussion_r501828351



##########
File path: lucene/core/src/java/org/apache/lucene/index/VectorValues.java
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.index;
+
+import java.io.IOException;
+
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Access to per-document vector value.
+ */
+public abstract class VectorValues extends DocIdSetIterator {
+
+  /** The maximum length of a vector */
+  public static int MAX_DIMENSIONS = 1024;
+
+  /** Sole constructor */
+  protected VectorValues() {}
+
+  /**
+   * Return the dimension of the vectors
+   */
+  public abstract int dimension();
+
+  /**
+   * TODO: should we use cost() for this? We rely on its always being exactly the number
+   * of documents having a value for this field, which is not guaranteed by the cost() contract,
+   * but in all the implementations so far they are the same.
+   * @return the number of vectors returned by this iterator
+   */
+  public abstract int size();
+
+  /**
+   * Return the score function used to compare these vectors
+   */
+  public abstract ScoreFunction scoreFunction();
+
+  /**
+   * Return the vector value for the current document ID.
+   * It is illegal to call this method after the iterator failed to advance.
+   * @return the vector value
+   */
+  public abstract float[] vectorValue() throws IOException;
+
+  /**
+   * Return the binary encoded vector value for the current document ID.
+   * It is illegal to call this method after the iterator failed to advance.
+   * @return the binary value
+   */
+  public BytesRef binaryValue() throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  /**
+   * Return a random access interface over this iterator's vectors.
+   */
+  public abstract RandomAccess randomAccess();
+
+  /**
+   * Provides random access to vectors by dense ordinal
+   */
+  public interface RandomAccess {
+
+    /**
+     * Return the vector value as a floating point array.
+     * @param targetOrd a valid ordinal, &ge; 0 and &lt; {@link #size()}.
+     */
+    float[] vectorValue(int targetOrd) throws IOException;
+
+    /**
+     * Return the vector value as a byte array; these are the bytes corresponding to the float array
+     * encoded using little-endian byte order.
+     * @param targetOrd a valid ordinal, &ge; 0 and &lt; {@link #size()}.
+     */
+    BytesRef binaryValue(int targetOrd) throws IOException;
+
+    /**
+     * Return the k nearest neighbor documents as determined by comparison of their vector values
+     * for this field, to the given vector, by the field's score function. If the score function is
+     * reversed, lower values indicate nearer vectors, otherwise higher scores indicate nearer
+     * vectors. Unlike relevance scores, vector scores may be negative.
+     * @param target the vector-valued query
+     * @param k      the number of docs to return
+     * @param fanout control the accuracy/speed tradeoff - larger values give better recall at higher cost
+     * @return the k nearest neighbor documents, along with their (scoreFunction-specific) scores.
+     */
+    TopDocs search(float[] target, int k, int fanout) throws IOException;
+  }
+
+  /**
+   * Score function. This is used during indexing and searching of the vectors to determine the nearest neighbors.
+   * Score values may be negative. By default high scores indicate nearer documents, unless the function is reversed.
+   */
+  public enum ScoreFunction {
+    /** No distance function is used. Note: {@link VectorValues.RandomAccess#search(float[], int, int)}

Review comment:
       OK I opened LUCENE-9573. I have to admit I don't fully understand the timing constraints/dependencies here. Maybe you could comment on that issue?  Re: the ids I opted to move to using the enum ordinal as you suggested later. I can't see how that restricts us in any meaningful way. Perhaps we add a back-compat test to verify that the enum ordinals don't change




----------------------------------------------------------------
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.

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


[GitHub] [lucene-solr] msokolov commented on a change in pull request #1930: LUCENE-9322: add VectorValues to new Lucene90 codec

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #1930:
URL: https://github.com/apache/lucene-solr/pull/1930#discussion_r506729885



##########
File path: lucene/core/src/test/org/apache/lucene/index/TestVectorValues.java
##########
@@ -0,0 +1,650 @@
+/*
+ * 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.index;
+
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.document.VectorField;
+import org.apache.lucene.index.VectorValues.ScoreFunction;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.FSDirectory;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+/** Test Indexing/IndexWriter with vectors */
+public class TestVectorValues extends LuceneTestCase {
+
+  private IndexWriterConfig createIndexWriterConfig() {
+    IndexWriterConfig iwc = newIndexWriterConfig();
+    iwc.setCodec(Codec.forName("Lucene90"));
+    return iwc;
+  }
+
+  // Suddenly add vectors to an existing field:
+  public void testUpgradeFieldToVectors() throws Exception {
+    try (Directory dir = newDirectory()) {
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(newStringField("dim", "foo", Store.NO));
+        w.addDocument(doc);
+      }
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+        w.addDocument(doc);
+      }
+    }
+  }
+
+  public void testFieldConstructor() {
+    float[] v = new float[1];
+    VectorField field = new VectorField("f", v);
+    assertEquals(1, field.fieldType().vectorDimension());
+    assertEquals(ScoreFunction.EUCLIDEAN, field.fieldType().vectorScoreFunction());
+    assertSame(v, field.vectorValue());
+  }
+
+  public void testFieldConstructorExceptions() {
+    expectThrows(IllegalArgumentException.class, () -> new VectorField(null, new float[1]));
+    expectThrows(IllegalArgumentException.class, () -> new VectorField("f", null));
+    expectThrows(IllegalArgumentException.class, () -> new VectorField("f", new float[1], null));
+    expectThrows(IllegalArgumentException.class, () -> new VectorField("f", new float[0]));
+    expectThrows(IllegalArgumentException.class, () -> new VectorField("f", new float[VectorValues.MAX_DIMENSIONS + 1]));
+  }
+
+  public void testFieldSetValue() {
+    VectorField field = new VectorField("f", new float[1]);
+    float[] v1 = new float[1];
+    field.setVectorValue(v1);
+    assertSame(v1, field.vectorValue());
+    expectThrows(IllegalArgumentException.class, () -> field.setVectorValue(new float[2]));
+    expectThrows(NullPointerException.class, () -> field.setVectorValue(null));
+  }
+
+  // Illegal schema change tests:
+
+  public void testIllegalDimChangeTwoDocs() throws Exception {
+    try (Directory dir = newDirectory();
+         IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+      Document doc = new Document();
+      doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+      w.addDocument(doc);
+      if (random().nextBoolean()) {
+        // sometimes test with two segments
+        w.commit();
+      }
+
+      Document doc2 = new Document();
+      doc2.add(new VectorField("dim", new float[3], ScoreFunction.DOT_PRODUCT));
+      IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+          () -> w.addDocument(doc2));
+      assertEquals("cannot change vector dimension from 4 to 3 for field=\"dim\"", expected.getMessage());
+    }
+  }
+
+  public void testIllegalScoreFunctionChange() throws Exception {
+    try (Directory dir = newDirectory();
+         IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+      Document doc = new Document();
+      doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+      w.addDocument(doc);
+      if (random().nextBoolean()) {
+        // sometimes test with two segments
+        w.commit();
+      }
+
+      Document doc2 = new Document();
+      doc2.add(new VectorField("dim", new float[4], ScoreFunction.EUCLIDEAN));
+      IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+          () -> w.addDocument(doc2));
+      assertEquals("cannot change vector score function from DOT_PRODUCT to EUCLIDEAN for field=\"dim\"", expected.getMessage());
+    }
+  }
+
+  public void testIllegalDimChangeTwoWriters() throws Exception {
+    try (Directory dir = newDirectory()) {
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+        w.addDocument(doc);
+      }
+
+      try (IndexWriter w2 = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc2 = new Document();
+        doc2.add(new VectorField("dim", new float[1], ScoreFunction.DOT_PRODUCT));
+        IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+            () -> w2.addDocument(doc2));
+        assertEquals("cannot change vector dimension from 4 to 1 for field=\"dim\"", expected.getMessage());
+      }
+    }
+  }
+
+  public void testIllegalScoreFunctionChangeTwoWriters() throws Exception {
+    try (Directory dir = newDirectory()) {
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+        w.addDocument(doc);
+      }
+
+      try (IndexWriter w2 = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc2 = new Document();
+        doc2.add(new VectorField("dim", new float[4], ScoreFunction.EUCLIDEAN));
+        IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+            () -> w2.addDocument(doc2));
+        assertEquals("cannot change vector score function from DOT_PRODUCT to EUCLIDEAN for field=\"dim\"", expected.getMessage());
+      }
+    }
+  }
+
+  public void testIllegalDimChangeViaAddIndexesDirectory() throws Exception {
+    try (Directory dir = newDirectory();
+         Directory dir2 = newDirectory()) {
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+        w.addDocument(doc);
+      }
+      try (IndexWriter w2 = new IndexWriter(dir2, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[5], ScoreFunction.DOT_PRODUCT));
+        w2.addDocument(doc);
+        IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+            () -> w2.addIndexes(new Directory[]{dir}));
+        assertEquals("cannot change vector dimension from 5 to 4 for field=\"dim\"", expected.getMessage());
+      }
+    }
+  }
+
+  public void testIllegalScoreFunctionChangeViaAddIndexesDirectory() throws Exception {
+    try (Directory dir = newDirectory();
+         Directory dir2 = newDirectory()) {
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+        w.addDocument(doc);
+      }
+      try (IndexWriter w2 = new IndexWriter(dir2, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.EUCLIDEAN));
+        w2.addDocument(doc);
+        IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+            () -> w2.addIndexes(dir));
+        assertEquals("cannot change vector score function from EUCLIDEAN to DOT_PRODUCT for field=\"dim\"", expected.getMessage());
+      }
+    }
+  }
+
+  public void testIllegalDimChangeViaAddIndexesCodecReader() throws Exception {
+    try (Directory dir = newDirectory();
+         Directory dir2 = newDirectory()) {
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+        w.addDocument(doc);
+      }
+      try (IndexWriter w2 = new IndexWriter(dir2, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[5], ScoreFunction.DOT_PRODUCT));
+        w2.addDocument(doc);
+        try (DirectoryReader r = DirectoryReader.open(dir)) {
+          IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+              () -> w2.addIndexes(new CodecReader[]{(CodecReader) getOnlyLeafReader(r)}));
+          assertEquals("cannot change vector dimension from 5 to 4 for field=\"dim\"", expected.getMessage());
+        }
+      }
+    }
+  }
+
+  public void testIllegalScoreFunctionChangeViaAddIndexesCodecReader() throws Exception {
+    try (Directory dir = newDirectory();
+         Directory dir2 = newDirectory()) {
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+        w.addDocument(doc);
+      }
+      try (IndexWriter w2 = new IndexWriter(dir2, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.EUCLIDEAN));
+        w2.addDocument(doc);
+        try (DirectoryReader r = DirectoryReader.open(dir)) {
+          IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+              () -> w2.addIndexes(new CodecReader[]{(CodecReader) getOnlyLeafReader(r)}));
+          assertEquals("cannot change vector score function from EUCLIDEAN to DOT_PRODUCT for field=\"dim\"", expected.getMessage());
+        }
+      }
+    }
+  }
+
+  public void testIllegalDimChangeViaAddIndexesSlowCodecReader() throws Exception {
+    try (Directory dir = newDirectory();
+         Directory dir2 = newDirectory()) {
+      try (IndexWriter w = new IndexWriter(dir, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[4], ScoreFunction.DOT_PRODUCT));
+        w.addDocument(doc);
+      }
+      try (IndexWriter w2 = new IndexWriter(dir2, createIndexWriterConfig())) {
+        Document doc = new Document();
+        doc.add(new VectorField("dim", new float[5], ScoreFunction.DOT_PRODUCT));
+        w2.addDocument(doc);
+        try (DirectoryReader r = DirectoryReader.open(dir)) {
+          IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+              () -> TestUtil.addIndexesSlowly(w2, r));
+          assertEquals("cannot change vector dimension from 5 to 4 for field=\"dim\"", expected.getMessage());
+        }
+      }
+    }
+  }
+
+  public void testIllegalScoreFunctionChangeViaAddIndexesSlowCodecReader() throws Exception {

Review comment:
       Ah, good idea, I will add




----------------------------------------------------------------
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.

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


[GitHub] [lucene-solr] mikemccand commented on pull request #1930: LUCENE-9322: add VectorValues to new Lucene90 codec

Posted by GitBox <gi...@apache.org>.
mikemccand commented on pull request #1930:
URL: https://github.com/apache/lucene-solr/pull/1930#issuecomment-710035455


   > So .. this has been out for 17 days now - I'd like to commit in the next day or so in order to enable moving forward with the matching piece of this, adding some benchmarking to luceneutil that depends on it, etc. @mikemccand do you want to have another look? - I think I've addressed your comments.
   
   Thanks @msokolov -- I'll have another look today.  It was already close the first time I looked :)


----------------------------------------------------------------
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.

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


[GitHub] [lucene-solr] msokolov edited a comment on pull request #1930: LUCENE-9322: add VectorValues to new Lucene90 codec

Posted by GitBox <gi...@apache.org>.
msokolov edited a comment on pull request #1930:
URL: https://github.com/apache/lucene-solr/pull/1930#issuecomment-707088977


   > it will be helpful to use ann-benchmark
   
   so .. I finally did get it working, but I have a few questions. Aside: I think my main stumbling block was running on an ARM instance - that may have caused some dependency issues, and then I found most of the algorithms are compiled with x86-only compiler extensions, sigh. But my main concern there is aboutt the way this benchmarking system runs. It seems to expect your algorithm to be delivered as an in-process extension to Python, which works OK for a native code library, but I'm not sure how we'd present Lucene to it. We don't want to have to call through a network API?


----------------------------------------------------------------
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.

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


[GitHub] [lucene-solr] msokolov commented on pull request #1930: LUCENE-9322: add VectorValues to new Lucene90 codec

Posted by GitBox <gi...@apache.org>.
msokolov commented on pull request #1930:
URL: https://github.com/apache/lucene-solr/pull/1930#issuecomment-703872279


   > Thank you for ... the tests catching mis-use where user tries to change dimension or scoring function in an existing field.
   Thanks to @mocobeta for those; I was able to carry that forward from her earlier patch
   
   > I see you implemented the two score functions, but are they ever exercised in tests
   True - this was extracted from a bigger change including usage of those methods as part of KNN search, but they deserve their own unit tests - I'll add.
   
   > I would love to see a "Vector Overview" javadoc somewhere ...
   Yes - I'll add to the VectorValues/VectorField class javadocs I think that's the most natural/visible place.
   
   > I am curious how the basic vector usage performs -- just indexing one vector field, and retrieving it at search time. We can (separately) enable luceneutil to support testing vectors, somehow. But I wonder where we'll get semi-realistic vectors derived from Wikipedia content 
   Agreed that benchmarking is needed. I think we can use http://ann-benchmarks.com/ as a guide for some standardized test vectors. They won't be related to wikipedia? If we get to wanting that, we could also make use of something like https://fasttext.cc/docs/en/pretrained-vectors.html that is trained on ngrams taken from Wikipedia (for many languages)? I don't know how suited it is, just found in a google search. For that, we'd have to compute document/query vectors based on an ngram-vector dictionary. I think a simple thing is to sum all the ngram-vectors for all the ngrams in a document / query


----------------------------------------------------------------
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.

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


[GitHub] [lucene-solr] muse-dev[bot] commented on a change in pull request #1930: LUCENE-9322: add VectorValues to new Lucene90 codec

Posted by GitBox <gi...@apache.org>.
muse-dev[bot] commented on a change in pull request #1930:
URL: https://github.com/apache/lucene-solr/pull/1930#discussion_r501886824



##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexingChain.java
##########
@@ -562,6 +614,12 @@ private int processField(int docID, IndexableField field, long fieldGen, int fie
       }
       indexPoint(docID, fp, field);
     }
+    if (fieldType.vectorDimension() != 0) {
+      if (fp == null) {
+        fp = getOrAddField(fieldName, fieldType, false);
+      }
+      indexVector(docID, fp, field);

Review comment:
       *NULL_DEREFERENCE:*  object `fp` last assigned on line 619 could be null and is dereferenced by call to `indexVector(...)` at line 621.




----------------------------------------------------------------
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.

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


[GitHub] [lucene-solr] msokolov commented on a change in pull request #1930: LUCENE-9322: add VectorValues to new Lucene90 codec

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #1930:
URL: https://github.com/apache/lucene-solr/pull/1930#discussion_r499858704



##########
File path: lucene/core/src/java/org/apache/lucene/codecs/VectorWriter.java
##########
@@ -0,0 +1,253 @@
+/*
+ * 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;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.lucene.index.DocIDMerger;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.MergeState;
+import org.apache.lucene.index.VectorValues;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.util.BytesRef;
+
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+/**
+ * Writes vectors to an index.
+ */
+public abstract class VectorWriter implements Closeable {
+
+  /** Sole constructor */
+  protected VectorWriter() {}
+
+  /** Write all values contained in the provided reader */
+  public abstract void writeField(FieldInfo fieldInfo, VectorValues values) throws IOException;
+
+  /** Called once at the end before close */
+  public abstract void finish() throws IOException;
+
+  /** Merge the vector values from multiple segments, for all fields */
+  public void merge(MergeState mergeState) throws IOException {
+    for (VectorReader reader : mergeState.vectorReaders) {
+      if (reader != null) {
+        reader.checkIntegrity();
+      }
+    }
+    for (FieldInfo fieldInfo : mergeState.mergeFieldInfos) {
+      if (fieldInfo.hasVectorValues()) {
+        mergeVectors(fieldInfo, mergeState);
+      }
+    }
+    finish();
+  }
+
+  private void mergeVectors(FieldInfo mergeFieldInfo, final MergeState mergeState) throws IOException {
+    if (mergeState.infoStream.isEnabled("VV")) {
+      mergeState.infoStream.message("VV", "merging " + mergeState.segmentInfo);
+    }
+    List<VectorValuesSub> subs = new ArrayList<>();
+    int dimension = -1;
+    VectorValues.ScoreFunction scoreFunction = null;
+    for (int i = 0; i < mergeState.vectorReaders.length; i++) {
+      VectorReader vectorReader = mergeState.vectorReaders[i];
+      if (vectorReader != null) {
+        if (mergeFieldInfo != null && mergeFieldInfo.hasVectorValues()) {
+          int segmentDimension = mergeFieldInfo.getVectorDimension();
+          VectorValues.ScoreFunction segmentScoreFunction = mergeFieldInfo.getVectorScoreFunction();
+          if (dimension == -1) {
+            dimension = segmentDimension;
+            scoreFunction = mergeFieldInfo.getVectorScoreFunction();
+          } else if (dimension != segmentDimension) {
+            throw new IllegalStateException("Varying dimensions for vector-valued field " + mergeFieldInfo.name
+                + ": " + dimension + "!=" + segmentDimension);
+          } else if (scoreFunction != segmentScoreFunction) {

Review comment:
       Yes, `IndexingChain.indexVector` calls `FieldInfo.setVectorDimensionAndScoreFunction`, which checks against existing values. It allows to go from 0 to non-zero dimension (setting scoreFunction at that time), but no other change is allowed. This is tested in TestVectorValues




----------------------------------------------------------------
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.

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


[GitHub] [lucene-solr] msokolov commented on a change in pull request #1930: LUCENE-9322: add VectorValues to new Lucene90 codec

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #1930:
URL: https://github.com/apache/lucene-solr/pull/1930#discussion_r499863809



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -374,7 +377,25 @@ private FieldNormStatus() {
       /** Total number of fields with points. */
       public int totalValueFields;
       
-      /** Exception thrown during doc values test (null on success) */
+      /** Exception thrown during point values test (null on success) */

Review comment:
       Yeah, Points was a model for much of this, so I stumbled on that




----------------------------------------------------------------
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.

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


[GitHub] [lucene-solr] jtibshirani commented on pull request #1930: LUCENE-9322: add VectorValues to new Lucene90 codec

Posted by GitBox <gi...@apache.org>.
jtibshirani commented on pull request #1930:
URL: https://github.com/apache/lucene-solr/pull/1930#issuecomment-706813811


   > Hmm I tried to get that benchmarking suite to run and it requires some major Python-fu.
   
   I managed to get this working a few months ago while experimenting with a clustering-based approach: https://github.com/jtibshirani/ann-benchmarks/pull/2. It indeed involved a lot of set-up -- I can try to get it working again post results. Going forward, I think it will be helpful to use ann-benchmarks to compare recall and QPS against the ANN reference implementations.


----------------------------------------------------------------
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.

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


[GitHub] [lucene-solr] msokolov closed pull request #1930: LUCENE-9322: add VectorValues to new Lucene90 codec

Posted by GitBox <gi...@apache.org>.
msokolov closed pull request #1930:
URL: https://github.com/apache/lucene-solr/pull/1930


   


----------------------------------------------------------------
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.

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


[GitHub] [lucene-solr] jtibshirani removed a comment on pull request #1930: LUCENE-9322: add VectorValues to new Lucene90 codec

Posted by GitBox <gi...@apache.org>.
jtibshirani removed a comment on pull request #1930:
URL: https://github.com/apache/lucene-solr/pull/1930#issuecomment-711759394


   @msokolov I did a belated review and had some high-level thoughts. Hopefully post-commit reviews are still helpful in giving ideas for follow-ups.
   
   > It seems messy to have the ScoringFunction implementation in the main VectorValues interface API file. 
   
   It seems like we’re using `ScoreFunction` for two separate use cases:
   1. Configuring the ANN indexing + search strategy (to be introduced later). For example, HNSW needs to know up-front what similarity measure to use when building the graph.
   2. Providing convenience methods for users that want to incorporate vectors into scoring, outside of ANN.
   
   We could think about separating out these cases. Maybe `ScoreFunction` could stick closely to case 1 and avoid exposing score implementations. This fits well with its role of specifying the ANN index strategy, which might encompass other ANN approaches in the future like `KMEANS_EUCLIDEAN`. And for use case 2, we could introduce static utility methods like `dotProduct(float[] a, float[] b)` ? The ANN implementations would be free to make use of these methods if they prove helpful. If we go with this design, perhaps `ScoreFunction` would be renamed to something like `SearchStrategy`.
   
   Separately, I wonder if we need to expose a random access interface on the public `VectorValues`. When using vectors for scoring, iterator access seems sufficient. I’m having trouble thinking of a use for random access outside of ANN, and some ANN implementations don’t even need it -- for example most k-means and LSH approaches only need forward iteration. Would it make sense to keep the `VectorValues` surface area smaller and only provide iterator access? Random access could still be supported at a lower level, in a place that's available to HNSW. I’m not sure exactly how this would look, perhaps it'd be clearer once we see how HNSW will be integrated.


----------------------------------------------------------------
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.

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


[GitHub] [lucene-solr] msokolov commented on a change in pull request #1930: LUCENE-9322: add VectorValues to new Lucene90 codec

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #1930:
URL: https://github.com/apache/lucene-solr/pull/1930#discussion_r499898687



##########
File path: lucene/core/src/java/org/apache/lucene/index/VectorValuesWriter.java
##########
@@ -0,0 +1,272 @@
+/*
+ * 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.index;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.codecs.VectorWriter;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.Counter;
+
+/** Buffers up pending vector value(s) per doc, then flushes when segment flushes. */
+public class VectorValuesWriter {
+
+  private final FieldInfo fieldInfo;
+  private final Counter iwBytesUsed;
+  private final List<float[]> vectors = new ArrayList<>();
+  private final DocsWithFieldSet docsWithField;
+
+  private int lastDocID = -1;
+
+  private long bytesUsed;
+
+  VectorValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
+    this.fieldInfo = fieldInfo;
+    this.iwBytesUsed = iwBytesUsed;
+    this.docsWithField = new DocsWithFieldSet();
+    this.bytesUsed = docsWithField.ramBytesUsed();
+    if (iwBytesUsed != null) {
+      iwBytesUsed.addAndGet(bytesUsed);
+    }
+  }
+
+  /**
+   * Adds a value for the given document. Only a single value may be added.
+   * @param docID the value is added to this document
+   * @param vectorValue the value to add
+   * @throws IllegalArgumentException if a value has already been added to the given document
+   */
+  public void addValue(int docID, float[] vectorValue) {
+    if (docID == lastDocID) {
+      throw new IllegalArgumentException("VectorValuesField \"" + fieldInfo.name + "\" appears more than once in this document (only one value is allowed per field)");
+    }
+    assert docID > lastDocID;
+    docsWithField.add(docID);
+    vectors.add(ArrayUtil.copyOfSubArray(vectorValue, 0, vectorValue.length));

Review comment:
       Yes in the normal case, but a sneaky/unlucky person could bypass Field type safety; I'll add a check here.




----------------------------------------------------------------
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.

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


[GitHub] [lucene-solr] msokolov commented on pull request #1930: LUCENE-9322: add VectorValues to new Lucene90 codec

Posted by GitBox <gi...@apache.org>.
msokolov commented on pull request #1930:
URL: https://github.com/apache/lucene-solr/pull/1930#issuecomment-707088977


   > it will be helpful to use ann-benchmark
   so .. I finally did get it working, but I have a few questions. Aside: I think my main stumbling block was running on an ARM instance - that may have caused some dependency issues, and then I found most of the algorithms are compiled with x86-only compiler extensions, sigh. But my main concern there is aboutt the way this benchmarking system runs. It seems to expect your algorithm to be delivered as an in-process extension to Python, which works OK for a native code library, but I'm not sure how we'd present Lucene to it. We don't want to have to call through a network API?


----------------------------------------------------------------
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.

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


[GitHub] [lucene-solr] msokolov commented on pull request #1930: LUCENE-9322: add VectorValues to new Lucene90 codec

Posted by GitBox <gi...@apache.org>.
msokolov commented on pull request #1930:
URL: https://github.com/apache/lucene-solr/pull/1930#issuecomment-703831393


   Thanks for the extensive comments, @mikemccand - I'll address soon with an updated PR. 
   
   I also found some bugs in implementations of the random access interface, and I want to fix those and enhance the test coverage. Mainly, they were incorrectly sharing state with the enclosing iterators, which causes issues if you simultaneously iterate and access randomly. There were also a couple of bugs dealing with empty segments that apparently weren't caught by testRandom.


----------------------------------------------------------------
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.

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