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 2023/01/03 18:17:33 UTC

[GitHub] [lucene] benwtrent opened a new pull request, #12064: Create new KnnByteVectorField and KnnVectorsReader#getByteVectorValues(String)

benwtrent opened a new pull request, #12064:
URL: https://github.com/apache/lucene/pull/12064

   This completes the refactoring as described in: https://github.com/apache/lucene/issues/11963
   
   This commit:
    - splits out `ByteVectorValues` from `VectorValues`.
    - Adds `getByteVectorValues(String field)` to `KnnVectorsReader`
    - Adds a new `KnnByteVectorField` and disallows `BytesRef` values in the `KnnVectorField`
    - No longer allows `ByteVectorValues` to be read from a `KnnVectorField`.
   
   These refactors are difficult to split up any further.


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


[GitHub] [lucene] jpountz commented on pull request #12064: Create new KnnByteVectorField and KnnVectorsReader#getByteVectorValues(String)

Posted by GitBox <gi...@apache.org>.
jpountz commented on PR #12064:
URL: https://github.com/apache/lucene/pull/12064#issuecomment-1374770702

   I haven't thought much about it but I'd prefer to avoid increasing the number of vector types that are supported, 2 already feels like a lot. For instance, terms and points only understand byte[] and push the problem of encoding data (numbers, strings) to sortable byte[]s higher up. It's a bit more complicated with vectors since it's not only about preserving order but maybe it's not that different? E.g. could hamming distance reuse the byte[] API by introducing a new distance function and half-float/bfloat16 reuse the float[] 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.

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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12064: Create new KnnByteVectorField and KnnVectorsReader#getByteVectorValues(String)

Posted by GitBox <gi...@apache.org>.
benwtrent commented on code in PR #12064:
URL: https://github.com/apache/lucene/pull/12064#discussion_r1066187876


##########
lucene/core/src/java/org/apache/lucene/index/ByteVectorValues.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.document.KnnByteVectorField;
+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
+ * KnnByteVectorField}.
+ *
+ * @lucene.experimental
+ */
+public abstract class ByteVectorValues extends DocIdSetIterator {
+
+  /** The maximum length of a vector */
+  public static final int MAX_DIMENSIONS = 1024;
+
+  /** Sole constructor */
+  protected ByteVectorValues() {}
+
+  /** Return the dimension of the vectors */
+  public abstract int dimension();
+
+  /**
+   * Return the number of vectors for this field.
+   *
+   * @return the number of vectors returned by this iterator
+   */
+  public abstract int size();
+
+  @Override
+  public final long cost() {
+    return size();
+  }
+
+  /**
+   * 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 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();
+  }

Review Comment:
   @jpountz correct. We could remove it, it would just be some work around `addField`, merging, and writing.



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


[GitHub] [lucene] rmuir commented on pull request #12064: Create new KnnByteVectorField and KnnVectorsReader#getByteVectorValues(String)

Posted by GitBox <gi...@apache.org>.
rmuir commented on PR #12064:
URL: https://github.com/apache/lucene/pull/12064#issuecomment-1377152361

   > I agree. We need to address this. Makes me wonder about the work done here: #10177. Seems promising, though the cost of flush increases (because of clustering), but the data structure seems to fit WAY better inside of Lucene.
   
   another pathetic, unscalable in-memory toy. i'll reject it as we don't need yet another HNSW, it is the last thing we need.
   
   please, ignore such garbage, ppl can do that in-memory stuff with a jazillion python libraries. this is a search engine.
   
   look at scalable algorithms *ONLY*, e.g. DiskANN is the baseline for https://big-ann-benchmarks.com/, so that seems like a reasonable start.


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


[GitHub] [lucene] rmuir commented on pull request #12064: Create new KnnByteVectorField and KnnVectorsReader#getByteVectorValues(String)

Posted by GitBox <gi...@apache.org>.
rmuir commented on PR #12064:
URL: https://github.com/apache/lucene/pull/12064#issuecomment-1374841078

   Yes, lets design for today. Personally I will push back against new vector types/functions as long as performance is in its current state.


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


[GitHub] [lucene] jpountz commented on pull request #12064: Create new KnnByteVectorField and KnnVectorsReader#getByteVectorValues(String)

Posted by GitBox <gi...@apache.org>.
jpountz commented on PR #12064:
URL: https://github.com/apache/lucene/pull/12064#issuecomment-1378387639

   @benwtrent Would you mind working on a backport PR, since there are a few conflicts that need resolving?


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


[GitHub] [lucene] jpountz merged pull request #12064: Create new KnnByteVectorField and KnnVectorsReader#getByteVectorValues(String)

Posted by GitBox <gi...@apache.org>.
jpountz merged PR #12064:
URL: https://github.com/apache/lucene/pull/12064


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


[GitHub] [lucene] jpountz commented on a diff in pull request #12064: Create new KnnByteVectorField and KnnVectorsReader#getByteVectorValues(String)

Posted by GitBox <gi...@apache.org>.
jpountz commented on code in PR #12064:
URL: https://github.com/apache/lucene/pull/12064#discussion_r1061739690


##########
lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene94/Lucene94HnswVectorsReader.java:
##########
@@ -233,12 +234,19 @@ public void checkIntegrity() throws IOException {
   @Override
   public VectorValues getVectorValues(String field) throws IOException {
     FieldEntry fieldEntry = fields.get(field);
-    VectorValues values = OffHeapVectorValues.load(fieldEntry, vectorData);
-    if (fieldEntry.vectorEncoding == VectorEncoding.BYTE) {
-      return new ExpandingVectorValues(values);
-    } else {
-      return values;
+    if (fieldEntry.vectorEncoding != VectorEncoding.FLOAT32) {
+      return null;
     }
+    return OffHeapVectorValues.load(fieldEntry, vectorData);
+  }
+
+  @Override
+  public ByteVectorValues getByteVectorValues(String field) throws IOException {
+    FieldEntry fieldEntry = fields.get(field);
+    if (fieldEntry.vectorEncoding != VectorEncoding.BYTE) {
+      return null;
+    }

Review Comment:
   this `if` statement should be unnecessary since CodecReader already checks field infos? Can we fail hard if the encoding is not right instead of returning null?



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


[GitHub] [lucene] benwtrent commented on pull request #12064: Create new KnnByteVectorField and KnnVectorsReader#getByteVectorValues(String)

Posted by GitBox <gi...@apache.org>.
benwtrent commented on PR #12064:
URL: https://github.com/apache/lucene/pull/12064#issuecomment-1371336800

   @jpountz 
   
   > One thing I'd like to see is whether we can avoid making AbstractVectorValues public and duplicate logic for byte and float vector values in e.g. VectorScorer and CheckIndex.
   
   I can see about that. The downside is that internally, when writing the vectors, having a known `binaryValue` really simplifies things. I will see if I can get the same simplifications in another way.


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


[GitHub] [lucene] benwtrent commented on pull request #12064: Create new KnnByteVectorField and KnnVectorsReader#getByteVectorValues(String)

Posted by GitBox <gi...@apache.org>.
benwtrent commented on PR #12064:
URL: https://github.com/apache/lucene/pull/12064#issuecomment-1375565264

   >  E.g. could hamming distance reuse the byte[] API by introducing a new distance function and half-float/bfloat16 reuse the float[] API?
   
   Hamming distance for binary vectors will be a bitwise operation. So, we store the binary vectors as some numerical type (some folks use int). If we use the `byte` that means we use 4x as many operations vs `int`. We can cross that bridge when we come to it I suppose.
   
   > Yes, lets design for today. 
   
   OK, I will rewrite to remove the abstract class.
   
   > Personally I will push back against new vector types/functions as long as performance is in its current state.
   
   I agree. We need to address this. Makes me wonder about the work done here: https://github.com/apache/lucene/issues/10177. Seems promising, though the cost of flush increases (because of clustering), but the data structure seems to fit WAY better inside of Lucene.


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


[GitHub] [lucene] jpountz commented on a diff in pull request #12064: Create new KnnByteVectorField and KnnVectorsReader#getByteVectorValues(String)

Posted by GitBox <gi...@apache.org>.
jpountz commented on code in PR #12064:
URL: https://github.com/apache/lucene/pull/12064#discussion_r1066124189


##########
lucene/core/src/java/org/apache/lucene/index/ByteVectorValues.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.document.KnnByteVectorField;
+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
+ * KnnByteVectorField}.
+ *
+ * @lucene.experimental
+ */
+public abstract class ByteVectorValues extends DocIdSetIterator {
+
+  /** The maximum length of a vector */
+  public static final int MAX_DIMENSIONS = 1024;
+
+  /** Sole constructor */
+  protected ByteVectorValues() {}
+
+  /** Return the dimension of the vectors */
+  public abstract int dimension();
+
+  /**
+   * Return the number of vectors for this field.
+   *
+   * @return the number of vectors returned by this iterator
+   */
+  public abstract int size();
+
+  @Override
+  public final long cost() {
+    return size();
+  }
+
+  /**
+   * 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 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();
+  }

Review Comment:
   Are you adding it for consistency with `VectorValues`?



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


[GitHub] [lucene] benwtrent commented on pull request #12064: Create new KnnByteVectorField and KnnVectorsReader#getByteVectorValues(String)

Posted by GitBox <gi...@apache.org>.
benwtrent commented on PR #12064:
URL: https://github.com/apache/lucene/pull/12064#issuecomment-1372294789

   Digging into it more, removing `AbstractVectorValues` will add a fair bit of extra code to the KnnVectorWriters and testing (though testing is a lesser concern I suppose). 
   
   My thoughts on keeping it are that eventually, we will want to add support to binary vectors (to be used specifically with hamming distance) and half-float (or float16, admittedly, this one may wait until JVM has better float16 support).
   
   I am not sure there are other vector encodings we will want to support, but I can see Lucene supporting at least these 4 (including our byte & float32) eventually.
   
   There is already a fair bit of duplication. If the prevailing opinion is completely remove `AbstractVectorValues<T>` and make the writers handle individual vector encodings (instead of relying on the underlying BytesRef), I will comply.
   
   What say you @rmuir && @jpountz ?


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