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

[GitHub] [lucene] jtibshirani commented on a diff in pull request #792: LUCENE-10502: Use IndexedDISI to store docIds and DirectMonotonicWriter/Reader to handle ordToDoc

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


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene91/Lucene91HnswVectorsReader.java:
##########
@@ -320,13 +323,19 @@ private static class FieldEntry {
     final int numLevels;
     final int dimension;
     private final int size;
-    final int[] ordToDoc;
-    private final IntUnaryOperator ordToDocOperator;
     final int[][] nodesByLevel;
     // for each level the start offsets in vectorIndex file from where to read neighbours
     final long[] graphOffsetsByLevel;
-
-    FieldEntry(DataInput input, VectorSimilarityFunction similarityFunction) throws IOException {
+    final long docsWithFieldOffset;

Review Comment:
   It'd be great to add short comments here explaining what these different values are used for.



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene91/Lucene91HnswVectorsReader.java:
##########
@@ -507,8 +515,90 @@ public BytesRef binaryValue(int targetOrd) throws IOException {
     }
 
     private void readValue(int targetOrd) throws IOException {
-      dataIn.seek((long) targetOrd * byteSize);
-      dataIn.readBytes(byteBuffer.array(), byteBuffer.arrayOffset(), byteSize);
+      slice.seek((long) targetOrd * byteSize);
+      slice.readBytes(byteBuffer.array(), byteBuffer.arrayOffset(), byteSize);
+    }
+
+    public IndexedDISI initDISI(IndexInput vectorData) throws IOException {
+      // dense
+      if (fieldEntry == null || fieldEntry.docsWithFieldOffset == -1) {
+        return null;
+      }
+      assert fieldEntry.docsWithFieldOffset != -2;
+      // sparse
+      return new IndexedDISI(
+          vectorData,
+          fieldEntry.docsWithFieldOffset,
+          fieldEntry.docsWithFieldLength,
+          fieldEntry.jumpTableEntryCount,
+          fieldEntry.denseRankPower,
+          fieldEntry.size);
+    }
+
+    public static final OffHeapVectorValues emptyOffHeapVectorValues(int dimension)

Review Comment:
   I think we should avoid creating this new empty `OffHeapVectorValues` class to keep things simple. Instead we can just assert in `getOffHeapVectorValues` that the field is not empty and make sure to always return early if the field is empty.



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene91/Lucene91HnswVectorsReader.java:
##########
@@ -400,27 +400,42 @@ static class OffHeapVectorValues extends VectorValues
 
     private final int dimension;
     private final int size;
-    private final int[] ordToDoc;
-    private final IntUnaryOperator ordToDocOperator;
+    // dataIn was used to init a new IndexedDIS for #randomAccess()
     private final IndexInput dataIn;
+    private final IndexInput slice;
     private final BytesRef binaryValue;
     private final ByteBuffer byteBuffer;
     private final int byteSize;
     private final float[] value;
+    private final IndexedDISI disi;
+    private final FieldEntry fieldEntry;
+    final DirectMonotonicReader ordToDoc;
 
     private int ord = -1;
     private int doc = -1;
 
-    OffHeapVectorValues(int dimension, int size, int[] ordToDoc, IndexInput dataIn) {
+    OffHeapVectorValues(
+        int dimension, int size, FieldEntry fieldEntry, IndexInput dataIn, IndexInput slice)
+        throws IOException {
       this.dimension = dimension;
       this.size = size;
-      this.ordToDoc = ordToDoc;
-      ordToDocOperator = ordToDoc == null ? IntUnaryOperator.identity() : (ord) -> ordToDoc[ord];
+      this.fieldEntry = fieldEntry;
       this.dataIn = dataIn;
+      this.slice = slice;
+      this.disi = initDISI(dataIn);

Review Comment:
   I think it'd be clearer if we separated out the sparse and dense cases into two different subclasses. Currently it feels pretty fragile/ confusing, for example in `Lucene91HnswVectorsWriter` line 148 we create an `OffHeapVectorValues` that has null for one `dataIn` but not for `slice`. Instead we could just have this be its own class like `OffHeapVectorValues.Dense` that does not require the `dataIn` at all.
   
   I also think we can simplify the dense logic. We don't need different variables `ord` and `doc` because these are always equal in the dense case.
   
   



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene91/Lucene91HnswVectorsReader.java:
##########
@@ -335,23 +344,23 @@ private static class FieldEntry {
       dimension = input.readInt();
       size = input.readInt();
 
-      int denseSparseMarker = input.readByte();
-      if (denseSparseMarker == -1) {
-        ordToDoc = null; // each document has a vector value
+      docsWithFieldOffset = input.readLong();
+      docsWithFieldLength = input.readLong();
+      jumpTableEntryCount = input.readShort();
+      denseRankPower = input.readByte();
+
+      // sparse
+      if (docsWithFieldOffset != -1 && docsWithFieldOffset != -2) {

Review Comment:
   I think it'd be clearer to just say `docsWithFieldOffset == 0` here. Then you could assert in the `else` that there are no other options.



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

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

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


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