You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2022/08/18 01:01:02 UTC

[GitHub] [lucene] jtibshirani commented on a diff in pull request #1054: LUCENE-10577: enable quantization of HNSW vectors to 8 bits

jtibshirani commented on code in PR #1054:
URL: https://github.com/apache/lucene/pull/1054#discussion_r948548244


##########
lucene/core/src/java/org/apache/lucene/search/KnnVectorQuery.java:
##########
@@ -133,22 +130,21 @@ private TopDocs searchLeaf(LeafReaderContext ctx, Weight filterWeight) throws IO
       return NO_RESULTS;
     }
 
-    BitSet bitSet = createBitSet(scorer.iterator(), liveDocs, maxDoc);
-    BitSetIterator filterIterator = new BitSetIterator(bitSet, bitSet.cardinality());
+    BitSet acceptDocs = createBitSet(scorer.iterator(), liveDocs, maxDoc);
 
-    if (filterIterator.cost() <= k) {
+    if (acceptDocs.cardinality() <= k) {

Review Comment:
   Whenever possible, we should avoiding calling `cardinality` multiple times since it can run in linear time. I thought the original logic was clearer (but I'm biased since I wrote it 😊 )



##########
lucene/core/src/java/org/apache/lucene/index/VectorEncoding.java:
##########
@@ -0,0 +1,45 @@
+/*
+ * 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;
+
+/** The numeric datatype of the vector values. */
+public enum VectorEncoding {
+
+  /**
+   * Encodes vector using 8 bits of precision per sample. Use only with DOT_PRODUCT similarity.

Review Comment:
   Is it still true that it should only be used with DOT_PRODUCT similarity?



##########
lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsFormat.java:
##########
@@ -76,6 +78,15 @@ public static KnnVectorsFormat forName(String name) {
   /** Returns a {@link KnnVectorsReader} to read the vectors from the index. */
   public abstract KnnVectorsReader fieldsReader(SegmentReadState state) throws IOException;
 
+  /**
+   * Returns the current KnnVectorsFormat version number. Indexes written using the format will be
+   * "stamped" with this version.
+   */
+  public int currentVersion() {

Review Comment:
   It seems confusing to have a new concept of "version" separate from the codec version. It's only used in `BaseKnnVectorsFormatTestCase` -- could we instead make the `randomVectorEncoding` overridable? It would default to all encodings but older codecs could override it and just return float32?



##########
lucene/core/src/java/org/apache/lucene/document/KnnVectorField.java:
##########
@@ -117,6 +160,21 @@ public KnnVectorField(String name, float[] vector, FieldType fieldType) {
     fieldsData = vector;
   }
 
+  /**
+   * Creates a numeric vector field. Fields are single-valued: each document has either one value or
+   * no value. Vectors of a single field share the same dimension and similarity function.
+   *
+   * @param name field name
+   * @param vector value
+   * @param fieldType field type
+   * @throws IllegalArgumentException if any parameter is null, or the vector is empty or has
+   *     dimension &gt; 1024.
+   */
+  public KnnVectorField(String name, BytesRef vector, FieldType fieldType) {

Review Comment:
   I think this method is only meant to be used with `VectorEncoding.BYTE`? Then it'd be good to validate this on the `FieldType`. The same thought applies to the float-oriented constructor.



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene94/Lucene94HnswVectorsWriter.java:
##########
@@ -249,6 +261,29 @@ private void writeSortingField(FieldWriter fieldData, int maxDoc, Sorter.DocMap
         mockGraph);
   }
 
+  private long writeSortedFloat32Vectors(FieldWriter<?> fieldData, int[] ordMap)
+      throws IOException {
+    long vectorDataOffset = vectorData.alignFilePointer(Float.BYTES);
+    final ByteBuffer buffer =
+        ByteBuffer.allocate(fieldData.dim * Float.BYTES).order(ByteOrder.LITTLE_ENDIAN);
+    final BytesRef binaryValue = new BytesRef(buffer.array());
+    for (int ordinal : ordMap) {
+      float[] vector = (float[]) fieldData.vectors.get(ordinal);
+      buffer.asFloatBuffer().put(vector);
+      vectorData.writeBytes(binaryValue.bytes, binaryValue.offset, binaryValue.length);
+    }
+    return vectorDataOffset;
+  }
+
+  private long writeSortedByteVectors(FieldWriter<?> fieldData, int[] ordMap) throws IOException {
+    long vectorDataOffset = vectorData.alignFilePointer(Float.BYTES);
+    for (int ordinal : ordMap) {
+      byte[] vector = (byte[]) fieldData.vectors.get(ordinal);

Review Comment:
   Is it correct to cast to `byte[]` here? I think that these values are `BytesRef`.



##########
lucene/core/src/java/org/apache/lucene/util/VectorUtil.java:
##########
@@ -213,4 +243,48 @@ public static void add(float[] u, float[] v) {
       u[i] += v[i];
     }
   }
+
+  /**
+   * Dot product computed over signed bytes.
+   *
+   * @param a bytes containing a vector
+   * @param b bytes containing another vector, of the same dimension
+   * @return the value of the dot product of the two vectors
+   */
+  public static float dotProduct(BytesRef a, BytesRef b) {
+    assert a.length == b.length;
+    int total = 0;
+    int aOffset = a.offset, bOffset = b.offset;
+    for (int i = 0; i < a.length; i++) {
+      total += a.bytes[aOffset++] * b.bytes[bOffset++];
+    }
+    return total;
+  }
+
+  /**
+   * Dot product score computed over signed bytes, scaled to be in [0, 1].
+   *
+   * @param a bytes containing a vector
+   * @param b bytes containing another vector, of the same dimension
+   * @return the value of the similarity function applied to the two vectors
+   */
+  public static float dotProductScore(BytesRef a, BytesRef b) {
+    // divide by 2 * 2^14 (maximum absolute value of product of 2 signed bytes) * len
+    return (1 + dotProduct(a, b)) / (float) (a.length * (1 << 15));
+  }
+
+  /**
+   * Convert a floating point vector to an array of bytes using casting; the vector values should be
+   * in [-128,127]
+   *
+   * @param vector a vector
+   * @return a new BytesRef containing the vector's values cast to byte.
+   */
+  public static BytesRef toBytesRef(float[] vector) {

Review Comment:
   Should we verify that each element lies within the expected range? This would definitely indicate an error, but right now we silently allow it and just cast the floats to bytes.



##########
lucene/core/src/java/org/apache/lucene/codecs/KnnFieldVectorsWriter.java:
##########
@@ -20,8 +20,12 @@
 import java.io.IOException;
 import org.apache.lucene.util.Accountable;
 
-/** Vectors' writer for a field */
-public abstract class KnnFieldVectorsWriter implements Accountable {
+/**
+ * Vectors' writer for a field
+ *
+ * @param <T> an array type; the type of vectors to be written
+ */
+public abstract class KnnFieldVectorsWriter<T> implements Accountable {

Review Comment:
   One of our goals discussed on the JIRA was to avoid having exposing multiple element types in the APIs. We accomplished that for the read APIs where we always use `float[]`. But in the write APIs we expose separate `float[]` and `BytesRef` versions. This affects a bunch of places (`KnnVectorField`, all the writer implementations, `IndexingChain`, etc.)
   
   I wonder if we could standardize on one type for the write APIs too, for example always accept `float[]`. That could help remove some places where we need to check the encoding (like `IndexingChain`) and remove some generics?



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

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

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


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