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/06/27 13:12:37 UTC

[GitHub] [lucene] luyuncheng opened a new pull request, #987: Using CompositeByteBuf to Reduce Memory Copy

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

   JIRA: https://issues.apache.org/jira/browse/LUCENE-10627
   I see When Lucene Do flush and merge store fields, need many memory copies:
   ```
   Lucene Merge Thread #25940]" #906546 daemon prio=5 os_prio=0 cpu=20503.95ms elapsed=68.76s tid=0x00007ee990002c50 nid=0x3aac54 runnable  [0x00007f17718db000]
      java.lang.Thread.State: RUNNABLE
   	at org.apache.lucene.store.ByteBuffersDataOutput.toArrayCopy(ByteBuffersDataOutput.java:271)
   	at org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.flush(CompressingStoredFieldsWriter.java:239)
   	at org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.finishDocument(CompressingStoredFieldsWriter.java:169)
   	at org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.merge(CompressingStoredFieldsWriter.java:654)
   	at org.apache.lucene.index.SegmentMerger.mergeFields(SegmentMerger.java:228)
   	at org.apache.lucene.index.SegmentMerger.merge(SegmentMerger.java:105)
   	at org.apache.lucene.index.IndexWriter.mergeMiddle(IndexWriter.java:4760)
   	at org.apache.lucene.index.IndexWriter.merge(IndexWriter.java:4364)
   	at org.apache.lucene.index.IndexWriter$IndexWriterMergeSource.merge(IndexWriter.java:5923)
   	at org.apache.lucene.index.ConcurrentMergeScheduler.doMerge(ConcurrentMergeScheduler.java:624)
   	at org.elasticsearch.index.engine.ElasticsearchConcurrentMergeScheduler.doMerge(ElasticsearchConcurrentMergeScheduler.java:100)
   	at org.apache.lucene.index.ConcurrentMergeScheduler$MergeThread.run(ConcurrentMergeScheduler.java:682)
   ```
   
   When Lucene CompressingStoredFieldsWriter do flush documents, it needs many memory copies:
   
   - With Lucene90 using LZ4WithPresetDictCompressionMode:
   
   1. bufferedDocs.toArrayCopy copy blocks into one continue content for chunk compress
   2. compressor copy dict and data into one block buffer
   3. do compress
   4. copy compressed data out
   
   - With Lucene90 using DeflateWithPresetDictCompressionMode:
   
   1. bufferedDocs.toArrayCopy copy blocks into one continue content for chunk compress
   2. do compress
   3. copy compressed data out
   
   I think we can `use CompositeByteBuf` to **reduce temp memory copies** :
   - we do not have to bufferedDocs.toArrayCopy when just need continues content for chunk compress
   
   
   I write a simple mini benchamrk in test code:
   LZ4WithPresetDict run Capacity:41943040(bytes) , iter 10times: 
   `Origin elapse:5391ms , New elapse:5297ms`
   DeflateWithPresetDict run Capacity:41943040(bytes), iter 10times: 
   `Origin elapse:115ms, New elapse:12ms`
    
   And I run runStoredFieldsBenchmark with doc_limit=-1:
   shows:
   
   
   Msec to index | BEST_SPEED | BEST_COMPRESSION
   -- | -- | --
   Baseline | 318877.00 | 606288.00
   Candidate | 314442.00 | 604719.00
   
   


-- 
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] luyuncheng commented on a diff in pull request #987: LUCENE-10627: Using ByteBuffersDataInput reduce memory copy on compressing data

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


##########
lucene/core/src/java/org/apache/lucene/store/ByteBuffersDataInput.java:
##########
@@ -165,6 +165,36 @@ public void readBytes(byte[] arr, int off, int len) throws EOFException {
       }
     }
   }
+  /**
+   * ReadBytes from position with length, if [pos, pos + len] stay in one ByteBuffer can ignore
+   * memory copy, otherwise return a new ByteBuffer with continuous byte array
+   *
+   * @param length from position to length
+   * @return ByteBuffer which bytes read from [pos, pos + length]
+   */
+  public ByteBuffer readBytes(int length) throws EOFException {

Review Comment:
   LGTM, rename it with `readNBytes`



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

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

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


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


[GitHub] [lucene] jpountz merged pull request #987: LUCENE-10627: Using ByteBuffersDataInput reduce memory copy on compressing data

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


-- 
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] luyuncheng commented on a diff in pull request #987: LUCENE-10627: Using CompositeByteBuf to Reduce Memory Copy

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


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene90/compressing/Lucene90CompressingStoredFieldsWriter.java:
##########
@@ -519,7 +518,13 @@ private void copyOneDoc(Lucene90CompressingStoredFieldsReader reader, int docID)
     assert reader.getVersion() == VERSION_CURRENT;
     SerializedDocument doc = reader.document(docID);
     startDocument();
-    bufferedDocs.copyBytes(doc.in, doc.length);
+
+    if (doc.in instanceof ByteArrayDataInput) {
+      // reuse ByteArrayDataInput to reduce memory copy
+      bufferedDocs.copyBytes((ByteArrayDataInput) doc.in, doc.length);
+    } else {
+      bufferedDocs.copyBytes(doc.in, doc.length);
+    }

Review Comment:
   > I think that we could avoid this `instanceof` check by overriding `ByteBuffersDataOutput#copyBytes` to read directly into its internal buffers when they are not direct (ie. backed by a `byte[]`)? (Maybe in a separate change?)
   
   @jpountz  thanks for your advice! i think overriding `ByteBuffersDataOutput#copyBytes` is a great idea, it can reduce many method do memory copy
   
   I opened a new issue: LUCENE-10657 #1034
   i think when we move this logic into `ByteBuffersDataOutput#copyBytes` it can reduce more memory copy method:
   
   1. Reduce memory copy in Lucene90CompressingStoredFieldsWriter#copyOneDoc -> bufferdDocs.copyBytes(DataInput input)
   2. Reduce memory copy in Lucene90CompoundFormat.writeCompoundFile -> data.copyBytes when input is BufferedChecksumIndexinput and output is ByteBuffersDataOutput
   3. Reduce memory IndexWriter#copySegmentAsIs ->CopyFrom -> copyBytes



-- 
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] luyuncheng commented on pull request #987: LUCENE-10627: Using CompositeByteBuf to Reduce Memory Copy

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

   > Thanks for the iteration, the high-level idea looks good to me, I left some suggestions.
   
   @jpountz Thanks for your nice suggestions. at latest commits [ebcbdd](https://github.com/apache/lucene/commit/457bab5abfbaa3aabebe99a97e645b4f623be15f)
   1. change statement byteBuffers in Lucene90CompressingStoredFieldsWriter
   2. move sliceOne logic into ByteBuffersDataInput level and it would determine whether use internal buffers or create a new buffer
   3. delete DeflateWithPresetDictCompressionMode#doCompress method which param is byte[]
   4. delete copyOneDoc-> readBytes which instance ByteArrayDataInput optimize. and i would create a new issue to reduce memory copy in copyOneDoc


-- 
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] luyuncheng commented on pull request #987: LUCENE-10627: Using ByteBuffersDataInput reduce memory copy on compressing data

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

   > Also I wonder if the `readNBytes` trick actually helps in practice, or if we end up crossing ByteBuffer boundaries most of the time in practice 
   
   @jpountz  I have the same question that when crossing ByteBuffer would happened. So, i do a simple test:
   
   As the result, i think `have a shared byte[]` is LGTM, i try to update it.
   
   i rewrite the method with readNBytesWithoutCopy and readNBytesWithCopy as follows.
   
   then i try to use arthas `monitor` with commands `monitor -c 10 org.apache.lucene.store.ByteBuffersDataInput readNBytesWith*` AND the running task is runStoredFieldsBenchmark with BEST_COMPRESSION:
   it shows:
   
   __#readNBytesWithoutCopy : #readNBytesWithCopy = 1 : 10__ every 10 seconds
   ![image](https://user-images.githubusercontent.com/12760367/181586795-f5da6a13-b475-43a5-9ade-4c2f52162d6c.png)
   
   codes:
   ```
     public ByteBuffer readNBytes(int length) throws EOFException {
       .....
       if (block.remaining() >= length) {
         this.pos += length;
         return readNBytesWithoutCopy(block, blockOffset, length);
       } else {
         return readNBytesWithCopy(length);
       }
     }
   
     private ByteBuffer readNBytesWithoutCopy(ByteBuffer block, int blockOffset, int length) {
       return block.slice(blockOffset, length);
     }
   
     private ByteBuffer readNBytesWithCopy(int length) throws EOFException {
       ByteBuffer copyBuffer = ByteBuffer.allocate(length);
       readBytes(copyBuffer, length);
       return copyBuffer.rewind().order(ByteOrder.LITTLE_ENDIAN);
     }
   ```


-- 
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] luyuncheng commented on a diff in pull request #987: LUCENE-10627: Using CompositeByteBuf to Reduce Memory Copy

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


##########
lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressionMode.java:
##########
@@ -257,9 +270,13 @@ private static class DeflateCompressor extends Compressor {
     }
 
     @Override
-    public void compress(byte[] bytes, int off, int len, DataOutput out) throws IOException {
+    public void compress(ByteBuffersDataInput buffersInput, int off, int len, DataOutput out)

Review Comment:
   > Should we remove `off` and `len` and rely on callers to create a `ByteBuffersDataInput#slice` if they only need to compress a subset of the input?
   
   at commits [448e254](https://github.com/luyuncheng/lucene/blob/448e254e1d3c5323f369236492de0d512f537ac2/lucene/core/src/java/org/apache/lucene/codecs/compressing/Compressor.java#L35) i only use `  public abstract void compress(ByteBuffersDataInput buffersInput, DataOutput out)`



-- 
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] luyuncheng commented on a diff in pull request #987: LUCENE-10627: Using CompositeByteBuf to Reduce Memory Copy

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


##########
lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressionMode.java:
##########
@@ -257,9 +270,13 @@ private static class DeflateCompressor extends Compressor {
     }
 
     @Override
-    public void compress(byte[] bytes, int off, int len, DataOutput out) throws IOException {
+    public void compress(ByteBuffersDataInput buffersInput, int off, int len, DataOutput out)

Review Comment:
   > Should we remove `off` and `len` and rely on callers to create a `ByteBuffersDataInput#slice` if they only need to compress a subset of the input?
   
   at latest [commits](https://github.com/luyuncheng/lucene/blob/448e254e1d3c5323f369236492de0d512f537ac2/lucene/core/src/java/org/apache/lucene/codecs/compressing/Compressor.java#L35) i only use `  public abstract void compress(ByteBuffersDataInput buffersInput, DataOutput out)`



-- 
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] luyuncheng commented on a diff in pull request #987: LUCENE-10627: Using ByteBuffersDataInput reduce memory copy on compressing data

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


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene90/DeflateWithPresetDictCompressionMode.java:
##########
@@ -198,22 +200,25 @@ private void doCompress(byte[] bytes, int off, int len, DataOutput out) throws I
     }
 
     @Override
-    public void compress(byte[] bytes, int off, int len, DataOutput out) throws IOException {
+    public void compress(ByteBuffersDataInput buffersInput, DataOutput out) throws IOException {
+      final int len = (int) (buffersInput.size() - buffersInput.position());

Review Comment:
   i think it is right that `position()` always be 0, if caller do compress method with param buffersInput have been sliced. but compress method is a public interface, i think we need check the position for the 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] luyuncheng commented on a diff in pull request #987: LUCENE-10627: Using CompositeByteBuf to Reduce Memory Copy

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


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene90/DeflateWithPresetDictCompressionMode.java:
##########
@@ -163,12 +165,16 @@ private static class DeflateWithPresetDictCompressor extends Compressor {
     final Deflater compressor;
     final BugfixDeflater_JDK8252739 deflaterBugfix;
     byte[] compressed;
+    byte[] bufferDict;
+    byte[] bufferBlock;
     boolean closed;
 
     DeflateWithPresetDictCompressor(int level) {
       compressor = new Deflater(level, true);
       deflaterBugfix = BugfixDeflater_JDK8252739.createBugfix(compressor);
       compressed = new byte[64];
+      bufferDict = BytesRef.EMPTY_BYTES;
+      bufferBlock = BytesRef.EMPTY_BYTES;
     }
 
     private void doCompress(byte[] bytes, int off, int len, DataOutput out) throws IOException {

Review Comment:
   ++ ,at [commits(ebcbd)](https://github.com/apache/lucene/blob/ebcbdd05fbf114ca15d04cd4a9172eb51fab9a1a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/DeflateWithPresetDictCompressionMode.java) i removed it



-- 
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 #987: LUCENE-10627: Using CompositeByteBuf to Reduce Memory Copy

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

   > if we only using compress method with variants ByteBuffersDataInput in LUCENE90, we can not using abstract method Compressor.compress, when we want to use other compression mode.
   
   I think that this downside is fine? We prefer codecs to evolve independently so when we start needing changes for a new codec, we prefer to fork the code so that old codecs still rely on the unchanged code (which should move to lucene/backward-codecs) while the new codecs only use the new code without carrying over legacy code.


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

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

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


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


[GitHub] [lucene] jpountz commented on a diff in pull request #987: LUCENE-10627: Using CompositeByteBuf to Reduce Memory Copy

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


##########
lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressionMode.java:
##########
@@ -257,9 +270,13 @@ private static class DeflateCompressor extends Compressor {
     }
 
     @Override
-    public void compress(byte[] bytes, int off, int len, DataOutput out) throws IOException {
+    public void compress(ByteBuffersDataInput buffersInput, int off, int len, DataOutput out)

Review Comment:
   Should we remove `off` and `len` and rely on callers to create a `ByteBuffersDataInput#slice` if they only need to compress a subset of the input?



-- 
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] luyuncheng commented on a diff in pull request #987: LUCENE-10627: Using ByteBuffersDataInput reduce memory copy on compressing data

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


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene90/DeflateWithPresetDictCompressionMode.java:
##########
@@ -198,22 +200,25 @@ private void doCompress(byte[] bytes, int off, int len, DataOutput out) throws I
     }
 
     @Override
-    public void compress(byte[] bytes, int off, int len, DataOutput out) throws IOException {
+    public void compress(ByteBuffersDataInput buffersInput, DataOutput out) throws IOException {
+      final int len = (int) (buffersInput.size() - buffersInput.position());

Review Comment:
   It is right that `position()` always be 0, if caller do compress method with param buffersInput have been sliced. As compress method is a public interface, i think we need check the position for the 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] jpountz commented on a diff in pull request #987: LUCENE-10627: Using ByteBuffersDataInput reduce memory copy on compressing data

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


##########
lucene/core/src/java/org/apache/lucene/store/ByteBuffersDataInput.java:
##########
@@ -165,6 +165,36 @@ public void readBytes(byte[] arr, int off, int len) throws EOFException {
       }
     }
   }
+  /**
+   * ReadBytes from position with length, if [pos, pos + len] stay in one ByteBuffer can ignore
+   * memory copy, otherwise return a new ByteBuffer with continuous byte array
+   *
+   * @param length from position to length
+   * @return ByteBuffer which bytes read from [pos, pos + length]
+   */
+  public ByteBuffer readBytes(int length) throws EOFException {

Review Comment:
   I wonder if it should be called `readNBytes` for consistency with `InputStream#readNBytes`. Maybe @uschindler has thoughts on this and on the implementation.



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene90/DeflateWithPresetDictCompressionMode.java:
##########
@@ -163,20 +165,22 @@ private static class DeflateWithPresetDictCompressor extends Compressor {
     final Deflater compressor;
     final BugfixDeflater_JDK8252739 deflaterBugfix;
     byte[] compressed;
+    byte[] bufferDict;
     boolean closed;
 
     DeflateWithPresetDictCompressor(int level) {
       compressor = new Deflater(level, true);
       deflaterBugfix = BugfixDeflater_JDK8252739.createBugfix(compressor);
       compressed = new byte[64];
+      bufferDict = BytesRef.EMPTY_BYTES;
     }
 
-    private void doCompress(byte[] bytes, int off, int len, DataOutput out) throws IOException {
+    private void doCompress(ByteBuffer bytes, int len, DataOutput out) throws IOException {

Review Comment:
   Can we remove the `len` parameter?



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene90/DeflateWithPresetDictCompressionMode.java:
##########
@@ -198,22 +200,25 @@ private void doCompress(byte[] bytes, int off, int len, DataOutput out) throws I
     }
 
     @Override
-    public void compress(byte[] bytes, int off, int len, DataOutput out) throws IOException {
+    public void compress(ByteBuffersDataInput buffersInput, DataOutput out) throws IOException {
+      final int len = (int) (buffersInput.size() - buffersInput.position());

Review Comment:
   Should position() always be 0?



##########
lucene/core/src/test/org/apache/lucene/codecs/lucene90/compressing/TestCompressingStoredFieldsFormat.java:
##########
@@ -325,4 +341,160 @@ public void testChunkCleanup() throws IOException {
     iw.close();
     dir.close();
   }
+
+  public void testSortedSetVariableLengthBigStoredFields() throws Exception {

Review Comment:
   did you add this because the doc values test case found bugs in stored fields?



-- 
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] luyuncheng commented on pull request #987: LUCENE-10627: Using CompositeByteBuf to Reduce Memory Copy

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

   > Would it be possible to remove all `CompressionMode#compress` variants that take a `byte[]` now that you introduced a new method that takes a `ByteBuffersDataInput`?
   > 
   > Also maybe we should keep old codecs unmodified and only make this change to `Lucene90Codec` where it makes most sense?
   
   Hi @jpountz Thanks for reviewing this code. 
   
   I prefer keeping old codecs unmodified, because `CompressionMode#compress` is a public abstract method, if we change it with variants `ByteBuffersDataInput` we need to backport in many codecs, like [commits](https://github.com/apache/lucene/blob/382962f22df3ee3af3fb538b877c98d61a622ddb/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene50/Lucene50StoredFieldsFormat.java).
   
   And if we only using compress method with variants ByteBuffersDataInput in LUCENE90, we can not using abstract method `Compressor.compress`, when we want to use other compression mode.
   
   Would it be possible to add a new method in Compressor, like following? it can keep the old codecs unmodified, and method with variants ByteBuffersDataInput only can be called in Lucene90Codec.
   
   ```
   public abstract void compress(byte[] bytes, int off, int len, DataOutput out) throws IOException;
   
   public void compress(CompositeByteBuf compositeByteBuf, DataOutput out) throws IOException;
   ```


-- 
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] luyuncheng commented on a diff in pull request #987: LUCENE-10627: Using ByteBuffersDataInput reduce memory copy on compressing data

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


##########
lucene/core/src/java/org/apache/lucene/store/ByteBuffersDataInput.java:
##########
@@ -165,6 +165,36 @@ public void readBytes(byte[] arr, int off, int len) throws EOFException {
       }
     }
   }
+  /**

Review Comment:
   Wondering +1, it seems like spotless can not format this situation. and i manually fixed it



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

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

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


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


[GitHub] [lucene] jpountz commented on a diff in pull request #987: LUCENE-10627: Using CompositeByteBuf to Reduce Memory Copy

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


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene90/compressing/Lucene90CompressingStoredFieldsWriter.java:
##########
@@ -247,21 +249,18 @@ private void flush(boolean force) throws IOException {
     writeHeader(docBase, numBufferedDocs, numStoredFields, lengths, sliced, dirtyChunk);
 
     // compress stored fields to fieldsStream.
-    //
-    // TODO: do we need to slice it since we already have the slices in the buffer? Perhaps
-    // we should use max-block-bits restriction on the buffer itself, then we won't have to check it
-    // here.
-    byte[] content = bufferedDocs.toArrayCopy();
-    bufferedDocs.reset();
-
     if (sliced) {
-      // big chunk, slice it
-      for (int compressed = 0; compressed < content.length; compressed += chunkSize) {
-        compressor.compress(
-            content, compressed, Math.min(chunkSize, content.length - compressed), fieldsStream);
+      // big chunk, slice it, using ByteBuffersDataInput ignore memory copy
+      ByteBuffersDataInput bytebuffers = bufferedDocs.toDataInput();
+      final int capacity = (int) bytebuffers.size();
+      for (int compressed = 0; compressed < capacity; compressed += chunkSize) {
+        int l = Math.min(chunkSize, capacity - compressed);
+        ByteBuffersDataInput bbdi = bytebuffers.slice(compressed, l);
+        compressor.compress(bbdi, fieldsStream);
       }
     } else {
-      compressor.compress(content, 0, content.length, fieldsStream);
+      ByteBuffersDataInput bytebuffers = bufferedDocs.toDataInput();

Review Comment:
   Maybe move this before the `if` statement since we create `byteBuffers` the same way on both branches?



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene90/compressing/Lucene90CompressingStoredFieldsWriter.java:
##########
@@ -519,7 +518,13 @@ private void copyOneDoc(Lucene90CompressingStoredFieldsReader reader, int docID)
     assert reader.getVersion() == VERSION_CURRENT;
     SerializedDocument doc = reader.document(docID);
     startDocument();
-    bufferedDocs.copyBytes(doc.in, doc.length);
+
+    if (doc.in instanceof ByteArrayDataInput) {
+      // reuse ByteArrayDataInput to reduce memory copy
+      bufferedDocs.copyBytes((ByteArrayDataInput) doc.in, doc.length);
+    } else {
+      bufferedDocs.copyBytes(doc.in, doc.length);
+    }

Review Comment:
   I think that we could avoid this `instanceof` check by overriding `ByteBuffersDataOutput#copyBytes` to read directly into its internal buffers when they are not direct (ie. backed by a `byte[]`)? (Maybe in a separate change?)



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene90/DeflateWithPresetDictCompressionMode.java:
##########
@@ -163,12 +165,16 @@ private static class DeflateWithPresetDictCompressor extends Compressor {
     final Deflater compressor;
     final BugfixDeflater_JDK8252739 deflaterBugfix;
     byte[] compressed;
+    byte[] bufferDict;
+    byte[] bufferBlock;
     boolean closed;
 
     DeflateWithPresetDictCompressor(int level) {
       compressor = new Deflater(level, true);
       deflaterBugfix = BugfixDeflater_JDK8252739.createBugfix(compressor);
       compressed = new byte[64];
+      bufferDict = BytesRef.EMPTY_BYTES;
+      bufferBlock = BytesRef.EMPTY_BYTES;
     }
 
     private void doCompress(byte[] bytes, int off, int len, DataOutput out) throws IOException {

Review Comment:
   Can we remove this one and require callers to use the ByteBuffer variant instead?



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene90/DeflateWithPresetDictCompressionMode.java:
##########
@@ -199,23 +205,65 @@ private void doCompress(byte[] bytes, int off, int len, DataOutput out) throws I
       out.writeBytes(compressed, totalCount);
     }
 
+    private void doCompress(ByteBuffer bytes, int len, DataOutput out) throws IOException {
+      if (len == 0) {
+        out.writeVInt(0);
+        return;
+      }
+      compressor.setInput(bytes);
+      compressor.finish();
+      if (compressor.needsInput()) {
+        throw new IllegalStateException();
+      }
+
+      int totalCount = 0;
+      for (; ; ) {
+        final int count =
+            compressor.deflate(compressed, totalCount, compressed.length - totalCount);
+        totalCount += count;
+        assert totalCount <= compressed.length;
+        if (compressor.finished()) {
+          break;
+        } else {
+          compressed = ArrayUtil.grow(compressed);
+        }
+      }
+
+      out.writeVInt(totalCount);
+      out.writeBytes(compressed, totalCount);
+    }
+
     @Override
-    public void compress(byte[] bytes, int off, int len, DataOutput out) throws IOException {
+    public void compress(ByteBuffersDataInput buffersInput, DataOutput out) throws IOException {
+      final int len = (int) (buffersInput.size() - buffersInput.position());
+      final int end = (int) buffersInput.size();
       final int dictLength = len / (NUM_SUB_BLOCKS * DICT_SIZE_FACTOR);
       final int blockLength = (len - dictLength + NUM_SUB_BLOCKS - 1) / NUM_SUB_BLOCKS;
       out.writeVInt(dictLength);
       out.writeVInt(blockLength);
-      final int end = off + len;
 
       // Compress the dictionary first
       compressor.reset();
-      doCompress(bytes, off, dictLength, out);
+      bufferDict = ArrayUtil.growNoCopy(bufferDict, dictLength);
+      buffersInput.readBytes(bufferDict, 0, dictLength);
+      doCompress(bufferDict, 0, dictLength, out);
 
       // And then sub blocks
-      for (int start = off + dictLength; start < end; start += blockLength) {
+      for (int start = dictLength; start < end; start += blockLength) {
         compressor.reset();
-        deflaterBugfix.setDictionary(bytes, off, dictLength);
-        doCompress(bytes, start, Math.min(blockLength, off + len - start), out);
+        deflaterBugfix.setDictionary(bufferDict, 0, dictLength);
+        int l = Math.min(blockLength, len - start);
+        // if [start,start + len] stay in one ByteBuffer, we can ignore memory copy
+        // otherwise need to copy bytes into on continuous byte array
+        ByteBuffer bb = buffersInput.sliceOne(start, l);

Review Comment:
   I would prefer abstracting this at the ByteBuffersDataInput level, e.g. maybe there could be a `ByteBuffer ByteBuffersDataInput#readBytes(int numBytes)` API that would automatically return a view of one of its internal buffers when the bytes are fully contained into a single buffer, and create a fresh new ByteBuffer otherwise?



-- 
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] luyuncheng commented on pull request #987: LUCENE-10627: Using CompositeByteBuf to Reduce Memory Copy

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

   >  we prefer to fork the code so that old codecs still rely on the unchanged code (which should move to lucene/backward-codecs) 
   
   Thanks for the your advice @jpountz , i think it is LGTM. 
   At commit [448e25](https://github.com/luyuncheng/lucene/commit/448e254e1d3c5323f369236492de0d512f537ac2) i try to move old compressor into backward_codecs. 
   And we only use one method `compress(ByteBuffersDataInput buffersInput, DataOutput out)` in Compressor
   
   When using ByteBuffersDataInput in compress mehtod, it can 
   1. Reuse ByteBuffersDataInput reduce memory copy in stored fields compressing
   2. Reuse ByteBuffersDataInput reduce memory copy in TermVectors compressing
   3. Reuse ByteArrayDataInput reduce memory copy in copyOneDoc
   


-- 
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] luyuncheng commented on a diff in pull request #987: LUCENE-10627: Using ByteBuffersDataInput reduce memory copy on compressing data

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


##########
lucene/CHANGES.txt:
##########
@@ -54,7 +54,8 @@ Improvements
 
 Optimizations
 ---------------------
-(No changes)
+
+* LUCENE-10627: Using ByteBuffersDataInput reduce memory copy on compressing data. (luyuncheng)

Review Comment:
   > Move it to the 9.4 section?
   
   DONE



-- 
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] luyuncheng commented on a diff in pull request #987: LUCENE-10627: Using CompositeByteBuf to Reduce Memory Copy

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


##########
lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressionMode.java:
##########
@@ -257,9 +270,13 @@ private static class DeflateCompressor extends Compressor {
     }
 
     @Override
-    public void compress(byte[] bytes, int off, int len, DataOutput out) throws IOException {
+    public void compress(ByteBuffersDataInput buffersInput, int off, int len, DataOutput out)

Review Comment:
   it is a nice suggestion, i try to use method `compress(CompositeByteBuf compositeByteBuf, DataOutput out) ` 



-- 
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] luyuncheng commented on pull request #987: LUCENE-10627: Using ByteBuffersDataInput reduce memory copy on compressing data

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

   > Thanks for running these tests, let's remove `readNBytes` and switch to a shared byte[] that we call `DataInput#readBytes` on instead.
   
   @jpountz Thanks a lot for you reviewing for the code. at https://github.com/apache/lucene/pull/987/commits/c0d31d3134653bf6009b798925fa2350e5f7ec9c i removed readNBytes and using shared byte[] buffer in `DeflateWithPresetDictCompressionMode`. 
   
   I try to do more test about why `readNBytes` with a lot of memory copy, i think this is relevant to the:
   1. `DeflateWithPresetCompressingCodec` chunk size: `1 << 18`
   2. `LZ4WithPresetDictCompressionMode` NUM_SUB_BLOCKS: `10` 
   3. `ByteBuffersDataOutput` BITS_PER_BLOCK
   
   The `chunk size` is larger than `block size` which made `ByteBuffersDataInput` fragment always copy datas


-- 
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 #987: LUCENE-10627: Using ByteBuffersDataInput reduce memory copy on compressing data

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

   Thanks for running these tests, let's remove `readNBytes` and switch to a shared byte[] that we call `DataInput#readBytes` on instead.


-- 
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] luyuncheng commented on a diff in pull request #987: LUCENE-10627: Using CompositeByteBuf to Reduce Memory Copy

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


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene90/DeflateWithPresetDictCompressionMode.java:
##########
@@ -163,12 +165,16 @@ private static class DeflateWithPresetDictCompressor extends Compressor {
     final Deflater compressor;
     final BugfixDeflater_JDK8252739 deflaterBugfix;
     byte[] compressed;
+    byte[] bufferDict;
+    byte[] bufferBlock;
     boolean closed;
 
     DeflateWithPresetDictCompressor(int level) {
       compressor = new Deflater(level, true);
       deflaterBugfix = BugfixDeflater_JDK8252739.createBugfix(compressor);
       compressed = new byte[64];
+      bufferDict = BytesRef.EMPTY_BYTES;
+      bufferBlock = BytesRef.EMPTY_BYTES;
     }
 
     private void doCompress(byte[] bytes, int off, int len, DataOutput out) throws IOException {

Review Comment:
   ++ ++,at [commits(ebcbd)](https://github.com/apache/lucene/blob/ebcbdd05fbf114ca15d04cd4a9172eb51fab9a1a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/DeflateWithPresetDictCompressionMode.java) i removed it



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene90/DeflateWithPresetDictCompressionMode.java:
##########
@@ -199,23 +205,65 @@ private void doCompress(byte[] bytes, int off, int len, DataOutput out) throws I
       out.writeBytes(compressed, totalCount);
     }
 
+    private void doCompress(ByteBuffer bytes, int len, DataOutput out) throws IOException {
+      if (len == 0) {
+        out.writeVInt(0);
+        return;
+      }
+      compressor.setInput(bytes);
+      compressor.finish();
+      if (compressor.needsInput()) {
+        throw new IllegalStateException();
+      }
+
+      int totalCount = 0;
+      for (; ; ) {
+        final int count =
+            compressor.deflate(compressed, totalCount, compressed.length - totalCount);
+        totalCount += count;
+        assert totalCount <= compressed.length;
+        if (compressor.finished()) {
+          break;
+        } else {
+          compressed = ArrayUtil.grow(compressed);
+        }
+      }
+
+      out.writeVInt(totalCount);
+      out.writeBytes(compressed, totalCount);
+    }
+
     @Override
-    public void compress(byte[] bytes, int off, int len, DataOutput out) throws IOException {
+    public void compress(ByteBuffersDataInput buffersInput, DataOutput out) throws IOException {
+      final int len = (int) (buffersInput.size() - buffersInput.position());
+      final int end = (int) buffersInput.size();
       final int dictLength = len / (NUM_SUB_BLOCKS * DICT_SIZE_FACTOR);
       final int blockLength = (len - dictLength + NUM_SUB_BLOCKS - 1) / NUM_SUB_BLOCKS;
       out.writeVInt(dictLength);
       out.writeVInt(blockLength);
-      final int end = off + len;
 
       // Compress the dictionary first
       compressor.reset();
-      doCompress(bytes, off, dictLength, out);
+      bufferDict = ArrayUtil.growNoCopy(bufferDict, dictLength);
+      buffersInput.readBytes(bufferDict, 0, dictLength);
+      doCompress(bufferDict, 0, dictLength, out);
 
       // And then sub blocks
-      for (int start = off + dictLength; start < end; start += blockLength) {
+      for (int start = dictLength; start < end; start += blockLength) {
         compressor.reset();
-        deflaterBugfix.setDictionary(bytes, off, dictLength);
-        doCompress(bytes, start, Math.min(blockLength, off + len - start), out);
+        deflaterBugfix.setDictionary(bufferDict, 0, dictLength);
+        int l = Math.min(blockLength, len - start);
+        // if [start,start + len] stay in one ByteBuffer, we can ignore memory copy
+        // otherwise need to copy bytes into on continuous byte array
+        ByteBuffer bb = buffersInput.sliceOne(start, l);

Review Comment:
   Nice suggestion, at [commits(ebcbd...)](https://github.com/apache/lucene/blob/ebcbdd05fbf114ca15d04cd4a9172eb51fab9a1a/lucene/core/src/java/org/apache/lucene/store/ByteBuffersDataInput.java#L175) i moved this logic into `ByteBuffersDataInput#readBytes`, in the method `readBytes`, it would determine whether use internal buffers or create a new buffer



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene90/compressing/Lucene90CompressingStoredFieldsWriter.java:
##########
@@ -519,7 +518,13 @@ private void copyOneDoc(Lucene90CompressingStoredFieldsReader reader, int docID)
     assert reader.getVersion() == VERSION_CURRENT;
     SerializedDocument doc = reader.document(docID);
     startDocument();
-    bufferedDocs.copyBytes(doc.in, doc.length);
+
+    if (doc.in instanceof ByteArrayDataInput) {
+      // reuse ByteArrayDataInput to reduce memory copy
+      bufferedDocs.copyBytes((ByteArrayDataInput) doc.in, doc.length);
+    } else {
+      bufferedDocs.copyBytes(doc.in, doc.length);
+    }

Review Comment:
   LGTM, i deleted this code at this PR, and i would open a new issue for reduce memory copy in copyOneDoc



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene90/compressing/Lucene90CompressingStoredFieldsWriter.java:
##########
@@ -247,21 +249,18 @@ private void flush(boolean force) throws IOException {
     writeHeader(docBase, numBufferedDocs, numStoredFields, lengths, sliced, dirtyChunk);
 
     // compress stored fields to fieldsStream.
-    //
-    // TODO: do we need to slice it since we already have the slices in the buffer? Perhaps
-    // we should use max-block-bits restriction on the buffer itself, then we won't have to check it
-    // here.
-    byte[] content = bufferedDocs.toArrayCopy();
-    bufferedDocs.reset();
-
     if (sliced) {
-      // big chunk, slice it
-      for (int compressed = 0; compressed < content.length; compressed += chunkSize) {
-        compressor.compress(
-            content, compressed, Math.min(chunkSize, content.length - compressed), fieldsStream);
+      // big chunk, slice it, using ByteBuffersDataInput ignore memory copy
+      ByteBuffersDataInput bytebuffers = bufferedDocs.toDataInput();
+      final int capacity = (int) bytebuffers.size();
+      for (int compressed = 0; compressed < capacity; compressed += chunkSize) {
+        int l = Math.min(chunkSize, capacity - compressed);
+        ByteBuffersDataInput bbdi = bytebuffers.slice(compressed, l);
+        compressor.compress(bbdi, fieldsStream);
       }
     } else {
-      compressor.compress(content, 0, content.length, fieldsStream);
+      ByteBuffersDataInput bytebuffers = bufferedDocs.toDataInput();

Review Comment:
   ++,at [commits(ebcbd...)](https://github.com/apache/lucene/blob/ebcbdd05fbf114ca15d04cd4a9172eb51fab9a1a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/compressing/Lucene90CompressingStoredFieldsWriter.java#L249) i fixed it



-- 
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 a diff in pull request #987: LUCENE-10627: Using ByteBuffersDataInput reduce memory copy on compressing data

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


##########
lucene/core/src/java/org/apache/lucene/store/ByteBuffersDataInput.java:
##########
@@ -165,6 +165,36 @@ public void readBytes(byte[] arr, int off, int len) throws EOFException {
       }
     }
   }
+  /**
+   * ReadBytes from position with length, if [pos, pos + len] stay in one ByteBuffer can ignore
+   * memory copy, otherwise return a new ByteBuffer with continuous byte array
+   *
+   * @param length from position to length
+   * @return ByteBuffer which bytes read from [pos, pos + length]
+   */
+  public ByteBuffer readBytes(int length) throws EOFException {

Review Comment:
   +1, we should rename the method, because it differs from standard readBytes() with multiple params. This one returns a newly allocated ByteBuffer.
   



-- 
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 a diff in pull request #987: LUCENE-10627: Using ByteBuffersDataInput reduce memory copy on compressing data

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


##########
lucene/core/src/java/org/apache/lucene/store/ByteBuffersDataInput.java:
##########
@@ -165,6 +165,36 @@ public void readBytes(byte[] arr, int off, int len) throws EOFException {
       }
     }
   }
+  /**

Review Comment:
   here is a newline missing, I wonder why the formatter does not add it?



-- 
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] luyuncheng commented on a diff in pull request #987: LUCENE-10627: Using ByteBuffersDataInput reduce memory copy on compressing data

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


##########
lucene/core/src/test/org/apache/lucene/codecs/lucene90/compressing/TestCompressingStoredFieldsFormat.java:
##########
@@ -325,4 +341,160 @@ public void testChunkCleanup() throws IOException {
     iw.close();
     dir.close();
   }
+
+  public void testSortedSetVariableLengthBigStoredFields() throws Exception {

Review Comment:
   In the 1st version, i try to use doc values check compressing and decompressing, and i founded it is same as `TestLucene90DocValuesFormat` testcase. 
   at commits [8c59d03](https://github.com/apache/lucene/pull/987/commits/8c59d035c504c9f51c8991a804f73d6ce7074449) i deleted `testSortedSetVariableLengthBigStoredFields`. 
   
   the compressing test case would using `TestDeflateWithPresetDictCompressionMode` and `TestLZ4WithPresetDictCompressionMode`
   
   and i added readNBytes test case in `TestByteBuffersDataInput`



-- 
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] luyuncheng commented on a diff in pull request #987: LUCENE-10627: Using ByteBuffersDataInput reduce memory copy on compressing data

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


##########
lucene/core/src/java/org/apache/lucene/codecs/lucene90/DeflateWithPresetDictCompressionMode.java:
##########
@@ -163,20 +165,22 @@ private static class DeflateWithPresetDictCompressor extends Compressor {
     final Deflater compressor;
     final BugfixDeflater_JDK8252739 deflaterBugfix;
     byte[] compressed;
+    byte[] bufferDict;
     boolean closed;
 
     DeflateWithPresetDictCompressor(int level) {
       compressor = new Deflater(level, true);
       deflaterBugfix = BugfixDeflater_JDK8252739.createBugfix(compressor);
       compressed = new byte[64];
+      bufferDict = BytesRef.EMPTY_BYTES;
     }
 
-    private void doCompress(byte[] bytes, int off, int len, DataOutput out) throws IOException {
+    private void doCompress(ByteBuffer bytes, int len, DataOutput out) throws IOException {

Review Comment:
   LGTM, FIXED it



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

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

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


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


[GitHub] [lucene] jpountz commented on a diff in pull request #987: LUCENE-10627: Using ByteBuffersDataInput reduce memory copy on compressing data

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


##########
lucene/CHANGES.txt:
##########
@@ -54,7 +54,8 @@ Improvements
 
 Optimizations
 ---------------------
-(No changes)
+
+* LUCENE-10627: Using ByteBuffersDataInput reduce memory copy on compressing data. (luyuncheng)

Review Comment:
   Move it to the 9.4 section?



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