You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jp...@apache.org on 2023/01/23 13:46:08 UTC

[lucene] branch branch_9x updated: Remove binaryValue() on VectorValues and ByteVectorValues. (#12101)

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

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


The following commit(s) were added to refs/heads/branch_9x by this push:
     new 4ff5f4b2da5 Remove binaryValue() on VectorValues and ByteVectorValues. (#12101)
4ff5f4b2da5 is described below

commit 4ff5f4b2da5c1f95e28b268815a12f8259b5331f
Author: Adrien Grand <jp...@gmail.com>
AuthorDate: Mon Jan 23 13:42:49 2023 +0000

    Remove binaryValue() on VectorValues and ByteVectorValues. (#12101)
    
    This method tries to expose an encoded view of vectors, but we shouldn't have
    this part of our user-facing API. With this change, the way vectors are encoded
    is entirely on the codec.
---
 lucene/CHANGES.txt                                 |  4 ++
 .../lucene90/Lucene90HnswVectorsReader.java        | 12 ------
 .../lucene91/Lucene91HnswVectorsReader.java        | 12 ------
 .../lucene92/OffHeapVectorValues.java              | 25 -----------
 .../lucene94/ExpandingVectorValues.java            | 49 ----------------------
 .../lucene94/OffHeapVectorValues.java              | 25 -----------
 .../lucene90/Lucene90HnswVectorsWriter.java        | 11 +++--
 .../lucene91/Lucene91HnswVectorsWriter.java        | 11 +++--
 .../lucene92/Lucene92HnswVectorsWriter.java        | 11 +++--
 .../lucene94/Lucene94HnswVectorsWriter.java        | 11 +++--
 .../simpletext/SimpleTextKnnVectorsReader.java     | 35 ++++------------
 .../lucene/codecs/BufferingKnnVectorsWriter.java   | 27 ------------
 .../org/apache/lucene/codecs/KnnVectorsWriter.java |  5 ---
 .../codecs/lucene95/Lucene95HnswVectorsWriter.java | 11 +++--
 .../codecs/lucene95/OffHeapVectorValues.java       | 25 -----------
 .../org/apache/lucene/index/ByteVectorValues.java  | 12 ------
 .../lucene/index/ExitableDirectoryReader.java      |  5 ---
 .../apache/lucene/index/FilterVectorValues.java    |  6 ---
 .../apache/lucene/index/SortingCodecReader.java    | 12 ------
 .../java/org/apache/lucene/index/VectorValues.java | 13 ------
 .../test/org/apache/lucene/document/TestField.java |  1 -
 21 files changed, 47 insertions(+), 276 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 11bf03a3a68..87bf501efb6 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -69,6 +69,10 @@ API Changes
   and SortedNumericDocValuesField#newSlowSetQuery. IntField, LongField, FloatField, and DoubleField
   implement newSetQuery with best-practice use of IndexOrDocValuesQuery. (Robert Muir)
 
+* GITHUB#12101: Remove VectorValues#binaryValue(). Vectors should only be
+  accessed through their high-level representation, via
+  VectorValues#vectorValue(). (Adrien Grand)
+
 New Features
 ---------------------
 * GITHUB#11795: Add ByteWritesTrackingDirectoryWrapper to expose metrics for bytes merged, flushed, and overall
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswVectorsReader.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswVectorsReader.java
index f33aa13e09f..39487ec0d3f 100644
--- a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswVectorsReader.java
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswVectorsReader.java
@@ -20,7 +20,6 @@ package org.apache.lucene.backward_codecs.lucene90;
 import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
 
 import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
@@ -365,8 +364,6 @@ public final class Lucene90HnswVectorsReader extends KnnVectorsReader {
     final int[] ordToDoc;
     final IndexInput dataIn;
 
-    final BytesRef binaryValue;
-    final ByteBuffer byteBuffer;
     final int byteSize;
     final float[] value;
 
@@ -379,9 +376,7 @@ public final class Lucene90HnswVectorsReader extends KnnVectorsReader {
       this.dataIn = dataIn;
 
       byteSize = Float.BYTES * dimension;
-      byteBuffer = ByteBuffer.allocate(byteSize);
       value = new float[dimension];
-      binaryValue = new BytesRef(byteBuffer.array(), byteBuffer.arrayOffset(), byteSize);
     }
 
     @Override
@@ -401,13 +396,6 @@ public final class Lucene90HnswVectorsReader extends KnnVectorsReader {
       return value;
     }
 
-    @Override
-    public BytesRef binaryValue() throws IOException {
-      dataIn.seek((long) ord * byteSize);
-      dataIn.readBytes(byteBuffer.array(), byteBuffer.arrayOffset(), byteSize, false);
-      return binaryValue;
-    }
-
     @Override
     public int docID() {
       return doc;
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene91/Lucene91HnswVectorsReader.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene91/Lucene91HnswVectorsReader.java
index 6367907f8c5..56ebb34d27c 100644
--- a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene91/Lucene91HnswVectorsReader.java
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene91/Lucene91HnswVectorsReader.java
@@ -20,7 +20,6 @@ package org.apache.lucene.backward_codecs.lucene91;
 import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
 
 import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
@@ -412,8 +411,6 @@ public final class Lucene91HnswVectorsReader extends KnnVectorsReader {
     private final int[] ordToDoc;
     private final IntUnaryOperator ordToDocOperator;
     private final IndexInput dataIn;
-    private final BytesRef binaryValue;
-    private final ByteBuffer byteBuffer;
     private final int byteSize;
     private final float[] value;
 
@@ -427,9 +424,7 @@ public final class Lucene91HnswVectorsReader extends KnnVectorsReader {
       ordToDocOperator = ordToDoc == null ? IntUnaryOperator.identity() : (ord) -> ordToDoc[ord];
       this.dataIn = dataIn;
       byteSize = Float.BYTES * dimension;
-      byteBuffer = ByteBuffer.allocate(byteSize);
       value = new float[dimension];
-      binaryValue = new BytesRef(byteBuffer.array(), byteBuffer.arrayOffset(), byteSize);
     }
 
     @Override
@@ -449,13 +444,6 @@ public final class Lucene91HnswVectorsReader extends KnnVectorsReader {
       return value;
     }
 
-    @Override
-    public BytesRef binaryValue() throws IOException {
-      dataIn.seek((long) ord * byteSize);
-      dataIn.readBytes(byteBuffer.array(), byteBuffer.arrayOffset(), byteSize, false);
-      return binaryValue;
-    }
-
     @Override
     public int docID() {
       return doc;
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene92/OffHeapVectorValues.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene92/OffHeapVectorValues.java
index 8ddf86cacfa..c693063f92a 100644
--- a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene92/OffHeapVectorValues.java
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene92/OffHeapVectorValues.java
@@ -18,13 +18,11 @@
 package org.apache.lucene.backward_codecs.lucene92;
 
 import java.io.IOException;
-import java.nio.ByteBuffer;
 import org.apache.lucene.codecs.lucene90.IndexedDISI;
 import org.apache.lucene.index.VectorValues;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.RandomAccessInput;
 import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.hnsw.RandomAccessVectorValues;
 import org.apache.lucene.util.packed.DirectMonotonicReader;
 
@@ -35,8 +33,6 @@ abstract class OffHeapVectorValues extends VectorValues
   protected final int dimension;
   protected final int size;
   protected final IndexInput slice;
-  protected final BytesRef binaryValue;
-  protected final ByteBuffer byteBuffer;
   protected final int byteSize;
   protected final float[] value;
 
@@ -45,9 +41,7 @@ abstract class OffHeapVectorValues extends VectorValues
     this.size = size;
     this.slice = slice;
     byteSize = Float.BYTES * dimension;
-    byteBuffer = ByteBuffer.allocate(byteSize);
     value = new float[dimension];
-    binaryValue = new BytesRef(byteBuffer.array(), byteBuffer.arrayOffset(), byteSize);
   }
 
   @Override
@@ -100,13 +94,6 @@ abstract class OffHeapVectorValues extends VectorValues
       return value;
     }
 
-    @Override
-    public BytesRef binaryValue() throws IOException {
-      slice.seek((long) doc * byteSize);
-      slice.readBytes(byteBuffer.array(), byteBuffer.arrayOffset(), byteSize, false);
-      return binaryValue;
-    }
-
     @Override
     public int docID() {
       return doc;
@@ -176,13 +163,6 @@ abstract class OffHeapVectorValues extends VectorValues
       return value;
     }
 
-    @Override
-    public BytesRef binaryValue() throws IOException {
-      slice.seek((long) (disi.index()) * byteSize);
-      slice.readBytes(byteBuffer.array(), byteBuffer.arrayOffset(), byteSize, false);
-      return binaryValue;
-    }
-
     @Override
     public int docID() {
       return disi.docID();
@@ -251,11 +231,6 @@ abstract class OffHeapVectorValues extends VectorValues
       throw new UnsupportedOperationException();
     }
 
-    @Override
-    public BytesRef binaryValue() throws IOException {
-      throw new UnsupportedOperationException();
-    }
-
     @Override
     public int docID() {
       return doc;
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene94/ExpandingVectorValues.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene94/ExpandingVectorValues.java
deleted file mode 100644
index 21eb7316ed0..00000000000
--- a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene94/ExpandingVectorValues.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.lucene.backward_codecs.lucene94;
-
-import java.io.IOException;
-import org.apache.lucene.index.FilterVectorValues;
-import org.apache.lucene.index.VectorValues;
-import org.apache.lucene.util.BytesRef;
-
-/** reads from byte-encoded data */
-class ExpandingVectorValues extends FilterVectorValues {
-
-  private final float[] value;
-
-  /**
-   * Constructs ExpandingVectorValues with passed byte encoded VectorValues iterator
-   *
-   * @param in the wrapped values
-   */
-  protected ExpandingVectorValues(VectorValues in) {
-    super(in);
-    value = new float[in.dimension()];
-  }
-
-  @Override
-  public float[] vectorValue() throws IOException {
-    BytesRef binaryValue = binaryValue();
-    byte[] bytes = binaryValue.bytes;
-    for (int i = 0, j = binaryValue.offset; i < value.length; i++, j++) {
-      value[i] = bytes[j];
-    }
-    return value;
-  }
-}
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene94/OffHeapVectorValues.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene94/OffHeapVectorValues.java
index e10fa45ce62..f90ac7e4b0b 100644
--- a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene94/OffHeapVectorValues.java
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene94/OffHeapVectorValues.java
@@ -18,13 +18,11 @@
 package org.apache.lucene.backward_codecs.lucene94;
 
 import java.io.IOException;
-import java.nio.ByteBuffer;
 import org.apache.lucene.codecs.lucene90.IndexedDISI;
 import org.apache.lucene.index.VectorValues;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.RandomAccessInput;
 import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.hnsw.RandomAccessVectorValues;
 import org.apache.lucene.util.packed.DirectMonotonicReader;
 
@@ -35,8 +33,6 @@ abstract class OffHeapVectorValues extends VectorValues
   protected final int dimension;
   protected final int size;
   protected final IndexInput slice;
-  protected final BytesRef binaryValue;
-  protected final ByteBuffer byteBuffer;
   protected final int byteSize;
   protected final float[] value;
 
@@ -45,9 +41,7 @@ abstract class OffHeapVectorValues extends VectorValues
     this.size = size;
     this.slice = slice;
     this.byteSize = byteSize;
-    byteBuffer = ByteBuffer.allocate(byteSize);
     value = new float[dimension];
-    binaryValue = new BytesRef(byteBuffer.array(), byteBuffer.arrayOffset(), byteSize);
   }
 
   @Override
@@ -102,13 +96,6 @@ abstract class OffHeapVectorValues extends VectorValues
       return value;
     }
 
-    @Override
-    public BytesRef binaryValue() throws IOException {
-      slice.seek((long) doc * byteSize);
-      slice.readBytes(byteBuffer.array(), byteBuffer.arrayOffset(), byteSize, false);
-      return binaryValue;
-    }
-
     @Override
     public int docID() {
       return doc;
@@ -181,13 +168,6 @@ abstract class OffHeapVectorValues extends VectorValues
       return value;
     }
 
-    @Override
-    public BytesRef binaryValue() throws IOException {
-      slice.seek((long) (disi.index()) * byteSize);
-      slice.readBytes(byteBuffer.array(), byteBuffer.arrayOffset(), byteSize, false);
-      return binaryValue;
-    }
-
     @Override
     public int docID() {
       return disi.docID();
@@ -256,11 +236,6 @@ abstract class OffHeapVectorValues extends VectorValues
       throw new UnsupportedOperationException();
     }
 
-    @Override
-    public BytesRef binaryValue() throws IOException {
-      throw new UnsupportedOperationException();
-    }
-
     @Override
     public int docID() {
       return doc;
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswVectorsWriter.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswVectorsWriter.java
index 4952029e947..55c3a434817 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswVectorsWriter.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswVectorsWriter.java
@@ -20,6 +20,8 @@ package org.apache.lucene.backward_codecs.lucene90;
 import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
 import java.util.Arrays;
 import org.apache.lucene.codecs.BufferingKnnVectorsWriter;
 import org.apache.lucene.codecs.CodecUtil;
@@ -32,7 +34,6 @@ import org.apache.lucene.index.VectorValues;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.hnsw.RandomAccessVectorValues;
 
@@ -178,11 +179,13 @@ public final class Lucene90HnswVectorsWriter extends BufferingKnnVectorsWriter {
       throws IOException {
     int[] docIds = new int[vectors.size()];
     int count = 0;
+    ByteBuffer binaryVector =
+        ByteBuffer.allocate(vectors.dimension() * Float.BYTES).order(ByteOrder.LITTLE_ENDIAN);
     for (int docV = vectors.nextDoc(); docV != NO_MORE_DOCS; docV = vectors.nextDoc(), count++) {
       // write vector
-      BytesRef binaryValue = vectors.binaryValue();
-      assert binaryValue.length == vectors.dimension() * Float.BYTES;
-      output.writeBytes(binaryValue.bytes, binaryValue.offset, binaryValue.length);
+      float[] vectorValue = vectors.vectorValue();
+      binaryVector.asFloatBuffer().put(vectorValue);
+      output.writeBytes(binaryVector.array(), 0, binaryVector.limit());
       docIds[count] = docV;
     }
 
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene91/Lucene91HnswVectorsWriter.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene91/Lucene91HnswVectorsWriter.java
index 2c603aba621..a99bcffd2d4 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene91/Lucene91HnswVectorsWriter.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene91/Lucene91HnswVectorsWriter.java
@@ -20,6 +20,8 @@ package org.apache.lucene.backward_codecs.lucene91;
 import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
 import java.util.Arrays;
 import org.apache.lucene.codecs.BufferingKnnVectorsWriter;
 import org.apache.lucene.codecs.CodecUtil;
@@ -33,7 +35,6 @@ import org.apache.lucene.index.VectorValues;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.hnsw.HnswGraph.NodesIterator;
 import org.apache.lucene.util.hnsw.RandomAccessVectorValues;
@@ -172,11 +173,13 @@ public final class Lucene91HnswVectorsWriter extends BufferingKnnVectorsWriter {
   private static DocsWithFieldSet writeVectorData(IndexOutput output, VectorValues vectors)
       throws IOException {
     DocsWithFieldSet docsWithField = new DocsWithFieldSet();
+    ByteBuffer binaryVector =
+        ByteBuffer.allocate(vectors.dimension() * Float.BYTES).order(ByteOrder.LITTLE_ENDIAN);
     for (int docV = vectors.nextDoc(); docV != NO_MORE_DOCS; docV = vectors.nextDoc()) {
       // write vector
-      BytesRef binaryValue = vectors.binaryValue();
-      assert binaryValue.length == vectors.dimension() * Float.BYTES;
-      output.writeBytes(binaryValue.bytes, binaryValue.offset, binaryValue.length);
+      float[] vectorValue = vectors.vectorValue();
+      binaryVector.asFloatBuffer().put(vectorValue);
+      output.writeBytes(binaryVector.array(), 0, binaryVector.limit());
       docsWithField.add(docV);
     }
     return docsWithField;
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene92/Lucene92HnswVectorsWriter.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene92/Lucene92HnswVectorsWriter.java
index df0c1032064..28a1fddeaee 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene92/Lucene92HnswVectorsWriter.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene92/Lucene92HnswVectorsWriter.java
@@ -21,6 +21,8 @@ import static org.apache.lucene.backward_codecs.lucene92.Lucene92RWHnswVectorsFo
 import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
 import java.util.Arrays;
 import org.apache.lucene.codecs.BufferingKnnVectorsWriter;
 import org.apache.lucene.codecs.CodecUtil;
@@ -36,7 +38,6 @@ import org.apache.lucene.index.VectorValues;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.hnsw.HnswGraph.NodesIterator;
 import org.apache.lucene.util.hnsw.HnswGraphBuilder;
@@ -180,11 +181,13 @@ public final class Lucene92HnswVectorsWriter extends BufferingKnnVectorsWriter {
   private static DocsWithFieldSet writeVectorData(IndexOutput output, VectorValues vectors)
       throws IOException {
     DocsWithFieldSet docsWithField = new DocsWithFieldSet();
+    ByteBuffer binaryVector =
+        ByteBuffer.allocate(vectors.dimension() * Float.BYTES).order(ByteOrder.LITTLE_ENDIAN);
     for (int docV = vectors.nextDoc(); docV != NO_MORE_DOCS; docV = vectors.nextDoc()) {
       // write vector
-      BytesRef binaryValue = vectors.binaryValue();
-      assert binaryValue.length == vectors.dimension() * Float.BYTES;
-      output.writeBytes(binaryValue.bytes, binaryValue.offset, binaryValue.length);
+      float[] vectorValue = vectors.vectorValue();
+      binaryVector.asFloatBuffer().put(vectorValue);
+      output.writeBytes(binaryVector.array(), 0, binaryVector.limit());
       docsWithField.add(docV);
     }
     return docsWithField;
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene94/Lucene94HnswVectorsWriter.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene94/Lucene94HnswVectorsWriter.java
index 70318077b41..f926a37be2e 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene94/Lucene94HnswVectorsWriter.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene94/Lucene94HnswVectorsWriter.java
@@ -617,7 +617,7 @@ public final class Lucene94HnswVectorsWriter extends KnnVectorsWriter {
         docV != NO_MORE_DOCS;
         docV = byteVectorValues.nextDoc()) {
       // write vector
-      BytesRef binaryValue = byteVectorValues.binaryValue();
+      BytesRef binaryValue = byteVectorValues.vectorValue();
       assert binaryValue.length == byteVectorValues.dimension() * VectorEncoding.BYTE.byteSize;
       output.writeBytes(binaryValue.bytes, binaryValue.offset, binaryValue.length);
       docsWithField.add(docV);
@@ -631,13 +631,16 @@ public final class Lucene94HnswVectorsWriter extends KnnVectorsWriter {
   private static DocsWithFieldSet writeVectorData(
       IndexOutput output, VectorValues floatVectorValues) throws IOException {
     DocsWithFieldSet docsWithField = new DocsWithFieldSet();
+    ByteBuffer binaryVector =
+        ByteBuffer.allocate(floatVectorValues.dimension() * VectorEncoding.FLOAT32.byteSize)
+            .order(ByteOrder.LITTLE_ENDIAN);
     for (int docV = floatVectorValues.nextDoc();
         docV != NO_MORE_DOCS;
         docV = floatVectorValues.nextDoc()) {
       // write vector
-      BytesRef binaryValue = floatVectorValues.binaryValue();
-      assert binaryValue.length == floatVectorValues.dimension() * VectorEncoding.FLOAT32.byteSize;
-      output.writeBytes(binaryValue.bytes, binaryValue.offset, binaryValue.length);
+      float[] vectorValue = floatVectorValues.vectorValue();
+      binaryVector.asFloatBuffer().put(vectorValue);
+      output.writeBytes(binaryVector.array(), 0, binaryVector.limit());
       docsWithField.add(docV);
     }
     return docsWithField;
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextKnnVectorsReader.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextKnnVectorsReader.java
index b575ab8fdbb..2043fbce815 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextKnnVectorsReader.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextKnnVectorsReader.java
@@ -17,10 +17,14 @@
 
 package org.apache.lucene.codecs.simpletext;
 
-import static org.apache.lucene.codecs.simpletext.SimpleTextKnnVectorsWriter.*;
+import static org.apache.lucene.codecs.simpletext.SimpleTextKnnVectorsWriter.FIELD_NAME;
+import static org.apache.lucene.codecs.simpletext.SimpleTextKnnVectorsWriter.FIELD_NUMBER;
+import static org.apache.lucene.codecs.simpletext.SimpleTextKnnVectorsWriter.SIZE;
+import static org.apache.lucene.codecs.simpletext.SimpleTextKnnVectorsWriter.VECTOR_DATA_LENGTH;
+import static org.apache.lucene.codecs.simpletext.SimpleTextKnnVectorsWriter.VECTOR_DATA_OFFSET;
+import static org.apache.lucene.codecs.simpletext.SimpleTextKnnVectorsWriter.VECTOR_DIMENSION;
 
 import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.nio.charset.StandardCharsets;
 import java.util.HashMap;
 import java.util.Map;
@@ -30,7 +34,6 @@ 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.VectorEncoding;
 import org.apache.lucene.index.VectorSimilarityFunction;
 import org.apache.lucene.index.VectorValues;
 import org.apache.lucene.search.DocIdSetIterator;
@@ -142,7 +145,7 @@ public class SimpleTextKnnVectorsReader extends KnnVectorsReader {
     }
     IndexInput bytesSlice =
         dataIn.slice("vector-data", fieldEntry.vectorDataOffset, fieldEntry.vectorDataLength);
-    return new SimpleTextVectorValues(fieldEntry, bytesSlice, info.getVectorEncoding());
+    return new SimpleTextVectorValues(fieldEntry, bytesSlice);
   }
 
   @Override
@@ -341,21 +344,15 @@ public class SimpleTextKnnVectorsReader extends KnnVectorsReader {
     private final BytesRefBuilder scratch = new BytesRefBuilder();
     private final FieldEntry entry;
     private final IndexInput in;
-    private final BytesRef binaryValue;
     private final float[][] values;
-    private final VectorEncoding vectorEncoding;
 
     int curOrd;
 
-    SimpleTextVectorValues(FieldEntry entry, IndexInput in, VectorEncoding vectorEncoding)
-        throws IOException {
+    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 * vectorEncoding.byteSize);
-      binaryValue.length = binaryValue.bytes.length;
       curOrd = -1;
-      this.vectorEncoding = vectorEncoding;
       readAllVectors();
     }
 
@@ -374,22 +371,6 @@ public class SimpleTextKnnVectorsReader extends KnnVectorsReader {
       return values[curOrd];
     }
 
-    @Override
-    public BytesRef binaryValue() {
-      switch (vectorEncoding) {
-          // we know that the floats are really just byte values
-        case BYTE:
-          for (int i = 0; i < values[curOrd].length; i++) {
-            binaryValue.bytes[i + binaryValue.offset] = (byte) values[curOrd][i];
-          }
-          break;
-        case FLOAT32:
-          ByteBuffer.wrap(binaryValue.bytes).asFloatBuffer().get(values[curOrd]);
-          break;
-      }
-      return binaryValue;
-    }
-
     @Override
     public RandomAccessVectorValues<float[]> copy() {
       return this;
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/BufferingKnnVectorsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/BufferingKnnVectorsWriter.java
index 760f959b2fd..8e0ce290050 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/BufferingKnnVectorsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/BufferingKnnVectorsWriter.java
@@ -18,8 +18,6 @@
 package org.apache.lucene.codecs;
 
 import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
 import java.util.ArrayList;
 import java.util.List;
 import org.apache.lucene.index.ByteVectorValues;
@@ -143,11 +141,6 @@ public abstract class BufferingKnnVectorsWriter extends KnnVectorsWriter {
       return docId;
     }
 
-    @Override
-    public BytesRef binaryValue() throws IOException {
-      return randomAccess.binaryValue(docIdOffsets[docId] - 1);
-    }
-
     @Override
     public float[] vectorValue() throws IOException {
       return randomAccess.vectorValue(docIdOffsets[docId] - 1);
@@ -277,11 +270,6 @@ public abstract class BufferingKnnVectorsWriter extends KnnVectorsWriter {
     final List<float[]> vectors;
     final int dimension;
 
-    final ByteBuffer buffer;
-    final BytesRef binaryValue;
-    final ByteBuffer raBuffer;
-    final BytesRef raBinaryValue;
-
     DocIdSetIterator docsWithFieldIter;
     int ord = -1;
 
@@ -289,10 +277,6 @@ public abstract class BufferingKnnVectorsWriter extends KnnVectorsWriter {
       this.docsWithField = docsWithField;
       this.vectors = vectors;
       this.dimension = dimension;
-      buffer = ByteBuffer.allocate(dimension * Float.BYTES).order(ByteOrder.LITTLE_ENDIAN);
-      binaryValue = new BytesRef(buffer.array());
-      raBuffer = ByteBuffer.allocate(dimension * Float.BYTES).order(ByteOrder.LITTLE_ENDIAN);
-      raBinaryValue = new BytesRef(raBuffer.array());
       docsWithFieldIter = docsWithField.iterator();
     }
 
@@ -310,17 +294,6 @@ public abstract class BufferingKnnVectorsWriter extends KnnVectorsWriter {
       return vectors.size();
     }
 
-    @Override
-    public BytesRef binaryValue() {
-      buffer.asFloatBuffer().put(vectorValue());
-      return binaryValue;
-    }
-
-    public BytesRef binaryValue(int targetOrd) {
-      raBuffer.asFloatBuffer().put(vectors.get(targetOrd));
-      return raBinaryValue;
-    }
-
     @Override
     public float[] vectorValue() {
       return vectors.get(ord);
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsWriter.java
index 64aa8edfebe..d073375e361 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsWriter.java
@@ -225,11 +225,6 @@ public abstract class KnnVectorsWriter implements Accountable, Closeable {
         return current.values.vectorValue();
       }
 
-      @Override
-      public BytesRef binaryValue() throws IOException {
-        return current.values.binaryValue();
-      }
-
       @Override
       public int advance(int target) {
         throw new UnsupportedOperationException();
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene95/Lucene95HnswVectorsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene95/Lucene95HnswVectorsWriter.java
index 1963a199cfd..0d7e8b953c8 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene95/Lucene95HnswVectorsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene95/Lucene95HnswVectorsWriter.java
@@ -670,7 +670,7 @@ public final class Lucene95HnswVectorsWriter extends KnnVectorsWriter {
         docV != NO_MORE_DOCS;
         docV = byteVectorValues.nextDoc()) {
       // write vector
-      BytesRef binaryValue = byteVectorValues.binaryValue();
+      BytesRef binaryValue = byteVectorValues.vectorValue();
       assert binaryValue.length == byteVectorValues.dimension() * VectorEncoding.BYTE.byteSize;
       output.writeBytes(binaryValue.bytes, binaryValue.offset, binaryValue.length);
       docsWithField.add(docV);
@@ -684,13 +684,16 @@ public final class Lucene95HnswVectorsWriter extends KnnVectorsWriter {
   private static DocsWithFieldSet writeVectorData(
       IndexOutput output, VectorValues floatVectorValues) throws IOException {
     DocsWithFieldSet docsWithField = new DocsWithFieldSet();
+    ByteBuffer buffer =
+        ByteBuffer.allocate(floatVectorValues.dimension() * VectorEncoding.FLOAT32.byteSize)
+            .order(ByteOrder.LITTLE_ENDIAN);
     for (int docV = floatVectorValues.nextDoc();
         docV != NO_MORE_DOCS;
         docV = floatVectorValues.nextDoc()) {
       // write vector
-      BytesRef binaryValue = floatVectorValues.binaryValue();
-      assert binaryValue.length == floatVectorValues.dimension() * VectorEncoding.FLOAT32.byteSize;
-      output.writeBytes(binaryValue.bytes, binaryValue.offset, binaryValue.length);
+      float[] value = floatVectorValues.vectorValue();
+      buffer.asFloatBuffer().put(value);
+      output.writeBytes(buffer.array(), 0, buffer.limit());
       docsWithField.add(docV);
     }
     return docsWithField;
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene95/OffHeapVectorValues.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene95/OffHeapVectorValues.java
index fdee80b186e..ac06dcc15c2 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene95/OffHeapVectorValues.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene95/OffHeapVectorValues.java
@@ -18,14 +18,12 @@
 package org.apache.lucene.codecs.lucene95;
 
 import java.io.IOException;
-import java.nio.ByteBuffer;
 import org.apache.lucene.codecs.lucene90.IndexedDISI;
 import org.apache.lucene.index.VectorEncoding;
 import org.apache.lucene.index.VectorValues;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.RandomAccessInput;
 import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.hnsw.RandomAccessVectorValues;
 import org.apache.lucene.util.packed.DirectMonotonicReader;
 
@@ -36,8 +34,6 @@ abstract class OffHeapVectorValues extends VectorValues
   protected final int dimension;
   protected final int size;
   protected final IndexInput slice;
-  protected final BytesRef binaryValue;
-  protected final ByteBuffer byteBuffer;
   protected final int byteSize;
   protected final float[] value;
 
@@ -46,9 +42,7 @@ abstract class OffHeapVectorValues extends VectorValues
     this.size = size;
     this.slice = slice;
     this.byteSize = byteSize;
-    byteBuffer = ByteBuffer.allocate(byteSize);
     value = new float[dimension];
-    binaryValue = new BytesRef(byteBuffer.array(), byteBuffer.arrayOffset(), byteSize);
   }
 
   @Override
@@ -104,13 +98,6 @@ abstract class OffHeapVectorValues extends VectorValues
       return value;
     }
 
-    @Override
-    public BytesRef binaryValue() throws IOException {
-      slice.seek((long) doc * byteSize);
-      slice.readBytes(byteBuffer.array(), byteBuffer.arrayOffset(), byteSize, false);
-      return binaryValue;
-    }
-
     @Override
     public int docID() {
       return doc;
@@ -183,13 +170,6 @@ abstract class OffHeapVectorValues extends VectorValues
       return value;
     }
 
-    @Override
-    public BytesRef binaryValue() throws IOException {
-      slice.seek((long) (disi.index()) * byteSize);
-      slice.readBytes(byteBuffer.array(), byteBuffer.arrayOffset(), byteSize, false);
-      return binaryValue;
-    }
-
     @Override
     public int docID() {
       return disi.docID();
@@ -258,11 +238,6 @@ abstract class OffHeapVectorValues extends VectorValues
       throw new UnsupportedOperationException();
     }
 
-    @Override
-    public BytesRef binaryValue() throws IOException {
-      throw new UnsupportedOperationException();
-    }
-
     @Override
     public int docID() {
       return doc;
diff --git a/lucene/core/src/java/org/apache/lucene/index/ByteVectorValues.java b/lucene/core/src/java/org/apache/lucene/index/ByteVectorValues.java
index eaac008388c..d9e9f9d9105 100644
--- a/lucene/core/src/java/org/apache/lucene/index/ByteVectorValues.java
+++ b/lucene/core/src/java/org/apache/lucene/index/ByteVectorValues.java
@@ -58,16 +58,4 @@ public abstract class ByteVectorValues extends DocIdSetIterator {
    * @return the vector value
    */
   public abstract BytesRef vectorValue() throws IOException;
-
-  /**
-   * Return the binary encoded vector value for the current document ID. These are the bytes
-   * corresponding to the float array return by {@link #vectorValue}. 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 final BytesRef binaryValue() throws IOException {
-    return vectorValue();
-  }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/index/ExitableDirectoryReader.java b/lucene/core/src/java/org/apache/lucene/index/ExitableDirectoryReader.java
index 7e36380a501..583d394d888 100644
--- a/lucene/core/src/java/org/apache/lucene/index/ExitableDirectoryReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/ExitableDirectoryReader.java
@@ -445,11 +445,6 @@ public class ExitableDirectoryReader extends FilterDirectoryReader {
         return vectorValues.size();
       }
 
-      @Override
-      public BytesRef binaryValue() throws IOException {
-        return vectorValues.binaryValue();
-      }
-
       /**
        * Throws {@link ExitingReaderException} if {@link QueryTimeout#shouldExit()} returns true, or
        * if {@link Thread#interrupted()} returns true.
diff --git a/lucene/core/src/java/org/apache/lucene/index/FilterVectorValues.java b/lucene/core/src/java/org/apache/lucene/index/FilterVectorValues.java
index 6ab47be9753..71a1f90d334 100644
--- a/lucene/core/src/java/org/apache/lucene/index/FilterVectorValues.java
+++ b/lucene/core/src/java/org/apache/lucene/index/FilterVectorValues.java
@@ -19,7 +19,6 @@ package org.apache.lucene.index;
 
 import java.io.IOException;
 import java.util.Objects;
-import org.apache.lucene.util.BytesRef;
 
 /** Delegates all methods to a wrapped {@link VectorValues}. */
 public abstract class FilterVectorValues extends VectorValues {
@@ -62,9 +61,4 @@ public abstract class FilterVectorValues extends VectorValues {
   public float[] vectorValue() throws IOException {
     return in.vectorValue();
   }
-
-  @Override
-  public BytesRef binaryValue() throws IOException {
-    return in.binaryValue();
-  }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/index/SortingCodecReader.java b/lucene/core/src/java/org/apache/lucene/index/SortingCodecReader.java
index 3fae5973855..4847598e049 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SortingCodecReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SortingCodecReader.java
@@ -20,8 +20,6 @@ package org.apache.lucene.index;
 import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
 
 import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -221,7 +219,6 @@ public final class SortingCodecReader extends FilterCodecReader {
     final int dimension;
     final FixedBitSet docsWithField;
     final float[][] vectors;
-    final ByteBuffer vectorAsBytes;
 
     private int docId = -1;
 
@@ -230,9 +227,6 @@ public final class SortingCodecReader extends FilterCodecReader {
       this.dimension = delegate.dimension();
       docsWithField = new FixedBitSet(sortMap.size());
       vectors = new float[sortMap.size()][];
-      vectorAsBytes =
-          ByteBuffer.allocate(delegate.dimension() * VectorEncoding.FLOAT32.byteSize)
-              .order(ByteOrder.LITTLE_ENDIAN);
       for (int doc = delegate.nextDoc(); doc != NO_MORE_DOCS; doc = delegate.nextDoc()) {
         int newDocID = sortMap.oldToNew(doc);
         docsWithField.set(newDocID);
@@ -250,12 +244,6 @@ public final class SortingCodecReader extends FilterCodecReader {
       return advance(docId + 1);
     }
 
-    @Override
-    public BytesRef binaryValue() throws IOException {
-      vectorAsBytes.asFloatBuffer().put(vectors[docId]);
-      return new BytesRef(vectorAsBytes.array());
-    }
-
     @Override
     public float[] vectorValue() throws IOException {
       return vectors[docId];
diff --git a/lucene/core/src/java/org/apache/lucene/index/VectorValues.java b/lucene/core/src/java/org/apache/lucene/index/VectorValues.java
index 549fa6ef55b..c7c0b1de898 100644
--- a/lucene/core/src/java/org/apache/lucene/index/VectorValues.java
+++ b/lucene/core/src/java/org/apache/lucene/index/VectorValues.java
@@ -19,7 +19,6 @@ package org.apache.lucene.index;
 import java.io.IOException;
 import org.apache.lucene.document.KnnVectorField;
 import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.util.BytesRef;
 
 /**
  * This class provides access to per-document floating point vector values indexed as {@link
@@ -58,16 +57,4 @@ public abstract class VectorValues extends DocIdSetIterator {
    * @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 array return by {@link #vectorValue}. 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();
-  }
 }
diff --git a/lucene/core/src/test/org/apache/lucene/document/TestField.java b/lucene/core/src/test/org/apache/lucene/document/TestField.java
index 534f06ff7a6..6671f19cfa8 100644
--- a/lucene/core/src/test/org/apache/lucene/document/TestField.java
+++ b/lucene/core/src/test/org/apache/lucene/document/TestField.java
@@ -633,7 +633,6 @@ public class TestField extends LuceneTestCase {
         VectorValues floatValues = r.leaves().get(0).reader().getVectorValues("float");
         assertEquals(1, floatValues.size());
         assertNotEquals(NO_MORE_DOCS, floatValues.nextDoc());
-        assertNotNull(floatValues.binaryValue());
         assertEquals(vector.length, floatValues.vectorValue().length);
         assertEquals(vector[0], floatValues.vectorValue()[0], 0);
         assertEquals(NO_MORE_DOCS, floatValues.nextDoc());