You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by so...@apache.org on 2020/11/09 15:46:30 UTC

[lucene-solr] branch master updated: LUCENE-9583: extract separate RandomAccessVectorValues interface (#2037)

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

sokolov pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/master by this push:
     new 8be0cea  LUCENE-9583: extract separate RandomAccessVectorValues interface (#2037)
8be0cea is described below

commit 8be0cea5442c2edab260d0598b920ba832506f80
Author: Michael Sokolov <so...@falutin.net>
AuthorDate: Mon Nov 9 10:46:16 2020 -0500

    LUCENE-9583: extract separate RandomAccessVectorValues interface (#2037)
---
 .../codecs/simpletext/SimpleTextVectorReader.java  | 10 ++--
 .../org/apache/lucene/codecs/VectorWriter.java     | 26 +++++----
 .../codecs/lucene90/Lucene90VectorReader.java      | 17 +++---
 .../lucene/index/RandomAccessVectorValues.java     | 60 +++++++++++++++++++
 .../index/RandomAccessVectorValuesProducer.java    | 32 +++++++++++
 .../java/org/apache/lucene/index/VectorValues.java | 67 ++++------------------
 .../apache/lucene/index/VectorValuesWriter.java    | 28 ++++-----
 .../org/apache/lucene/index/TestVectorValues.java  |  2 +-
 8 files changed, 151 insertions(+), 91 deletions(-)

diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextVectorReader.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextVectorReader.java
index 4c19625..862d0b6 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextVectorReader.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextVectorReader.java
@@ -27,6 +27,8 @@ import org.apache.lucene.codecs.VectorReader;
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.RandomAccessVectorValues;
+import org.apache.lucene.index.RandomAccessVectorValuesProducer;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.VectorValues;
 import org.apache.lucene.search.TopDocs;
@@ -158,7 +160,7 @@ public class SimpleTextVectorReader extends VectorReader {
     }
   }
 
-  private static class SimpleTextVectorValues extends VectorValues implements VectorValues.RandomAccess {
+  private static class SimpleTextVectorValues extends VectorValues implements RandomAccessVectorValues, RandomAccessVectorValuesProducer {
 
     private final BytesRefBuilder scratch = new BytesRefBuilder();
     private final FieldEntry entry;
@@ -205,7 +207,7 @@ public class SimpleTextVectorReader extends VectorReader {
     }
 
     @Override
-    public RandomAccess randomAccess() {
+    public RandomAccessVectorValues randomAccess() {
       return this;
     }
 
@@ -236,8 +238,8 @@ public class SimpleTextVectorReader extends VectorReader {
     }
 
     private void readAllVectors() throws IOException {
-      for (int i = 0; i < values.length; i++) {
-        readVector(values[i]);
+      for (float[] value : values) {
+        readVector(value);
       }
     }
 
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/VectorWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/VectorWriter.java
index a5f9e7f..9b05cc6 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/VectorWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/VectorWriter.java
@@ -26,6 +26,8 @@ import java.util.List;
 import org.apache.lucene.index.DocIDMerger;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.MergeState;
+import org.apache.lucene.index.RandomAccessVectorValues;
+import org.apache.lucene.index.RandomAccessVectorValuesProducer;
 import org.apache.lucene.index.VectorValues;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.util.BytesRef;
@@ -135,7 +137,7 @@ public abstract class VectorWriter implements Closeable {
    * View over multiple VectorValues supporting iterator-style access via DocIdMerger. Maintains a reverse ordinal
    * mapping for documents having values in order to support random access by dense ordinal.
    */
-  private static class VectorValuesMerger extends VectorValues {
+  private static class VectorValuesMerger extends VectorValues implements RandomAccessVectorValuesProducer {
     private final List<VectorValuesSub> subs;
     private final DocIDMerger<VectorValuesSub> docIdMerger;
     private final int[] ordBase;
@@ -198,7 +200,7 @@ public abstract class VectorWriter implements Closeable {
     }
 
     @Override
-    public RandomAccess randomAccess() {
+    public RandomAccessVectorValues randomAccess() {
       return new MergerRandomAccess();
     }
 
@@ -227,14 +229,23 @@ public abstract class VectorWriter implements Closeable {
       return subs.get(0).values.searchStrategy();
     }
 
-    class MergerRandomAccess implements VectorValues.RandomAccess {
+    @Override
+    public TopDocs search(float[] target, int k, int fanout) throws IOException {
+      throw new UnsupportedOperationException();
+    }
 
-      private final List<RandomAccess> raSubs;
+    class MergerRandomAccess implements RandomAccessVectorValues {
+
+      private final List<RandomAccessVectorValues> raSubs;
 
       MergerRandomAccess() {
         raSubs = new ArrayList<>(subs.size());
         for (VectorValuesSub sub : subs) {
-          raSubs.add(sub.values.randomAccess());
+          if (sub.values instanceof RandomAccessVectorValuesProducer) {
+            raSubs.add(((RandomAccessVectorValuesProducer) sub.values).randomAccess());
+          } else {
+            throw new IllegalStateException("Cannot merge VectorValues without support for random access");
+          }
         }
       }
 
@@ -273,11 +284,6 @@ public abstract class VectorWriter implements Closeable {
         throw new UnsupportedOperationException();
       }
 
-      @Override
-      public TopDocs search(float[] target, int k, int fanout) throws IOException {
-        throw new UnsupportedOperationException();
-      }
-
     }
   }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90VectorReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90VectorReader.java
index 9d313d0..1403108 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90VectorReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90VectorReader.java
@@ -29,6 +29,8 @@ import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.RandomAccessVectorValues;
+import org.apache.lucene.index.RandomAccessVectorValuesProducer;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.VectorValues;
 import org.apache.lucene.search.TopDocs;
@@ -196,7 +198,7 @@ public final class Lucene90VectorReader extends VectorReader {
   }
 
   /** Read the vector values from the index input. This supports both iterated and random access. */
-  private final static class OffHeapVectorValues extends VectorValues {
+  private final class OffHeapVectorValues extends VectorValues implements RandomAccessVectorValuesProducer {
 
     final FieldEntry fieldEntry;
     final IndexInput dataIn;
@@ -251,6 +253,11 @@ public final class Lucene90VectorReader extends VectorReader {
     }
 
     @Override
+    public TopDocs search(float[] target, int k, int fanout) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
     public int docID() {
       return doc;
     }
@@ -277,12 +284,12 @@ public final class Lucene90VectorReader extends VectorReader {
     }
 
     @Override
-    public RandomAccess randomAccess() {
+    public RandomAccessVectorValues randomAccess() {
       return new OffHeapRandomAccess(dataIn.clone());
     }
 
 
-    class OffHeapRandomAccess implements VectorValues.RandomAccess {
+    class OffHeapRandomAccess implements RandomAccessVectorValues {
 
       final IndexInput dataIn;
 
@@ -336,10 +343,6 @@ public final class Lucene90VectorReader extends VectorReader {
         dataIn.readBytes(byteBuffer.array(), byteBuffer.arrayOffset(), byteSize);
       }
 
-      @Override
-      public TopDocs search(float[] vector, int topK, int fanout) throws IOException {
-        throw new UnsupportedOperationException();
-      }
     }
   }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/index/RandomAccessVectorValues.java b/lucene/core/src/java/org/apache/lucene/index/RandomAccessVectorValues.java
new file mode 100644
index 0000000..f19ea33
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/index/RandomAccessVectorValues.java
@@ -0,0 +1,60 @@
+/*
+ * 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.util.BytesRef;
+
+/**
+ * Provides random access to vectors by dense ordinal.
+ *
+ * @lucene.experimental
+ */
+public interface RandomAccessVectorValues {
+
+  /**
+   * Return the number of vector values
+   */
+  int size();
+
+  /**
+   * Return the dimension of the returned vector values
+   */
+  int dimension();
+
+  /**
+   * Return the search strategy used to compare these vectors
+   */
+  VectorValues.SearchStrategy searchStrategy();
+
+  /**
+   * Return the vector value indexed at the given ordinal. The provided floating point array may
+   * be shared and overwritten by subsequent calls to this method and {@link #binaryValue(int)}.
+   * @param targetOrd a valid ordinal, &ge; 0 and &lt; {@link #size()}.
+   */
+  float[] vectorValue(int targetOrd) throws IOException;
+
+  /**
+   * Return the vector indexed at the given ordinal value as an array of bytes in a BytesRef;
+   * these are the bytes corresponding to the float array. The provided bytes may be shared and overwritten
+   * by subsequent calls to this method and {@link #vectorValue(int)}.
+   * @param targetOrd a valid ordinal, &ge; 0 and &lt; {@link #size()}.
+   */
+  BytesRef binaryValue(int targetOrd) throws IOException;
+}
diff --git a/lucene/core/src/java/org/apache/lucene/index/RandomAccessVectorValuesProducer.java b/lucene/core/src/java/org/apache/lucene/index/RandomAccessVectorValuesProducer.java
new file mode 100644
index 0000000..beabace
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/index/RandomAccessVectorValuesProducer.java
@@ -0,0 +1,32 @@
+/*
+ * 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;
+
+/**
+ * Something (generally a {@link VectorValues}) that provides a {@link RandomAccessVectorValues}.
+ *
+ * @lucene.experimental
+ */
+public interface RandomAccessVectorValuesProducer {
+  /**
+   * Return a random access interface over this iterator's vectors. Calling the RandomAccess methods will
+   * have no effect on the progress of the iteration or the values returned by this iterator. Successive calls
+   * will retrieve independent copies that do not overwrite each others' returned values.
+   */
+  RandomAccessVectorValues randomAccess();
+}
diff --git a/lucene/core/src/java/org/apache/lucene/index/VectorValues.java b/lucene/core/src/java/org/apache/lucene/index/VectorValues.java
index c6534f0..bd3b69c 100644
--- a/lucene/core/src/java/org/apache/lucene/index/VectorValues.java
+++ b/lucene/core/src/java/org/apache/lucene/index/VectorValues.java
@@ -75,68 +75,23 @@ public abstract class VectorValues extends DocIdSetIterator {
   }
 
   /**
-   * Return a random access interface over this iterator's vectors. Calling the RandomAccess methods will
-   * have no effect on the progress of the iteration or the values returned by this iterator. Successive calls
-   * will retrieve independent copies that do not overwrite each others' returned values.
+   * Return the k nearest neighbor documents as determined by comparison of their vector values
+   * for this field, to the given vector, by the field's search strategy. If the search strategy is
+   * reversed, lower values indicate nearer vectors, otherwise higher scores indicate nearer
+   * vectors. Unlike relevance scores, vector scores may be negative.
+   * @param target the vector-valued query
+   * @param k      the number of docs to return
+   * @param fanout control the accuracy/speed tradeoff - larger values give better recall at higher cost
+   * @return the k nearest neighbor documents, along with their (searchStrategy-specific) scores.
    */
-  public abstract RandomAccess randomAccess();
-
-  /**
-   * Provides random access to vectors by dense ordinal.
-   *
-   * @lucene.experimental
-   */
-  public interface RandomAccess {
-
-    /**
-     * Return the number of vector values
-     */
-    int size();
-
-    /**
-     * Return the dimension of the returned vector values
-     */
-    int dimension();
-
-    /**
-     * Return the search strategy used to compare these vectors
-     */
-    SearchStrategy searchStrategy();
-
-    /**
-     * Return the vector value indexed at the given ordinal. The provided floating point array may
-     * be shared and overwritten by subsequent calls to this method and {@link #binaryValue(int)}.
-     * @param targetOrd a valid ordinal, &ge; 0 and &lt; {@link #size()}.
-     */
-    float[] vectorValue(int targetOrd) throws IOException;
-
-    /**
-     * Return the vector indexed at the given ordinal value as an array of bytes in a BytesRef;
-     * these are the bytes corresponding to the float array. The provided bytes may be shared and overwritten 
-     * by subsequent calls to this method and {@link #vectorValue(int)}.
-     * @param targetOrd a valid ordinal, &ge; 0 and &lt; {@link #size()}.
-     */
-    BytesRef binaryValue(int targetOrd) throws IOException;
-
-    /**
-     * Return the k nearest neighbor documents as determined by comparison of their vector values
-     * for this field, to the given vector, by the field's search strategy. If the search strategy is
-     * reversed, lower values indicate nearer vectors, otherwise higher scores indicate nearer
-     * vectors. Unlike relevance scores, vector scores may be negative.
-     * @param target the vector-valued query
-     * @param k      the number of docs to return
-     * @param fanout control the accuracy/speed tradeoff - larger values give better recall at higher cost
-     * @return the k nearest neighbor documents, along with their (searchStrategy-specific) scores.
-     */
-    TopDocs search(float[] target, int k, int fanout) throws IOException;
-  }
+  public abstract TopDocs search(float[] target, int k, int fanout) throws IOException;
 
   /**
    * Search strategy. This is a label describing the method used during indexing and searching of the vectors in order to
    * determine the nearest neighbors.
    */
   public enum SearchStrategy {
-    /** No search strategy is provided. Note: {@link VectorValues.RandomAccess#search(float[], int, int)}
+    /** No search strategy is provided. Note: {@link VectorValues#search(float[], int, int)}
      * is not supported for fields specifying this strategy. */
     NONE,
 
@@ -174,7 +129,7 @@ public abstract class VectorValues extends DocIdSetIterator {
     }
 
     @Override
-    public RandomAccess randomAccess() {
+    public TopDocs search(float[] target, int k, int fanout) {
       throw new UnsupportedOperationException();
     }
 
diff --git a/lucene/core/src/java/org/apache/lucene/index/VectorValuesWriter.java b/lucene/core/src/java/org/apache/lucene/index/VectorValuesWriter.java
index ae39b3a..5158150 100644
--- a/lucene/core/src/java/org/apache/lucene/index/VectorValuesWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/VectorValuesWriter.java
@@ -98,17 +98,17 @@ class VectorValuesWriter {
     }
   }
 
-  static class SortingVectorValues extends VectorValues {
+  static class SortingVectorValues extends VectorValues implements RandomAccessVectorValuesProducer {
 
     private final VectorValues delegate;
-    private final VectorValues.RandomAccess randomAccess;
+    private final RandomAccessVectorValues randomAccess;
     private final int[] docIdOffsets;
     private final int[] ordMap;
     private int docId = -1;
 
     SortingVectorValues(VectorValues delegate, Sorter.DocMap sortMap) throws IOException {
       this.delegate = delegate;
-      randomAccess = delegate.randomAccess();
+      randomAccess = ((RandomAccessVectorValuesProducer) delegate).randomAccess();
       docIdOffsets = new int[sortMap.size()];
 
       int offset = 1; // 0 means no vector for this (field, document)
@@ -181,10 +181,16 @@ class VectorValuesWriter {
       return size();
     }
 
+
+    @Override
+    public TopDocs search(float[] target, int k, int fanout) {
+      throw new UnsupportedOperationException();
+    }
+
     @Override
-    public RandomAccess randomAccess() {
-      RandomAccess ra = delegate.randomAccess();
-      return new RandomAccess() {
+    public RandomAccessVectorValues randomAccess() {
+
+      return new RandomAccessVectorValues() {
 
         @Override
         public int size() {
@@ -203,7 +209,7 @@ class VectorValuesWriter {
 
         @Override
         public float[] vectorValue(int targetOrd) throws IOException {
-          return ra.vectorValue(ordMap[targetOrd]);
+          return randomAccess.vectorValue(ordMap[targetOrd]);
         }
 
         @Override
@@ -211,15 +217,11 @@ class VectorValuesWriter {
           throw new UnsupportedOperationException();
         }
 
-        @Override
-        public TopDocs search(float[] target, int k, int fanout) {
-          throw new UnsupportedOperationException();
-        }
       };
     }
   }
 
-  private static class BufferedVectorValues extends VectorValues implements VectorValues.RandomAccess {
+  private static class BufferedVectorValues extends VectorValues implements RandomAccessVectorValues, RandomAccessVectorValuesProducer {
 
     final DocsWithFieldSet docsWithField;
 
@@ -249,7 +251,7 @@ class VectorValuesWriter {
     }
 
     @Override
-    public RandomAccess randomAccess() {
+    public RandomAccessVectorValues randomAccess() {
       return this;
     }
 
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestVectorValues.java b/lucene/core/src/test/org/apache/lucene/index/TestVectorValues.java
index 6e54b88..4b7ccbd 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestVectorValues.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestVectorValues.java
@@ -593,7 +593,7 @@ public class TestVectorValues extends LuceneTestCase {
         assertEquals(4f, vectorValues.vectorValue()[0], 0);
         assertEquals(NO_MORE_DOCS, vectorValues.nextDoc());
 
-        VectorValues.RandomAccess ra = vectorValues.randomAccess();
+        RandomAccessVectorValues ra = ((RandomAccessVectorValuesProducer) vectorValues).randomAccess();
         assertEquals(1f, ra.vectorValue(0)[0], 0);
         assertEquals(2f, ra.vectorValue(1)[0], 0);
         assertEquals(4f, ra.vectorValue(2)[0], 0);