You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@solr.apache.org by "eliaporciani (via GitHub)" <gi...@apache.org> on 2023/03/07 14:43:23 UTC

[GitHub] [solr] eliaporciani opened a new pull request, #1435: SOLR-16674: Introduce dense vector byte encoding

eliaporciani opened a new pull request, #1435:
URL: https://github.com/apache/solr/pull/1435

   https://issues.apache.org/jira/browse/SOLR-16674
   
   # Description
   
   Lucene has introduced the BYTE encoding for dense vector fields.
   The values have to be in the range (-128, +127). 
   
   # Solution
   
   Add a parameter to DenseVectorField for choosing the encoding.
   vectorEncoding:
   * FLOAT32 (default)
   * BYTE
   
   If the byte encoding is chosen, a float array is transformed into a byte array (rounding the values and checking that the values are within the boundaries). This byte array is used for both the index and stored fields.
   
   The knnQParser is not influenced by the new encoding.
   
   # Tests
   
   Added tests in DenseVectorFieldTest and KnnQParserTest:
   * Test indexing and queries with BYTE encoding.
   * Test correct exceptions are raised in case of errors
   * Test stored fields are stored in the correct format
   * Test atomic updates work with BYTE encoding
   
   
   # Checklist
   
   Please review the following and check all that apply:
   
   - [x] I have reviewed the guidelines for [How to Contribute](https://wiki.apache.org/solr/HowToContribute) and my code conforms to the standards described there to the best of my ability.
   - [x] I have created a Jira issue and added the issue ID to my pull request title.
   - [x] I have given Solr maintainers [access](https://help.github.com/en/articles/allowing-changes-to-a-pull-request-branch-created-from-a-fork) to contribute to my PR branch. (optional but recommended)
   - [x] I have developed this patch against the `main` branch.
   - [x] I have run `./gradlew check`.
   - [x] I have added tests for my changes.
   - [x] I have added documentation for the [Reference Guide](https://github.com/apache/solr/tree/main/solr/solr-ref-guide)
   


-- 
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@solr.apache.org

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


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


[GitHub] [solr] cpoerschke commented on a diff in pull request #1435: SOLR-16674: Introduce dense vector byte encoding

Posted by "cpoerschke (via GitHub)" <gi...@apache.org>.
cpoerschke commented on code in PR #1435:
URL: https://github.com/apache/solr/pull/1435#discussion_r1136807166


##########
solr/core/src/java/org/apache/solr/schema/DenseVectorField.java:
##########
@@ -46,19 +50,17 @@
  */
 public class DenseVectorField extends FloatPointField {
   public static final String HNSW_ALGORITHM = "hnsw";
-
   public static final String DEFAULT_KNN_ALGORITHM = HNSW_ALGORITHM;
   static final String KNN_VECTOR_DIMENSION = "vectorDimension";
   static final String KNN_SIMILARITY_FUNCTION = "similarityFunction";
-
   static final String KNN_ALGORITHM = "knnAlgorithm";
   static final String HNSW_MAX_CONNECTIONS = "hnswMaxConnections";
   static final String HNSW_BEAM_WIDTH = "hnswBeamWidth";
-
+  static final String VECTOR_ENCODING = "vectorEncoding";
+  private final VectorEncoding DEFAULT_VECTOR_ENCODING = VectorEncoding.FLOAT32;
+  private final VectorSimilarityFunction DEFAULT_SIMILARITY = VectorSimilarityFunction.EUCLIDEAN;

Review Comment:
   minor: keep `DEFAULT_SIMILARITY` at its current location adjacent to `similarityFunction`



##########
solr/core/src/java/org/apache/solr/schema/DenseVectorField.java:
##########
@@ -106,19 +114,22 @@ public void init(IndexSchema schema, Map<String, String> args) {
     args.remove(KNN_SIMILARITY_FUNCTION);
 
     this.knnAlgorithm = args.getOrDefault(KNN_ALGORITHM, DEFAULT_KNN_ALGORITHM);
-
     args.remove(KNN_ALGORITHM);
 
+    this.vectorEncoding =
+        ofNullable(args.get(VECTOR_ENCODING))
+            .map(value -> VectorEncoding.valueOf(value.toUpperCase(Locale.ROOT)))
+            .orElse(DEFAULT_VECTOR_ENCODING);
+    ;
+
+    args.remove(VECTOR_ENCODING);

Review Comment:
   minor (consistency with existing style)
   ```suggestion
               .orElse(DEFAULT_VECTOR_ENCODING);
       args.remove(VECTOR_ENCODING);
   ```



##########
solr/core/src/java/org/apache/solr/schema/DenseVectorField.java:
##########
@@ -179,24 +206,37 @@ public List<IndexableField> createFields(SchemaField field, Object value) {
               + "', expected format:'[f1, f2, f3...fn]' e.g. [1.0, 3.4, 5.6]",

Review Comment:
   subjective: if byte encoding is used might it be confusing to see floats in the example here?
   ```suggestion
                 + "', expected format:'[f1, f2, f3...fn]'",
   ```



##########
solr/core/src/java/org/apache/solr/schema/DenseVectorField.java:
##########
@@ -165,10 +180,22 @@ public void checkSchemaField(final SchemaField field) throws SolrException {
 
   @Override
   public List<IndexableField> createFields(SchemaField field, Object value) {
-    ArrayList<IndexableField> fields = new ArrayList<>();
-    float[] parsedVector;
     try {
-      parsedVector = parseVector(value);
+      ArrayList<IndexableField> fields = new ArrayList<>();
+      VectorValue vectorValue = new VectorValue(value);
+      if (field.indexed()) {
+        fields.add(createField(field, vectorValue));
+      }
+      if (field.stored()) {
+        if (vectorEncoding.equals(VectorEncoding.FLOAT32)) {
+          for (float vectorElement : vectorValue.getFloatVector()) {
+            fields.add(getStoredField(field, vectorElement));
+          }
+        } else {
+          fields.add(new StoredField(field.getName(), vectorValue.getByteVector()));
+        }

Review Comment:
   ```suggestion
           } else if (vectorEncoding.equals(VectorEncoding.BYTE)) {
             fields.add(new StoredField(field.getName(), vectorValue.getByteVector()));
           } else {
             should not happen, throw something (mentioning the unsupported vectorEncoding value)
           }
   ```



##########
solr/core/src/java/org/apache/solr/schema/DenseVectorField.java:
##########
@@ -179,24 +206,37 @@ public List<IndexableField> createFields(SchemaField field, Object value) {
               + "', expected format:'[f1, f2, f3...fn]' e.g. [1.0, 3.4, 5.6]",
           e);
     }
+  }
 
-    if (field.indexed()) {
-      fields.add(createField(field, parsedVector));
-    }
-    if (field.stored()) {
-      fields.ensureCapacity(parsedVector.length + 1);
-      for (float vectorElement : parsedVector) {
-        fields.add(getStoredField(field, vectorElement));
-      }
+  @Override
+  public IndexableField createField(SchemaField field, Object vectorValue) {
+    if (vectorValue == null) return null;
+    VectorValue typedVectorValue = (VectorValue) vectorValue;
+    if (vectorEncoding.equals(VectorEncoding.BYTE)) {
+      return new KnnVectorField(
+          field.getName(), typedVectorValue.getByteVector(), similarityFunction);
+    } else {
+      return new KnnVectorField(
+          field.getName(), typedVectorValue.getFloatVector(), similarityFunction);

Review Comment:
   ```suggestion
       } else if (vectorEncoding.equals(VectorEncoding.FLOAT31)) {
         return new KnnVectorField(
             field.getName(), typedVectorValue.getFloatVector(), similarityFunction);
       } else {
         return null; // or throw something
   ```



##########
solr/core/src/java/org/apache/solr/schema/DenseVectorField.java:
##########
@@ -179,24 +206,37 @@ public List<IndexableField> createFields(SchemaField field, Object value) {
               + "', expected format:'[f1, f2, f3...fn]' e.g. [1.0, 3.4, 5.6]",
           e);
     }
+  }
 
-    if (field.indexed()) {
-      fields.add(createField(field, parsedVector));
-    }
-    if (field.stored()) {
-      fields.ensureCapacity(parsedVector.length + 1);
-      for (float vectorElement : parsedVector) {
-        fields.add(getStoredField(field, vectorElement));
-      }
+  @Override
+  public IndexableField createField(SchemaField field, Object vectorValue) {
+    if (vectorValue == null) return null;
+    VectorValue typedVectorValue = (VectorValue) vectorValue;
+    if (vectorEncoding.equals(VectorEncoding.BYTE)) {
+      return new KnnVectorField(
+          field.getName(), typedVectorValue.getByteVector(), similarityFunction);
+    } else {
+      return new KnnVectorField(
+          field.getName(), typedVectorValue.getFloatVector(), similarityFunction);
     }
-    return fields;
   }
 
   @Override
-  public IndexableField createField(SchemaField field, Object parsedVector) {
-    if (parsedVector == null) return null;
-    float[] typedVector = (float[]) parsedVector;
-    return new KnnVectorField(field.getName(), typedVector, similarityFunction);
+  public Object toObject(IndexableField f) {
+    if (vectorEncoding.equals(VectorEncoding.BYTE)) {
+      BytesRef bytesRef = f.binaryValue();
+      if (bytesRef != null) {
+        List<Number> ret = new ArrayList<>();

Review Comment:
   Wondering if the required capacity of the array is knowable at this point e.g. based on dimension maybe?
   ```suggestion
           List<Number> ret = new ArrayList<>(dimension);
   ```



##########
solr/core/src/test/org/apache/solr/schema/DenseVectorFieldTest.java:
##########
@@ -458,4 +495,128 @@ public void query_functionQueryUsage_shouldThrowException() throws Exception {
       deleteCore();
     }
   }
+
+  @Test
+  public void denseVectorField_shouldBePresentAfterAtomicUpdate() throws Exception {
+    try {
+      initCore("solrconfig.xml", "schema-densevector.xml");
+      SolrInputDocument doc = new SolrInputDocument();
+      doc.addField("id", "0");
+      doc.addField("vector", Arrays.asList(1.1, 2.2, 3.3, 4.4));
+      doc.addField("vector_byte_encoding", Arrays.asList(5.5, 6.6, 7.7, 8.8));
+      doc.addField("string_field", "test");
+
+      assertU(adoc(doc));
+      assertU(commit());
+
+      assertJQ(
+          req("q", "id:0", "fl", "*"),
+          "/response/docs/[0]/vector==[1.1,2.2,3.3,4.4]",
+          "/response/docs/[0]/vector_byte_encoding==[5,6,7,8]",
+          "/response/docs/[0]/string_field==test");
+
+      SolrInputDocument updateDoc = new SolrInputDocument();
+      updateDoc.addField("id", "0");
+      updateDoc.addField("string_field", ImmutableMap.of("set", "other test"));
+      assertU(adoc(updateDoc));
+      assertU(commit());
+
+      assertJQ(
+          req("q", "id:0", "fl", "*"),
+          "/response/docs/[0]/vector==[1.1,2.2,3.3,4.4]",
+          "/response/docs/[0]/vector_byte_encoding==[5,6,7,8]",
+          "/response/docs/[0]/string_field=='other test'");
+
+    } finally {
+      deleteCore();
+    }
+  }
+
+  @Test
+  public void denseVectorFieldOnAtomicUpdate_shouldBeUpdatedCorrectly() throws Exception {
+    try {
+      initCore("solrconfig.xml", "schema-densevector.xml");
+      SolrInputDocument doc = new SolrInputDocument();
+      doc.addField("id", "0");
+      doc.addField("vector", Arrays.asList(1.1, 2.2, 3.3, 4.4));
+      doc.addField("vector_byte_encoding", Arrays.asList(5.5, 6.6, 7.7, 8.8));

Review Comment:
   ```suggestion
         doc.addField("vector_byte_encoding", Arrays.asList(5, 6, 7, 8));
   ```



##########
solr/core/src/test/org/apache/solr/schema/DenseVectorFieldTest.java:
##########
@@ -458,4 +495,128 @@ public void query_functionQueryUsage_shouldThrowException() throws Exception {
       deleteCore();
     }
   }
+
+  @Test
+  public void denseVectorField_shouldBePresentAfterAtomicUpdate() throws Exception {
+    try {
+      initCore("solrconfig.xml", "schema-densevector.xml");
+      SolrInputDocument doc = new SolrInputDocument();
+      doc.addField("id", "0");
+      doc.addField("vector", Arrays.asList(1.1, 2.2, 3.3, 4.4));
+      doc.addField("vector_byte_encoding", Arrays.asList(5.5, 6.6, 7.7, 8.8));

Review Comment:
   subjective: decimals removal already tested elsewhere?
   ```suggestion
         doc.addField("vector_byte_encoding", Arrays.asList(5, 6, 7, 8));
   ```



##########
solr/core/src/test-files/solr/collection1/conf/schema-densevector.xml:
##########
@@ -20,11 +20,19 @@
 
 <schema name="schema-densevector" version="1.0">
   <fieldType name="string" class="solr.StrField" multiValued="true"/>  
-  <fieldType name="knn_vector" class="solr.DenseVectorField" vectorDimension="4" similarityFunction="cosine"/>
-  
+  <fieldType name="knn_vector" class="solr.DenseVectorField" vectorDimension="4" similarityFunction="cosine" />
+  <fieldType name="plong" class="solr.LongPointField" useDocValuesAsStored="false"/>
+
+  <fieldType name="knn_vector_byte_encoding" class="solr.DenseVectorField" vectorDimension="4" similarityFunction="cosine" vectorEncoding="BYTE"/>
+  <fieldType name="knn_vector_esplicit_float32_encoding" class="solr.DenseVectorField" vectorDimension="4" similarityFunction="cosine" vectorEncoding="FLOAT32"/>

Review Comment:
   ```suggestion
     <fieldType name="knn_vector_explicit_float32_encoding" class="solr.DenseVectorField" vectorDimension="4" similarityFunction="cosine" vectorEncoding="FLOAT32"/>
   ```
   
   or it seems to be unused field type actually?
   
   ```suggestion
   ```



-- 
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@solr.apache.org

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


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


[GitHub] [solr] alessandrobenedetti commented on a diff in pull request #1435: SOLR-16674: Introduce dense vector byte encoding

Posted by "alessandrobenedetti (via GitHub)" <gi...@apache.org>.
alessandrobenedetti commented on code in PR #1435:
URL: https://github.com/apache/solr/pull/1435#discussion_r1154182254


##########
solr/core/src/test/org/apache/solr/schema/DenseVectorFieldTest.java:
##########
@@ -253,10 +332,17 @@ public void parseVector_incorrectElement_shouldThrowException() {
    */
   @Test
   public void parseVector_StringArrayList_shouldParseFloatArray() {
-    toTest = new DenseVectorField(3);
     float[] expected = new float[] {1.1f, 2.2f, 3.3f};
+    MatcherAssert.assertThat(
+        toTestFloatEncoding.getVectorBuilder(Arrays.asList(1.1f, 2.2f, 3.3f)).getFloatVector(),

Review Comment:
   this should be a list of strings



-- 
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@solr.apache.org

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


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


[GitHub] [solr] alessandrobenedetti commented on a diff in pull request #1435: SOLR-16674: Introduce dense vector byte encoding

Posted by "alessandrobenedetti (via GitHub)" <gi...@apache.org>.
alessandrobenedetti commented on code in PR #1435:
URL: https://github.com/apache/solr/pull/1435#discussion_r1166671404


##########
solr/core/src/java/org/apache/solr/schema/DenseVectorField.java:
##########
@@ -165,38 +183,69 @@ public void checkSchemaField(final SchemaField field) throws SolrException {
 
   @Override
   public List<IndexableField> createFields(SchemaField field, Object value) {
-    ArrayList<IndexableField> fields = new ArrayList<>();
-    float[] parsedVector;
     try {
-      parsedVector = parseVector(value);
+      ArrayList<IndexableField> fields = new ArrayList<>();
+      VectorBuilder vectorBuilder = getVectorBuilder(value, VectorBuilder.BuilderPhase.INDEX);
+
+      if (field.indexed()) {
+        fields.add(createField(field, vectorBuilder));
+      }
+      if (field.stored()) {
+        switch (vectorEncoding) {
+          case FLOAT32:
+            fields.ensureCapacity(vectorBuilder.getFloatVector().length + 1);
+            for (float vectorElement : vectorBuilder.getFloatVector()) {
+              fields.add(getStoredField(field, vectorElement));
+            }
+            break;
+          case BYTE:
+            fields.add(new StoredField(field.getName(), vectorBuilder.getByteVector()));
+            break;
+        }
+      }
+      return fields;
     } catch (RuntimeException e) {
       throw new SolrException(
           SolrException.ErrorCode.SERVER_ERROR,
-          "Error while creating field '"
-              + field
-              + "' from value '"
-              + value
-              + "', expected format:'[f1, f2, f3...fn]' e.g. [1.0, 3.4, 5.6]",
+          "Error while creating field '" + field + "' from value '" + value + "'",
           e);
     }
+  }
 
-    if (field.indexed()) {
-      fields.add(createField(field, parsedVector));
-    }
-    if (field.stored()) {
-      fields.ensureCapacity(parsedVector.length + 1);
-      for (float vectorElement : parsedVector) {
-        fields.add(getStoredField(field, vectorElement));
-      }
+  @Override
+  public IndexableField createField(SchemaField field, Object vectorValue) {
+    if (vectorValue == null) return null;
+    VectorBuilder vectorBuilder = (VectorBuilder) vectorValue;
+    switch (vectorEncoding) {
+      case BYTE:
+        return new KnnByteVectorField(
+            field.getName(), vectorBuilder.getByteVector(), similarityFunction);
+      case FLOAT32:
+        return new KnnFloatVectorField(
+            field.getName(), vectorBuilder.getFloatVector(), similarityFunction);
+      default:
+        throw new SolrException(
+            SolrException.ErrorCode.SERVER_ERROR,
+            "Unexpected state. Vector Encoding: " + vectorEncoding);
     }
-    return fields;
   }
 
   @Override
-  public IndexableField createField(SchemaField field, Object parsedVector) {
-    if (parsedVector == null) return null;
-    float[] typedVector = (float[]) parsedVector;
-    return new KnnVectorField(field.getName(), typedVector, similarityFunction);
+  public Object toObject(IndexableField f) {

Review Comment:
   can you refresh my memory on this method?



##########
solr/core/src/java/org/apache/solr/schema/DenseVectorField.java:
##########
@@ -205,53 +254,194 @@ public IndexableField createField(SchemaField field, Object parsedVector) {
    * org.apache.solr.handler.loader.CSVLoader} produces an ArrayList of String - {@link
    * org.apache.solr.handler.loader.JsonLoader} produces an ArrayList of Double - {@link
    * org.apache.solr.handler.loader.JavabinLoader} produces an ArrayList of Float
-   *
-   * @param inputValue - An {@link ArrayList} containing the elements of the vector
-   * @return the vector parsed
    */
-  float[] parseVector(Object inputValue) {
-    if (!(inputValue instanceof List)) {
-      throw new SolrException(
-          SolrException.ErrorCode.BAD_REQUEST,
-          "incorrect vector format."
-              + " The expected format is an array :'[f1,f2..f3]' where each element f is a float");
+  public VectorBuilder getVectorBuilder(Object inputValue, VectorBuilder.BuilderPhase phase) {
+    switch (vectorEncoding) {
+      case FLOAT32:
+        return new VectorBuilder.Float32VectorBuilder(dimension, inputValue, phase);
+      case BYTE:
+        return new VectorBuilder.ByteVectorBuilder(dimension, inputValue, phase);
+      default:
+        throw new SolrException(
+            SolrException.ErrorCode.SERVER_ERROR,
+            "Unexpected state. Vector Encoding: " + vectorEncoding);
     }
-    List<?> inputVector = (List<?>) inputValue;
-    if (inputVector.size() != dimension) {
-      throw new SolrException(
-          SolrException.ErrorCode.BAD_REQUEST,
-          "incorrect vector dimension."
-              + " The vector value has size "
-              + inputVector.size()
-              + " while it is expected a vector with size "
-              + dimension);
+  }
+
+  abstract static class VectorBuilder {

Review Comment:
   to discuss:  could this be an external class? maybe an util one? 
   Is it more a "VectorParser" rather than a builder maybe?



##########
solr/core/src/java/org/apache/solr/schema/DenseVectorField.java:
##########
@@ -205,53 +254,194 @@ public IndexableField createField(SchemaField field, Object parsedVector) {
    * org.apache.solr.handler.loader.CSVLoader} produces an ArrayList of String - {@link
    * org.apache.solr.handler.loader.JsonLoader} produces an ArrayList of Double - {@link
    * org.apache.solr.handler.loader.JavabinLoader} produces an ArrayList of Float
-   *
-   * @param inputValue - An {@link ArrayList} containing the elements of the vector
-   * @return the vector parsed
    */
-  float[] parseVector(Object inputValue) {
-    if (!(inputValue instanceof List)) {
-      throw new SolrException(
-          SolrException.ErrorCode.BAD_REQUEST,
-          "incorrect vector format."
-              + " The expected format is an array :'[f1,f2..f3]' where each element f is a float");
+  public VectorBuilder getVectorBuilder(Object inputValue, VectorBuilder.BuilderPhase phase) {
+    switch (vectorEncoding) {
+      case FLOAT32:
+        return new VectorBuilder.Float32VectorBuilder(dimension, inputValue, phase);
+      case BYTE:
+        return new VectorBuilder.ByteVectorBuilder(dimension, inputValue, phase);
+      default:
+        throw new SolrException(
+            SolrException.ErrorCode.SERVER_ERROR,
+            "Unexpected state. Vector Encoding: " + vectorEncoding);
     }
-    List<?> inputVector = (List<?>) inputValue;
-    if (inputVector.size() != dimension) {
-      throw new SolrException(
-          SolrException.ErrorCode.BAD_REQUEST,
-          "incorrect vector dimension."
-              + " The vector value has size "
-              + inputVector.size()
-              + " while it is expected a vector with size "
-              + dimension);
+  }
+
+  abstract static class VectorBuilder {
+
+    public static enum BuilderPhase {
+      INDEX,
+      QUERY
+    }
+
+    protected BuilderPhase builderPhase;
+
+    protected int dimension;
+    protected Object inputValue;
+
+    public float[] getFloatVector() {
+      throw new UnsupportedOperationException("Requested wrong vector type");
+    }
+
+    public byte[] getByteVector() {
+      throw new UnsupportedOperationException("Requested wrong vector type");
+    }
+
+    protected void parseVector() {
+      switch (builderPhase) {
+        case INDEX:
+          parseIndexVector();
+          break;
+        case QUERY:
+          parseQueryVector();
+          break;
+      }
     }
 
-    float[] vector = new float[dimension];
-    if (inputVector.get(0) instanceof CharSequence) {
+    protected void parseIndexVector() {
+      if (!(inputValue instanceof List)) {
+        throw new SolrException(
+            SolrException.ErrorCode.BAD_REQUEST, "incorrect vector format. " + errorMessage());
+      }
+      List<?> inputVector = (List<?>) inputValue;
+      if (inputVector.size() != dimension) {
+        throw new SolrException(
+            SolrException.ErrorCode.BAD_REQUEST,
+            "incorrect vector dimension."
+                + " The vector value has size "
+                + inputVector.size()
+                + " while it is expected a vector with size "
+                + dimension);
+      }

Review Comment:
   maybe this duplicated code can be extracted?



##########
solr/core/src/java/org/apache/solr/schema/DenseVectorField.java:
##########
@@ -205,53 +254,194 @@ public IndexableField createField(SchemaField field, Object parsedVector) {
    * org.apache.solr.handler.loader.CSVLoader} produces an ArrayList of String - {@link
    * org.apache.solr.handler.loader.JsonLoader} produces an ArrayList of Double - {@link
    * org.apache.solr.handler.loader.JavabinLoader} produces an ArrayList of Float
-   *
-   * @param inputValue - An {@link ArrayList} containing the elements of the vector
-   * @return the vector parsed
    */
-  float[] parseVector(Object inputValue) {
-    if (!(inputValue instanceof List)) {
-      throw new SolrException(
-          SolrException.ErrorCode.BAD_REQUEST,
-          "incorrect vector format."
-              + " The expected format is an array :'[f1,f2..f3]' where each element f is a float");
+  public VectorBuilder getVectorBuilder(Object inputValue, VectorBuilder.BuilderPhase phase) {
+    switch (vectorEncoding) {
+      case FLOAT32:
+        return new VectorBuilder.Float32VectorBuilder(dimension, inputValue, phase);
+      case BYTE:
+        return new VectorBuilder.ByteVectorBuilder(dimension, inputValue, phase);
+      default:
+        throw new SolrException(
+            SolrException.ErrorCode.SERVER_ERROR,
+            "Unexpected state. Vector Encoding: " + vectorEncoding);
     }
-    List<?> inputVector = (List<?>) inputValue;
-    if (inputVector.size() != dimension) {
-      throw new SolrException(
-          SolrException.ErrorCode.BAD_REQUEST,
-          "incorrect vector dimension."
-              + " The vector value has size "
-              + inputVector.size()
-              + " while it is expected a vector with size "
-              + dimension);
+  }
+
+  abstract static class VectorBuilder {
+
+    public static enum BuilderPhase {
+      INDEX,
+      QUERY
+    }
+
+    protected BuilderPhase builderPhase;
+
+    protected int dimension;
+    protected Object inputValue;
+
+    public float[] getFloatVector() {
+      throw new UnsupportedOperationException("Requested wrong vector type");

Review Comment:
   wrong? why wrong? what should it be the 'correct' one?



-- 
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@solr.apache.org

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


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


[GitHub] [solr] eliaporciani commented on a diff in pull request #1435: SOLR-16674: Introduce dense vector byte encoding

Posted by "eliaporciani (via GitHub)" <gi...@apache.org>.
eliaporciani commented on code in PR #1435:
URL: https://github.com/apache/solr/pull/1435#discussion_r1138497868


##########
solr/core/src/test-files/solr/collection1/conf/schema-densevector.xml:
##########
@@ -20,11 +20,19 @@
 
 <schema name="schema-densevector" version="1.0">
   <fieldType name="string" class="solr.StrField" multiValued="true"/>  
-  <fieldType name="knn_vector" class="solr.DenseVectorField" vectorDimension="4" similarityFunction="cosine"/>
-  
+  <fieldType name="knn_vector" class="solr.DenseVectorField" vectorDimension="4" similarityFunction="cosine" />
+  <fieldType name="plong" class="solr.LongPointField" useDocValuesAsStored="false"/>
+
+  <fieldType name="knn_vector_byte_encoding" class="solr.DenseVectorField" vectorDimension="4" similarityFunction="cosine" vectorEncoding="BYTE"/>
+  <fieldType name="knn_vector_esplicit_float32_encoding" class="solr.DenseVectorField" vectorDimension="4" similarityFunction="cosine" vectorEncoding="FLOAT32"/>

Review Comment:
   Removed



-- 
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@solr.apache.org

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


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


[GitHub] [solr] alessandrobenedetti commented on a diff in pull request #1435: SOLR-16674: Introduce dense vector byte encoding

Posted by "alessandrobenedetti (via GitHub)" <gi...@apache.org>.
alessandrobenedetti commented on code in PR #1435:
URL: https://github.com/apache/solr/pull/1435#discussion_r1154188607


##########
solr/core/src/test/org/apache/solr/search/neural/KnnQParserTest.java:
##########
@@ -205,6 +214,31 @@ public void correctVectorField_shouldSearchOnThatField() {
         "//result/doc[3]/str[@name='id'][.='12']");
   }
 
+  @Test
+  public void vectorByteEncodingField_shouldSearchOnThatField() {
+    String vectorToSearch = "[2.0, 2.0, 1.0, 3.0]";
+
+    assertQ(
+        req(CommonParams.Q, "{!knn f=vector_byte_encoding topK=2}" + vectorToSearch, "fl", "id"),
+        "//result[@numFound='2']",
+        "//result/doc[1]/str[@name='id'][.='2']",
+        "//result/doc[2]/str[@name='id'][.='3']");
+  }
+
+  @Test
+  public void vectorByteEncodingField_shouldNotApproximateQueryVectorValueToIntegerValues() {
+    String vectorToSearch = "[7.9f, 2.9f, 1.9f, 3.9f]";
+
+    // If we consider the vectors with floating values, document 7 has an higher score than document
+    // 6.
+    // If instead we take only the integer part, document 6 gets an higher score.

Review Comment:
   having various if in the comment make this test difficult to read, where is this condition? 



-- 
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@solr.apache.org

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


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


[GitHub] [solr] eliaporciani commented on a diff in pull request #1435: SOLR-16674: Introduce dense vector byte encoding

Posted by "eliaporciani (via GitHub)" <gi...@apache.org>.
eliaporciani commented on code in PR #1435:
URL: https://github.com/apache/solr/pull/1435#discussion_r1138497043


##########
solr/core/src/java/org/apache/solr/schema/DenseVectorField.java:
##########
@@ -179,24 +206,37 @@ public List<IndexableField> createFields(SchemaField field, Object value) {
               + "', expected format:'[f1, f2, f3...fn]' e.g. [1.0, 3.4, 5.6]",
           e);
     }
+  }
 
-    if (field.indexed()) {
-      fields.add(createField(field, parsedVector));
-    }
-    if (field.stored()) {
-      fields.ensureCapacity(parsedVector.length + 1);
-      for (float vectorElement : parsedVector) {
-        fields.add(getStoredField(field, vectorElement));
-      }
+  @Override
+  public IndexableField createField(SchemaField field, Object vectorValue) {
+    if (vectorValue == null) return null;
+    VectorValue typedVectorValue = (VectorValue) vectorValue;
+    if (vectorEncoding.equals(VectorEncoding.BYTE)) {
+      return new KnnVectorField(
+          field.getName(), typedVectorValue.getByteVector(), similarityFunction);
+    } else {
+      return new KnnVectorField(
+          field.getName(), typedVectorValue.getFloatVector(), similarityFunction);

Review Comment:
   I changed the if with a switch. However, there are only two possibilities here and if another value is provided, it fails in the enum parsing.



##########
solr/core/src/java/org/apache/solr/schema/DenseVectorField.java:
##########
@@ -179,24 +206,37 @@ public List<IndexableField> createFields(SchemaField field, Object value) {
               + "', expected format:'[f1, f2, f3...fn]' e.g. [1.0, 3.4, 5.6]",

Review Comment:
   Changed



-- 
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@solr.apache.org

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


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


[GitHub] [solr] alessandrobenedetti commented on a diff in pull request #1435: SOLR-16674: Introduce dense vector byte encoding

Posted by "alessandrobenedetti (via GitHub)" <gi...@apache.org>.
alessandrobenedetti commented on code in PR #1435:
URL: https://github.com/apache/solr/pull/1435#discussion_r1154179416


##########
solr/core/src/test/org/apache/solr/schema/DenseVectorFieldTest.java:
##########
@@ -171,57 +182,111 @@ public void parseVector_NotAList_shouldThrowException() {
             "Single string value should throw an exception",
             SolrException.class,
             () -> {
-              toTest.parseVector("string");
+              toTestFloatEncoding.getVectorBuilder("string").getFloatVector();
             });
     MatcherAssert.assertThat(
         thrown.getMessage(),
         is(
-            "incorrect vector format."
-                + " The expected format is an array :'[f1,f2..f3]' where each element f is a float"));
+            "incorrect vector format. The expected format is:'[f1,f2..f3]' where each element f is a float"));
+
+    thrown =
+        assertThrows(
+            "Single string value should throw an exception",
+            SolrException.class,
+            () -> {
+              toTestByteEncoding.getVectorBuilder("string").getByteVector();
+            });
+    MatcherAssert.assertThat(
+        thrown.getMessage(),
+        is(
+            "incorrect vector format. The expected format is:'[fb,b2..b3]' where each element b is a byte (-128 to 127)"));

Review Comment:
   same as above, I suspect there's a typo in the log message
   



-- 
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@solr.apache.org

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


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


[GitHub] [solr] alessandrobenedetti merged pull request #1435: SOLR-16674: Introduce dense vector byte encoding

Posted by "alessandrobenedetti (via GitHub)" <gi...@apache.org>.
alessandrobenedetti merged PR #1435:
URL: https://github.com/apache/solr/pull/1435


-- 
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@solr.apache.org

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


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


[GitHub] [solr] eliaporciani commented on a diff in pull request #1435: SOLR-16674: Introduce dense vector byte encoding

Posted by "eliaporciani (via GitHub)" <gi...@apache.org>.
eliaporciani commented on code in PR #1435:
URL: https://github.com/apache/solr/pull/1435#discussion_r1138498266


##########
solr/core/src/test/org/apache/solr/schema/DenseVectorFieldTest.java:
##########
@@ -458,4 +495,128 @@ public void query_functionQueryUsage_shouldThrowException() throws Exception {
       deleteCore();
     }
   }
+
+  @Test
+  public void denseVectorField_shouldBePresentAfterAtomicUpdate() throws Exception {
+    try {
+      initCore("solrconfig.xml", "schema-densevector.xml");
+      SolrInputDocument doc = new SolrInputDocument();
+      doc.addField("id", "0");
+      doc.addField("vector", Arrays.asList(1.1, 2.2, 3.3, 4.4));
+      doc.addField("vector_byte_encoding", Arrays.asList(5.5, 6.6, 7.7, 8.8));

Review Comment:
   Changed as it is already tested. 



##########
solr/core/src/test/org/apache/solr/schema/DenseVectorFieldTest.java:
##########
@@ -458,4 +495,128 @@ public void query_functionQueryUsage_shouldThrowException() throws Exception {
       deleteCore();
     }
   }
+
+  @Test
+  public void denseVectorField_shouldBePresentAfterAtomicUpdate() throws Exception {
+    try {
+      initCore("solrconfig.xml", "schema-densevector.xml");
+      SolrInputDocument doc = new SolrInputDocument();
+      doc.addField("id", "0");
+      doc.addField("vector", Arrays.asList(1.1, 2.2, 3.3, 4.4));
+      doc.addField("vector_byte_encoding", Arrays.asList(5.5, 6.6, 7.7, 8.8));
+      doc.addField("string_field", "test");
+
+      assertU(adoc(doc));
+      assertU(commit());
+
+      assertJQ(
+          req("q", "id:0", "fl", "*"),
+          "/response/docs/[0]/vector==[1.1,2.2,3.3,4.4]",
+          "/response/docs/[0]/vector_byte_encoding==[5,6,7,8]",
+          "/response/docs/[0]/string_field==test");
+
+      SolrInputDocument updateDoc = new SolrInputDocument();
+      updateDoc.addField("id", "0");
+      updateDoc.addField("string_field", ImmutableMap.of("set", "other test"));
+      assertU(adoc(updateDoc));
+      assertU(commit());
+
+      assertJQ(
+          req("q", "id:0", "fl", "*"),
+          "/response/docs/[0]/vector==[1.1,2.2,3.3,4.4]",
+          "/response/docs/[0]/vector_byte_encoding==[5,6,7,8]",
+          "/response/docs/[0]/string_field=='other test'");
+
+    } finally {
+      deleteCore();
+    }
+  }
+
+  @Test
+  public void denseVectorFieldOnAtomicUpdate_shouldBeUpdatedCorrectly() throws Exception {
+    try {
+      initCore("solrconfig.xml", "schema-densevector.xml");
+      SolrInputDocument doc = new SolrInputDocument();
+      doc.addField("id", "0");
+      doc.addField("vector", Arrays.asList(1.1, 2.2, 3.3, 4.4));
+      doc.addField("vector_byte_encoding", Arrays.asList(5.5, 6.6, 7.7, 8.8));

Review Comment:
   Same as above



-- 
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@solr.apache.org

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


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


[GitHub] [solr] eliaporciani commented on a diff in pull request #1435: SOLR-16674: Introduce dense vector byte encoding

Posted by "eliaporciani (via GitHub)" <gi...@apache.org>.
eliaporciani commented on code in PR #1435:
URL: https://github.com/apache/solr/pull/1435#discussion_r1138495435


##########
solr/core/src/java/org/apache/solr/schema/DenseVectorField.java:
##########
@@ -165,10 +180,22 @@ public void checkSchemaField(final SchemaField field) throws SolrException {
 
   @Override
   public List<IndexableField> createFields(SchemaField field, Object value) {
-    ArrayList<IndexableField> fields = new ArrayList<>();
-    float[] parsedVector;
     try {
-      parsedVector = parseVector(value);
+      ArrayList<IndexableField> fields = new ArrayList<>();
+      VectorValue vectorValue = new VectorValue(value);
+      if (field.indexed()) {
+        fields.add(createField(field, vectorValue));
+      }
+      if (field.stored()) {
+        if (vectorEncoding.equals(VectorEncoding.FLOAT32)) {
+          for (float vectorElement : vectorValue.getFloatVector()) {
+            fields.add(getStoredField(field, vectorElement));
+          }
+        } else {
+          fields.add(new StoredField(field.getName(), vectorValue.getByteVector()));
+        }

Review Comment:
   Changed with switch



-- 
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@solr.apache.org

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


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


[GitHub] [solr] alessandrobenedetti commented on a diff in pull request #1435: SOLR-16674: Introduce dense vector byte encoding

Posted by "alessandrobenedetti (via GitHub)" <gi...@apache.org>.
alessandrobenedetti commented on code in PR #1435:
URL: https://github.com/apache/solr/pull/1435#discussion_r1154177094


##########
solr/core/src/java/org/apache/solr/schema/DenseVectorField.java:
##########
@@ -205,53 +257,144 @@ public IndexableField createField(SchemaField field, Object parsedVector) {
    * org.apache.solr.handler.loader.CSVLoader} produces an ArrayList of String - {@link
    * org.apache.solr.handler.loader.JsonLoader} produces an ArrayList of Double - {@link
    * org.apache.solr.handler.loader.JavabinLoader} produces an ArrayList of Float
-   *
-   * @param inputValue - An {@link ArrayList} containing the elements of the vector
-   * @return the vector parsed
    */
-  float[] parseVector(Object inputValue) {
-    if (!(inputValue instanceof List)) {
-      throw new SolrException(
-          SolrException.ErrorCode.BAD_REQUEST,
-          "incorrect vector format."
-              + " The expected format is an array :'[f1,f2..f3]' where each element f is a float");
+  public VectorBuilder getVectorBuilder(Object inputValue) {
+    switch (vectorEncoding) {
+      case FLOAT32:
+        return new VectorBuilder.Float32VectorBuilder(dimension, inputValue);
+      case BYTE:
+        return new VectorBuilder.ByteVectorBuilder(dimension, inputValue);
+      default:
+        throw new SolrException(
+            SolrException.ErrorCode.SERVER_ERROR,
+            "Unexpected state. Vector Encoding: " + vectorEncoding);
     }
-    List<?> inputVector = (List<?>) inputValue;
-    if (inputVector.size() != dimension) {
-      throw new SolrException(
-          SolrException.ErrorCode.BAD_REQUEST,
-          "incorrect vector dimension."
-              + " The vector value has size "
-              + inputVector.size()
-              + " while it is expected a vector with size "
-              + dimension);
+  }
+
+  abstract static class VectorBuilder {
+
+    protected int dimension;
+    protected Object inputValue;
+
+    public float[] getFloatVector() {
+      throw new RuntimeException("Not implemented");
     }
 
-    float[] vector = new float[dimension];
-    if (inputVector.get(0) instanceof CharSequence) {
-      for (int i = 0; i < dimension; i++) {
-        try {
-          vector[i] = Float.parseFloat(inputVector.get(i).toString());
-        } catch (NumberFormatException e) {
-          throw new SolrException(
-              SolrException.ErrorCode.BAD_REQUEST,
-              "incorrect vector element: '"
-                  + inputVector.get(i)
-                  + "'. The expected format is:'[f1,f2..f3]' where each element f is a float");
+    protected BytesRef getByteVector() {
+      throw new RuntimeException("Not implemented");
+    }
+
+    protected void parseVector() {
+      if (!(inputValue instanceof List)) {
+        throw new SolrException(
+            SolrException.ErrorCode.BAD_REQUEST, "incorrect vector format. " + errorMessage());
+      }
+      List<?> inputVector = (List<?>) inputValue;
+      if (inputVector.size() != dimension) {
+        throw new SolrException(
+            SolrException.ErrorCode.BAD_REQUEST,
+            "incorrect vector dimension."
+                + " The vector value has size "
+                + inputVector.size()
+                + " while it is expected a vector with size "
+                + dimension);
+      }
+
+      if (inputVector.get(0) instanceof CharSequence) {
+        for (int i = 0; i < dimension; i++) {
+          try {
+            addStringElement(inputVector.get(i).toString());
+          } catch (NumberFormatException e) {
+            throw new SolrException(
+                SolrException.ErrorCode.BAD_REQUEST,
+                "incorrect vector element: '" + inputVector.get(i) + "'. " + errorMessage());
+          }
         }
+      } else if (inputVector.get(0) instanceof Number) {
+        for (int i = 0; i < dimension; i++) {
+          addNumberElement((Number) inputVector.get(i));
+        }
+      } else {
+        throw new SolrException(
+            SolrException.ErrorCode.BAD_REQUEST, "incorrect vector format. " + errorMessage());
       }
-    } else if (inputVector.get(0) instanceof Number) {
-      for (int i = 0; i < dimension; i++) {
-        vector[i] = ((Number) inputVector.get(i)).floatValue();
+    }
+
+    protected abstract void addNumberElement(Number element);
+
+    protected abstract void addStringElement(String element);
+
+    protected abstract String errorMessage();
+
+    static class ByteVectorBuilder extends VectorBuilder {
+      private BytesRefBuilder byteRefBuilder;
+      private BytesRef byteVector;
+
+      public ByteVectorBuilder(int dimension, Object inputValue) {
+        this.dimension = dimension;
+        this.inputValue = inputValue;
+      }
+
+      @Override
+      public BytesRef getByteVector() {
+        if (byteVector == null) {
+          byteRefBuilder = new BytesRefBuilder();
+          parseVector();
+          byteVector = byteRefBuilder.toBytesRef();
+        }
+        return byteVector;
+      }
+
+      @Override
+      protected void addNumberElement(Number element) {
+        byteRefBuilder.append(element.byteValue());
+      }
+
+      @Override
+      protected void addStringElement(String element) {
+        byteRefBuilder.append(Byte.parseByte(element));
+      }
+
+      @Override
+      protected String errorMessage() {
+        return "The expected format is:'[fb,b2..b3]' where each element b is a byte (-128 to 127)";

Review Comment:
   [fb,b2..b3] -> [b1,b2..b3] maybe?



-- 
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@solr.apache.org

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


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


[GitHub] [solr] alessandrobenedetti commented on a diff in pull request #1435: SOLR-16674: Introduce dense vector byte encoding

Posted by "alessandrobenedetti (via GitHub)" <gi...@apache.org>.
alessandrobenedetti commented on code in PR #1435:
URL: https://github.com/apache/solr/pull/1435#discussion_r1140498109


##########
solr/core/src/java/org/apache/solr/schema/DenseVectorField.java:
##########
@@ -165,38 +178,69 @@ public void checkSchemaField(final SchemaField field) throws SolrException {
 
   @Override
   public List<IndexableField> createFields(SchemaField field, Object value) {
-    ArrayList<IndexableField> fields = new ArrayList<>();
-    float[] parsedVector;
     try {
-      parsedVector = parseVector(value);
+      ArrayList<IndexableField> fields = new ArrayList<>();
+      VectorValue vectorValue = new VectorValue(value);
+      if (field.indexed()) {
+        fields.add(createField(field, vectorValue));
+      }
+      if (field.stored()) {
+        switch (vectorEncoding) {
+          case FLOAT32:
+            for (float vectorElement : vectorValue.getFloatVector()) {
+              fields.add(getStoredField(field, vectorElement));
+            }
+            break;
+          case BYTE:
+            fields.add(new StoredField(field.getName(), vectorValue.getByteVector()));
+            break;
+        }
+      }
+      return fields;
     } catch (RuntimeException e) {
       throw new SolrException(
           SolrException.ErrorCode.SERVER_ERROR,
           "Error while creating field '"
               + field
               + "' from value '"
               + value
-              + "', expected format:'[f1, f2, f3...fn]' e.g. [1.0, 3.4, 5.6]",
+              + "', expected format:'[f1, f2, f3...fn]'",
           e);
     }
+  }
 
-    if (field.indexed()) {
-      fields.add(createField(field, parsedVector));
-    }
-    if (field.stored()) {
-      fields.ensureCapacity(parsedVector.length + 1);

Review Comment:
   any reason for the removal of this?



-- 
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@solr.apache.org

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


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


[GitHub] [solr] eliaporciani commented on a diff in pull request #1435: SOLR-16674: Introduce dense vector byte encoding

Posted by "eliaporciani (via GitHub)" <gi...@apache.org>.
eliaporciani commented on code in PR #1435:
URL: https://github.com/apache/solr/pull/1435#discussion_r1138494823


##########
solr/core/src/java/org/apache/solr/schema/DenseVectorField.java:
##########
@@ -46,19 +50,17 @@
  */
 public class DenseVectorField extends FloatPointField {
   public static final String HNSW_ALGORITHM = "hnsw";
-
   public static final String DEFAULT_KNN_ALGORITHM = HNSW_ALGORITHM;
   static final String KNN_VECTOR_DIMENSION = "vectorDimension";
   static final String KNN_SIMILARITY_FUNCTION = "similarityFunction";
-
   static final String KNN_ALGORITHM = "knnAlgorithm";
   static final String HNSW_MAX_CONNECTIONS = "hnswMaxConnections";
   static final String HNSW_BEAM_WIDTH = "hnswBeamWidth";
-
+  static final String VECTOR_ENCODING = "vectorEncoding";
+  private final VectorEncoding DEFAULT_VECTOR_ENCODING = VectorEncoding.FLOAT32;
+  private final VectorSimilarityFunction DEFAULT_SIMILARITY = VectorSimilarityFunction.EUCLIDEAN;

Review Comment:
   fixed



##########
solr/core/src/java/org/apache/solr/schema/DenseVectorField.java:
##########
@@ -106,19 +114,22 @@ public void init(IndexSchema schema, Map<String, String> args) {
     args.remove(KNN_SIMILARITY_FUNCTION);
 
     this.knnAlgorithm = args.getOrDefault(KNN_ALGORITHM, DEFAULT_KNN_ALGORITHM);
-
     args.remove(KNN_ALGORITHM);
 
+    this.vectorEncoding =
+        ofNullable(args.get(VECTOR_ENCODING))
+            .map(value -> VectorEncoding.valueOf(value.toUpperCase(Locale.ROOT)))
+            .orElse(DEFAULT_VECTOR_ENCODING);
+    ;
+
+    args.remove(VECTOR_ENCODING);

Review Comment:
   fixed



-- 
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@solr.apache.org

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


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


[GitHub] [solr] alessandrobenedetti commented on a diff in pull request #1435: SOLR-16674: Introduce dense vector byte encoding

Posted by "alessandrobenedetti (via GitHub)" <gi...@apache.org>.
alessandrobenedetti commented on code in PR #1435:
URL: https://github.com/apache/solr/pull/1435#discussion_r1149553669


##########
solr/core/src/java/org/apache/solr/schema/DenseVectorField.java:
##########
@@ -205,53 +257,144 @@ public IndexableField createField(SchemaField field, Object parsedVector) {
    * org.apache.solr.handler.loader.CSVLoader} produces an ArrayList of String - {@link
    * org.apache.solr.handler.loader.JsonLoader} produces an ArrayList of Double - {@link
    * org.apache.solr.handler.loader.JavabinLoader} produces an ArrayList of Float
-   *
-   * @param inputValue - An {@link ArrayList} containing the elements of the vector
-   * @return the vector parsed
    */
-  float[] parseVector(Object inputValue) {
-    if (!(inputValue instanceof List)) {
-      throw new SolrException(
-          SolrException.ErrorCode.BAD_REQUEST,
-          "incorrect vector format."
-              + " The expected format is an array :'[f1,f2..f3]' where each element f is a float");
+  public VectorBuilder getVectorBuilder(Object inputValue) {
+    switch (vectorEncoding) {
+      case FLOAT32:

Review Comment:
   Question:
   I have seen the polimorphic approach and it's cool.
   But at the moment each class implementing the abstract will actually implement just one method (or the getByteVector or the getFloatVector).
   
   Can't we make the  vectorBuilder class depending on a type, and the getVector, being type parametric?
   I'm not sure if this will remove entirely the necessity of abstract  +2 implementers (probably not), but at least it will ease this piece of code, because potentially we have initialised the VectorBuilder in a switch case already, and we can just ask for the vector.
   
   Happy to have a call to clarify this



-- 
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@solr.apache.org

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


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


[GitHub] [solr] alessandrobenedetti commented on a diff in pull request #1435: SOLR-16674: Introduce dense vector byte encoding

Posted by "alessandrobenedetti (via GitHub)" <gi...@apache.org>.
alessandrobenedetti commented on code in PR #1435:
URL: https://github.com/apache/solr/pull/1435#discussion_r1154184243


##########
solr/core/src/test/org/apache/solr/schema/DenseVectorFieldTest.java:
##########
@@ -357,6 +452,27 @@ public void query_storedField_shouldBeReturnedInResults() throws Exception {
     }
   }
 
+  @Test
+  public void query_vectorByteEncoded_storedField_shouldBeReturnedInResults() throws Exception {

Review Comment:
   is there also a test for the float 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.

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

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


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


[GitHub] [solr] eliaporciani commented on a diff in pull request #1435: SOLR-16674: Introduce dense vector byte encoding

Posted by "eliaporciani (via GitHub)" <gi...@apache.org>.
eliaporciani commented on code in PR #1435:
URL: https://github.com/apache/solr/pull/1435#discussion_r1147400108


##########
solr/core/src/java/org/apache/solr/schema/DenseVectorField.java:
##########
@@ -165,38 +178,69 @@ public void checkSchemaField(final SchemaField field) throws SolrException {
 
   @Override
   public List<IndexableField> createFields(SchemaField field, Object value) {
-    ArrayList<IndexableField> fields = new ArrayList<>();
-    float[] parsedVector;
     try {
-      parsedVector = parseVector(value);
+      ArrayList<IndexableField> fields = new ArrayList<>();
+      VectorValue vectorValue = new VectorValue(value);
+      if (field.indexed()) {
+        fields.add(createField(field, vectorValue));
+      }
+      if (field.stored()) {
+        switch (vectorEncoding) {
+          case FLOAT32:
+            for (float vectorElement : vectorValue.getFloatVector()) {
+              fields.add(getStoredField(field, vectorElement));
+            }
+            break;
+          case BYTE:
+            fields.add(new StoredField(field.getName(), vectorValue.getByteVector()));
+            break;
+        }
+      }
+      return fields;
     } catch (RuntimeException e) {
       throw new SolrException(
           SolrException.ErrorCode.SERVER_ERROR,
           "Error while creating field '"
               + field
               + "' from value '"
               + value
-              + "', expected format:'[f1, f2, f3...fn]' e.g. [1.0, 3.4, 5.6]",
+              + "', expected format:'[f1, f2, f3...fn]'",
           e);
     }
+  }
 
-    if (field.indexed()) {
-      fields.add(createField(field, parsedVector));
-    }
-    if (field.stored()) {
-      fields.ensureCapacity(parsedVector.length + 1);

Review Comment:
   Removed by mistake. I put it back



-- 
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@solr.apache.org

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


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


[GitHub] [solr] alessandrobenedetti commented on a diff in pull request #1435: SOLR-16674: Introduce dense vector byte encoding

Posted by "alessandrobenedetti (via GitHub)" <gi...@apache.org>.
alessandrobenedetti commented on code in PR #1435:
URL: https://github.com/apache/solr/pull/1435#discussion_r1154186439


##########
solr/core/src/test/org/apache/solr/schema/DenseVectorFieldTest.java:
##########
@@ -458,4 +574,127 @@ public void query_functionQueryUsage_shouldThrowException() throws Exception {
       deleteCore();
     }
   }
+
+  @Test
+  public void denseVectorField_shouldBePresentAfterAtomicUpdate() throws Exception {
+    try {
+      initCore("solrconfig.xml", "schema-densevector.xml");
+      SolrInputDocument doc = new SolrInputDocument();
+      doc.addField("id", "0");
+      doc.addField("vector", Arrays.asList(1.1, 2.2, 3.3, 4.4));
+      doc.addField("vector_byte_encoding", Arrays.asList(5, 6, 7, 8));
+      doc.addField("string_field", "test");
+
+      assertU(adoc(doc));
+      assertU(commit());
+
+      assertJQ(
+          req("q", "id:0", "fl", "*"),
+          "/response/docs/[0]/vector==[1.1,2.2,3.3,4.4]",
+          "/response/docs/[0]/vector_byte_encoding==[5,6,7,8]",
+          "/response/docs/[0]/string_field==test");
+
+      SolrInputDocument updateDoc = new SolrInputDocument();
+      updateDoc.addField("id", "0");
+      updateDoc.addField("string_field", Map.of("set", "other test"));
+      assertU(adoc(updateDoc));
+      assertU(commit());
+
+      assertJQ(
+          req("q", "id:0", "fl", "*"),
+          "/response/docs/[0]/vector==[1.1,2.2,3.3,4.4]",
+          "/response/docs/[0]/vector_byte_encoding==[5,6,7,8]",
+          "/response/docs/[0]/string_field=='other test'");
+
+    } finally {
+      deleteCore();
+    }
+  }
+
+  @Test
+  public void denseVectorFieldOnAtomicUpdate_shouldBeUpdatedCorrectly() throws Exception {
+    try {
+      initCore("solrconfig.xml", "schema-densevector.xml");
+      SolrInputDocument doc = new SolrInputDocument();
+      doc.addField("id", "0");
+      doc.addField("vector", Arrays.asList(1.1, 2.2, 3.3, 4.4));
+      doc.addField("vector_byte_encoding", Arrays.asList(5, 6, 7, 8));
+      doc.addField("string_field", "test");
+
+      assertU(adoc(doc));
+      assertU(commit());
+
+      assertJQ(
+          req("q", "id:0", "fl", "*"),
+          "/response/docs/[0]/vector==[1.1,2.2,3.3,4.4]",
+          "/response/docs/[0]/vector_byte_encoding==[5,6,7,8]",
+          "/response/docs/[0]/string_field==test");
+
+      SolrInputDocument updateDoc = new SolrInputDocument();
+      updateDoc.addField("id", "0");
+      updateDoc.addField("vector", Map.of("set", Arrays.asList(9.2, 2.2, 3.3, 5.2)));
+      updateDoc.addField("vector_byte_encoding", Map.of("set", Arrays.asList(8, 3, 1, 3)));
+      assertU(adoc(updateDoc));
+      assertU(commit());
+
+      assertJQ(
+          req("q", "id:0", "fl", "*"),
+          "/response/docs/[0]/vector==[9.2,2.2,3.3,5.2]",
+          "/response/docs/[0]/vector_byte_encoding==[8,3,1,3]",
+          "/response/docs/[0]/string_field=='test'");
+
+    } finally {
+      deleteCore();
+    }
+  }
+
+  @Test
+  public void denseVectorByteEncoding_shouldRaiseExceptionWithValuesOutsideBoundaries()
+      throws Exception {
+    try {
+      initCore("solrconfig.xml", "schema-densevector.xml");
+      SolrInputDocument doc = new SolrInputDocument();
+      doc.addField("id", "0");
+      doc.addField("vector_byte_encoding", Arrays.asList(127.5, 6.6, 7.7, 8.8));
+
+      RuntimeException thrown =
+          assertThrows(
+              "Incorrect elements should throw an exception",
+              SolrException.class,
+              () -> {
+                assertU(adoc(doc));
+              });
+
+      MatcherAssert.assertThat(
+          thrown.getCause().getMessage(),
+          is(
+              "Error while creating field 'vector_byte_encoding{type=knn_vector_byte_encoding,properties=indexed,stored}' from value '[127.5, 6.6, 7.7, 8.8]', expected format:'[f1, f2, f3...fn]'"));

Review Comment:
   isn't this message incorrect?
   For byte the expected format should be '[b1, b2, b3...bn]' rather than '[f1, f2, f3...fn]'



-- 
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@solr.apache.org

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


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


[GitHub] [solr] alessandrobenedetti commented on a diff in pull request #1435: SOLR-16674: Introduce dense vector byte encoding

Posted by "alessandrobenedetti (via GitHub)" <gi...@apache.org>.
alessandrobenedetti commented on code in PR #1435:
URL: https://github.com/apache/solr/pull/1435#discussion_r1154182662


##########
solr/core/src/test/org/apache/solr/schema/DenseVectorFieldTest.java:
##########
@@ -253,10 +332,17 @@ public void parseVector_incorrectElement_shouldThrowException() {
    */
   @Test
   public void parseVector_StringArrayList_shouldParseFloatArray() {
-    toTest = new DenseVectorField(3);
     float[] expected = new float[] {1.1f, 2.2f, 3.3f};
+    MatcherAssert.assertThat(
+        toTestFloatEncoding.getVectorBuilder(Arrays.asList(1.1f, 2.2f, 3.3f)).getFloatVector(),
+        is(expected));
+  }
 
-    MatcherAssert.assertThat(toTest.parseVector(Arrays.asList("1.1", "2.2", "3.3")), is(expected));
+  @Test
+  public void parseVector_StringArrayList_shouldParseByteArray() {
+    BytesRef expected = newBytesRef(new byte[] {1, 2, 3});
+    MatcherAssert.assertThat(
+        toTestByteEncoding.getVectorBuilder(Arrays.asList(1, 2, 3)).getByteVector(), is(expected));

Review Comment:
   should be list of string



-- 
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@solr.apache.org

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


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


[GitHub] [solr] alessandrobenedetti commented on a diff in pull request #1435: SOLR-16674: Introduce dense vector byte encoding

Posted by "alessandrobenedetti (via GitHub)" <gi...@apache.org>.
alessandrobenedetti commented on code in PR #1435:
URL: https://github.com/apache/solr/pull/1435#discussion_r1154173619


##########
solr/core/src/java/org/apache/solr/schema/DenseVectorField.java:
##########
@@ -205,53 +257,144 @@ public IndexableField createField(SchemaField field, Object parsedVector) {
    * org.apache.solr.handler.loader.CSVLoader} produces an ArrayList of String - {@link
    * org.apache.solr.handler.loader.JsonLoader} produces an ArrayList of Double - {@link
    * org.apache.solr.handler.loader.JavabinLoader} produces an ArrayList of Float
-   *
-   * @param inputValue - An {@link ArrayList} containing the elements of the vector
-   * @return the vector parsed
    */
-  float[] parseVector(Object inputValue) {
-    if (!(inputValue instanceof List)) {
-      throw new SolrException(
-          SolrException.ErrorCode.BAD_REQUEST,
-          "incorrect vector format."
-              + " The expected format is an array :'[f1,f2..f3]' where each element f is a float");
+  public VectorBuilder getVectorBuilder(Object inputValue) {
+    switch (vectorEncoding) {
+      case FLOAT32:
+        return new VectorBuilder.Float32VectorBuilder(dimension, inputValue);
+      case BYTE:
+        return new VectorBuilder.ByteVectorBuilder(dimension, inputValue);
+      default:
+        throw new SolrException(
+            SolrException.ErrorCode.SERVER_ERROR,
+            "Unexpected state. Vector Encoding: " + vectorEncoding);
     }
-    List<?> inputVector = (List<?>) inputValue;
-    if (inputVector.size() != dimension) {
-      throw new SolrException(
-          SolrException.ErrorCode.BAD_REQUEST,
-          "incorrect vector dimension."
-              + " The vector value has size "
-              + inputVector.size()
-              + " while it is expected a vector with size "
-              + dimension);
+  }
+
+  abstract static class VectorBuilder {
+
+    protected int dimension;
+    protected Object inputValue;
+
+    public float[] getFloatVector() {
+      throw new RuntimeException("Not implemented");

Review Comment:
   check if "throw new UnsupportedOperationException" is better



-- 
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@solr.apache.org

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


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


[GitHub] [solr] alessandrobenedetti commented on a diff in pull request #1435: SOLR-16674: Introduce dense vector byte encoding

Posted by "alessandrobenedetti (via GitHub)" <gi...@apache.org>.
alessandrobenedetti commented on code in PR #1435:
URL: https://github.com/apache/solr/pull/1435#discussion_r1154191299


##########
solr/solr-ref-guide/modules/query-guide/pages/dense-vector-search.adoc:
##########
@@ -126,6 +126,18 @@ Accepted values: `hnsw`.
 
 Please note that the `knnAlgorithm` accepted values may change in future releases.
 
+`vectorEncoding`::
++
+[%autowidth,frame=none]
+|===
+|Optional |Default: `FLOAT32`
+|===
++
+(advanced) Specifies the underlying encoding of the dense vector. This affects both the index and the stored fields (if enabled)

Review Comment:
   (advanced) Specifies the underlying encoding of the dense vector elements. This affects memory/disk impact for both the indexed and stored fields (if enabled)



-- 
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@solr.apache.org

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


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


[GitHub] [solr] eliaporciani commented on a diff in pull request #1435: SOLR-16674: Introduce dense vector byte encoding

Posted by "eliaporciani (via GitHub)" <gi...@apache.org>.
eliaporciani commented on code in PR #1435:
URL: https://github.com/apache/solr/pull/1435#discussion_r1138497658


##########
solr/core/src/java/org/apache/solr/schema/DenseVectorField.java:
##########
@@ -179,24 +206,37 @@ public List<IndexableField> createFields(SchemaField field, Object value) {
               + "', expected format:'[f1, f2, f3...fn]' e.g. [1.0, 3.4, 5.6]",
           e);
     }
+  }
 
-    if (field.indexed()) {
-      fields.add(createField(field, parsedVector));
-    }
-    if (field.stored()) {
-      fields.ensureCapacity(parsedVector.length + 1);
-      for (float vectorElement : parsedVector) {
-        fields.add(getStoredField(field, vectorElement));
-      }
+  @Override
+  public IndexableField createField(SchemaField field, Object vectorValue) {
+    if (vectorValue == null) return null;
+    VectorValue typedVectorValue = (VectorValue) vectorValue;
+    if (vectorEncoding.equals(VectorEncoding.BYTE)) {
+      return new KnnVectorField(
+          field.getName(), typedVectorValue.getByteVector(), similarityFunction);
+    } else {
+      return new KnnVectorField(
+          field.getName(), typedVectorValue.getFloatVector(), similarityFunction);
     }
-    return fields;
   }
 
   @Override
-  public IndexableField createField(SchemaField field, Object parsedVector) {
-    if (parsedVector == null) return null;
-    float[] typedVector = (float[]) parsedVector;
-    return new KnnVectorField(field.getName(), typedVector, similarityFunction);
+  public Object toObject(IndexableField f) {
+    if (vectorEncoding.equals(VectorEncoding.BYTE)) {
+      BytesRef bytesRef = f.binaryValue();
+      if (bytesRef != null) {
+        List<Number> ret = new ArrayList<>();

Review Comment:
   You are right. fixed



-- 
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@solr.apache.org

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


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


[GitHub] [solr] dizzu commented on pull request #1435: SOLR-16674: Introduce dense vector byte encoding

Posted by "dizzu (via GitHub)" <gi...@apache.org>.
dizzu commented on PR #1435:
URL: https://github.com/apache/solr/pull/1435#issuecomment-1669531186

   @alessandrobenedetti We're seeing some weird output results for stored byte vectors in Solr 9.3.0 (SolrCloud).
   In a distributed query (default or distrib=true), the stored field vector is outputted as a string with random unicode characters (eg: €\u0016õœF¤‡\n\u000eׁ񷤝>\u0002' ... etc).
   In a non-distributed query(distrib=false), the stored field vector is outputted correctly as a vector [-128, 15, -112, etc]. 
   
   Any ideas about what might be causing this?


-- 
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@solr.apache.org

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


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