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

[GitHub] [lucene] iverase opened a new pull request, #12460: Allow reading binary doc values as a DataInput

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

   see https://github.com/apache/lucene/issues/12459


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

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

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


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


[GitHub] [lucene] iverase commented on pull request #12460: Allow reading binary doc values as a DataInput

Posted by "iverase (via GitHub)" <gi...@apache.org>.
iverase commented on PR #12460:
URL: https://github.com/apache/lucene/pull/12460#issuecomment-1715224914

   > I'm contemplating not introducing a new DataInputDocValues class, and instead have a dataInput() method on BinaryDocValues
   
   I think this approach defeats on of the main purposes for this change, that is to avoid allocating a byte array when reading doc values. I don't think we want BinaryDocValues to do that lazily:
   
   ```
   when one of the doc values is big, in the order of few megabytes, it can cause issues with small heaps (or even big heaps if 
   big enough). This is due to the allocation of a big byte array upfront, that can be consider humongous allocations by the G1 
   garbage collector and it can cause heap issues under high load.
   ```
   
   On my own use case, getting a DataInput is not enough as I need random access via get/set position, in a similar fashion to what I am doing now via ByteArrayDataInput. 
   
   
   
   
   
   
   
   
   


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

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

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


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


[GitHub] [lucene] iverase commented on pull request #12460: Allow reading binary doc values as a DataInput

Posted by "iverase (via GitHub)" <gi...@apache.org>.
iverase commented on PR #12460:
URL: https://github.com/apache/lucene/pull/12460#issuecomment-1685783623

   I am currently not planing to replace any of the usages as I am not familiar with them. Note that some of them encode data in big endian while DataOutput/DataInput uses little endian since 8.0 so there might not be compatible. The `SerializedDVStrategy' uses a `java.io.ByteArrayInputStream` so it is not a good candidate either.
   
   
   My use case is more similar to [ShapeDocValues](https://github.com/apache/lucene/blob/fad3108b27b7c9b9514a5b96e26295da3f7c8723/lucene/core/src/java/org/apache/lucene/document/ShapeDocValues.java#L578) and that would be a good candidate. I am not familiar with the implementation and it seems to requires some signature changes so left the implementation to whoever is interested.


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

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

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


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


[GitHub] [lucene] jpountz commented on pull request #12460: Allow reading binary doc values as a DataInput

Posted by "jpountz (via GitHub)" <gi...@apache.org>.
jpountz commented on PR #12460:
URL: https://github.com/apache/lucene/pull/12460#issuecomment-1715238722

   > I think this approach defeats on of the main purposes for this change, that is to avoid allocating a byte array when reading doc values. I don't think we want BinaryDocValues to do that lazily
   
   What is the problem with allocating lazily? It wouldn't make sense to me with the current API, where binaryValue() is the only way to retrieve the data, but if it were to only remain for bw compat it would make sense to me to only incur the byte[] overhead if the legacy `binaryValue()` API is used?
   
   > On my own use case, getting a DataInput is not enough as I need random access via get/set position, in a similar fashion to what I am doing now via ByteArrayDataInput.
   
   This has been a challenge so many times in the past, maybe it's time to add `seek()` support to `DataInput`?


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


Re: [PR] Allow reading binary doc values as a DataInput [lucene]

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #12460:
URL: https://github.com/apache/lucene/pull/12460#issuecomment-1880902629

   This PR has not had activity in the past 2 weeks, labeling it as stale. If the PR is waiting for review, notify the dev@lucene.apache.org list. Thank you for your contribution!


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

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

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


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


[GitHub] [lucene] uschindler commented on pull request #12460: Allow reading binary doc values as a DataInput

Posted by "uschindler (via GitHub)" <gi...@apache.org>.
uschindler commented on PR #12460:
URL: https://github.com/apache/lucene/pull/12460#issuecomment-1715550666

   To save more memory copies, the codec may use a slice from the underlying IndexInput directly to support both access apis. All file pointer checks would then be performed by the low level JVM intrinsics used my MemorySegmentIndexInput's slices. If you use those views and not let them escape the current method, the GC pressure isn't there (be careful: Profilers show them as escape analysis no longer works when the profiler prevents inlining).
   
   At some point we must anyways go away from the reusing of instances and move Lucene to the shortliving immutable instances used by modern JDK APIs. Escape analysis works fine, if you have simple apis.


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

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

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


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


[GitHub] [lucene] iverase commented on a diff in pull request #12460: Allow reading binary doc values as a DataInput

Posted by "iverase (via GitHub)" <gi...@apache.org>.
iverase commented on code in PR #12460:
URL: https://github.com/apache/lucene/pull/12460#discussion_r1275913689


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesProducer.java:
##########
@@ -820,6 +822,201 @@ public BytesRef binaryValue() throws IOException {
     }
   }
 
+  private abstract static class DenseDataInputDocValues extends DataInputDocValues {
+
+    final int maxDoc;
+    int doc = -1;
+
+    DenseDataInputDocValues(int maxDoc) {
+      this.maxDoc = maxDoc;
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      return advance(doc + 1);
+    }
+
+    @Override
+    public int docID() {
+      return doc;
+    }
+
+    @Override
+    public long cost() {
+      return maxDoc;
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      if (target >= maxDoc) {
+        return doc = NO_MORE_DOCS;
+      }
+      return doc = target;
+    }
+
+    @Override
+    public boolean advanceExact(int target) throws IOException {
+      doc = target;
+      return true;
+    }
+  }
+
+  private abstract static class SparseDataInputDocValues extends DataInputDocValues {
+
+    final IndexedDISI disi;
+
+    SparseDataInputDocValues(IndexedDISI disi) {
+      this.disi = disi;
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      return disi.nextDoc();
+    }
+
+    @Override
+    public int docID() {
+      return disi.docID();
+    }
+
+    @Override
+    public long cost() {
+      return disi.cost();
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      return disi.advance(target);
+    }
+
+    @Override
+    public boolean advanceExact(int target) throws IOException {
+      return disi.advanceExact(target);
+    }
+  }
+
+  private static class SlicedDataInput extends DataInput {
+
+    private final IndexInput in;
+    private int length;
+
+    private int read;

Review Comment:
   I changed to use `in#getFilePointer` but I wonder if we should remove all this low level checks similarly to `ByteArrayDataInput` so it is not slow down.



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

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

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


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


[GitHub] [lucene] iverase commented on pull request #12460: Allow reading binary doc values as a DataInput

Posted by "iverase (via GitHub)" <gi...@apache.org>.
iverase commented on PR #12460:
URL: https://github.com/apache/lucene/pull/12460#issuecomment-1720945832

   Thanks @jpountz and @uschindler for the input. I had a look into `RandomAccessInput` and I don't think this what we need. We need an DataInput that is positional ware so it supports seek and in addition it knows its length so we can still read the full stream via `readBytes(byte[] b, int offset, int len)` to support the current functionality.
   
   What I am missing is an abstraction between `DataInput` and `IndexInput` like:
   
   ```
   /**
    * A positional aware {@link DataInput}.
    */
   public abstract class SliceDataInput extends DataInput {
   
       /** The number of bytes in the stream. */
       public abstract long length();
   
       /**
        * Returns the current position in this stream, where the next read will occur.
        *
        * @see #seek(long)
        */
       public abstract long position();
   
       /**
        * Sets current position in this stream, where the next read will occur.
        *
        * @see #position()
        */
       public abstract void seek(long pos) throws IOException;
   
       /**
        * {@inheritDoc}
        *
        * <p>Behavior is functionally equivalent to seeking to <code>position() + numBytes</code>.
        *
        * @see #position()
        * @see #seek(long)
        */
       @Override
       public void skipBytes(long numBytes) throws IOException {
           if (numBytes < 0) {
               throw new IllegalArgumentException("numBytes must be >= 0, got " + numBytes);
           }
           final long skipTo = position() + numBytes;
           seek(skipTo);
       }
   }
   ```
   (Naming is so hard, naming proposals are welcome) 
   
   This abstraction can be extended by `ByteBuffersDataInput` and `IndexInput` and potentially by `ByteArrayDataInput` and `FTS.BytesReader`.
   
    
   


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

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

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


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


[GitHub] [lucene] stefanvodita commented on a diff in pull request #12460: Allow reading binary doc values as a DataInput

Posted by "stefanvodita (via GitHub)" <gi...@apache.org>.
stefanvodita commented on code in PR #12460:
URL: https://github.com/apache/lucene/pull/12460#discussion_r1299159293


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesProducer.java:
##########
@@ -820,6 +822,213 @@ public BytesRef binaryValue() throws IOException {
     }
   }
 
+  private abstract static class DenseDataInputDocValues extends DataInputDocValues {
+
+    final int maxDoc;
+    int doc = -1;
+
+    DenseDataInputDocValues(int maxDoc) {
+      this.maxDoc = maxDoc;
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      return advance(doc + 1);
+    }
+
+    @Override
+    public int docID() {
+      return doc;
+    }
+
+    @Override
+    public long cost() {
+      return maxDoc;
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      if (target >= maxDoc) {
+        return doc = NO_MORE_DOCS;
+      }
+      return doc = target;
+    }
+
+    @Override
+    public boolean advanceExact(int target) throws IOException {
+      doc = target;
+      return true;
+    }
+  }
+
+  private abstract static class SparseDataInputDocValues extends DataInputDocValues {
+
+    final IndexedDISI disi;
+
+    SparseDataInputDocValues(IndexedDISI disi) {
+      this.disi = disi;
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      return disi.nextDoc();
+    }
+
+    @Override
+    public int docID() {
+      return disi.docID();
+    }
+
+    @Override
+    public long cost() {
+      return disi.cost();
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      return disi.advance(target);
+    }
+
+    @Override
+    public boolean advanceExact(int target) throws IOException {
+      return disi.advanceExact(target);
+    }
+  }
+
+  private static class SlicedDataInput extends DataInputDocValues.DataInputDocValue {
+
+    private final IndexInput in;
+    private int length;
+    private long offset;
+
+    SlicedDataInput(IndexInput in) {
+      this.in = in;
+    }
+
+    void init(long offset, int length) throws IOException {
+      in.seek(offset);
+      this.length = length;
+      this.offset = offset;
+    }
+
+    private void checkBounds(int numBytes) throws EOFException {
+      if (Math.toIntExact(getPosition() + numBytes) - 1 >= length) {

Review Comment:
   Can we simplify this condition?
   `Math.toIntExact(getPosition() + numBytes) > length`



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

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

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


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


[GitHub] [lucene] uschindler commented on pull request #12460: Allow reading binary doc values as a DataInput

Posted by "uschindler (via GitHub)" <gi...@apache.org>.
uschindler commented on PR #12460:
URL: https://github.com/apache/lucene/pull/12460#issuecomment-1715514900

   > This has been a challenge so many times in the past, maybe it's time to add `seek()` support to `DataInput`?
   
   We have full random access (positional reads), if you extend the interface `RandomAccessInput`.
   
   Adding seek support to `DataInput` is not a good idea because it would prevent those `DataInput`s based on `InputStream` (e g. `InputStreamDataInput`, which is used at various places).


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

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

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


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


[GitHub] [lucene] iverase commented on pull request #12460: Allow reading binary doc values as a DataInput

Posted by "iverase (via GitHub)" <gi...@apache.org>.
iverase commented on PR #12460:
URL: https://github.com/apache/lucene/pull/12460#issuecomment-1735063044

   I have been thinking a bit longer about this and I think this approach of `DataInput` is not right. Instead we should try to return an API more similar to `RandomAccessInput` as Uwe suggested. 
   
   What convinces me is the fact that `BytesRef` can easily implement (and maybe should) RandomAccessInput. Unfortunately it is not straightforward to change BytesRef with RandomAccessInput as we are doing things with BytesRef that are challenging to implement generically:
   
   - Maybe we should add the length of the `RandomAccessInput` to the API? ` public long length()`
   - We can copy the contents of a `RandomAccessInput` byte by byte, should we add a bulk copy method like  ` public byte readBytes(long pos, byte[] b,  int offset, int length) throws IOException;`?
   - Can RandomAccessInput implement Comparable?
   - Can RandomAccessInput implementation have implementations of equals and hashCode?
   
   The first two are easy to implement and maybe we should do it regardless this issue. The other two are much trickier.
   
   


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

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

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


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


[GitHub] [lucene] iverase commented on a diff in pull request #12460: Allow reading binary doc values as a DataInput

Posted by "iverase (via GitHub)" <gi...@apache.org>.
iverase commented on code in PR #12460:
URL: https://github.com/apache/lucene/pull/12460#discussion_r1299684458


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesProducer.java:
##########
@@ -820,6 +822,213 @@ public BytesRef binaryValue() throws IOException {
     }
   }
 
+  private abstract static class DenseDataInputDocValues extends DataInputDocValues {
+
+    final int maxDoc;
+    int doc = -1;
+
+    DenseDataInputDocValues(int maxDoc) {
+      this.maxDoc = maxDoc;
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      return advance(doc + 1);
+    }
+
+    @Override
+    public int docID() {
+      return doc;
+    }
+
+    @Override
+    public long cost() {
+      return maxDoc;
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      if (target >= maxDoc) {
+        return doc = NO_MORE_DOCS;
+      }
+      return doc = target;
+    }
+
+    @Override
+    public boolean advanceExact(int target) throws IOException {
+      doc = target;
+      return true;
+    }
+  }
+
+  private abstract static class SparseDataInputDocValues extends DataInputDocValues {
+
+    final IndexedDISI disi;
+
+    SparseDataInputDocValues(IndexedDISI disi) {
+      this.disi = disi;
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      return disi.nextDoc();
+    }
+
+    @Override
+    public int docID() {
+      return disi.docID();
+    }
+
+    @Override
+    public long cost() {
+      return disi.cost();
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      return disi.advance(target);
+    }
+
+    @Override
+    public boolean advanceExact(int target) throws IOException {
+      return disi.advanceExact(target);
+    }
+  }
+
+  private static class SlicedDataInput extends DataInputDocValues.DataInputDocValue {
+
+    private final IndexInput in;
+    private int length;
+    private long offset;
+
+    SlicedDataInput(IndexInput in) {
+      this.in = in;
+    }
+
+    void init(long offset, int length) throws IOException {
+      in.seek(offset);
+      this.length = length;
+      this.offset = offset;
+    }
+
+    private void checkBounds(int numBytes) throws EOFException {
+      if (Math.toIntExact(getPosition() + numBytes) - 1 >= length) {

Review Comment:
   sure thanks!



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

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

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


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


[GitHub] [lucene] jpountz commented on pull request #12460: Allow reading binary doc values as a DataInput

Posted by "jpountz (via GitHub)" <gi...@apache.org>.
jpountz commented on PR #12460:
URL: https://github.com/apache/lucene/pull/12460#issuecomment-1715126194

   The more I think of this change, the more I like it: most of the time, you would need to read data out of binary doc values, e.g. (variable-length) integers, strings, etc. and exposing binary doc values as a data input not only makes this easier to do, but also more efficient by saving one memory copy.
   
   Thoughts:
    - We are losing the symmetry between the index-time API (BinaryDocValuesField), which takes a byte[] while the search-time API produces a `DataInput`. Not a deal breaker, but users have to be careful about endianness.
    - Today it's possible to consume values multiple times, but the data input is stateful, so if one consumer reads data from it, and then another consumer tries to read data again it will fail unless it resets the input via a call to `advanceExact`. Again not a deal breaker but something that could be a surprise to some callers.
   
   In terms of backward compatibility, I'm contemplating not introducing a new `DataInputDocValues` class, and instead have a `dataInput()` method on `BinaryDocValues`, as well as a default implementation of `binaryValue()` that reads all bytes from the data input. What do you think? I'm also curious of the opinion of our sophisticated backward compatibility policeman @uschindler.
   
   To @stefanvodita 's point, it would be nice to migrate at least one or two call sites that would get simpler with a `DataInput` to further validate this change.


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