You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2022/04/06 02:47:57 UTC

[GitHub] [pulsar] tjiuming opened a new pull request, #15033: [client-producer] avoid too large memory preallocation for batch message.

tjiuming opened a new pull request, #15033:
URL: https://github.com/apache/pulsar/pull/15033

   Fixes (https://github.com/apache/pulsar/issues/14943)
   
   ### Motivation
   
   Allocate small memory for batch message first, and let it grow(with no memory resizes and memory copies).
   
   ### Does this pull request potentially affect one of the following parts:
   
   *If `yes` was chosen, please highlight the changes*
   
     - Dependencies (does it add or upgrade a dependency): (no)
     - The public API: (no)
     - The schema: (no)
     - The default values of configurations: (no)
     - The wire protocol: (no)
     - The rest endpoints: (no)
     - The admin cli options: (no)
     - Anything that affects deployment: (no)
   
   ### Documentation
   
   Check the box below or label this PR directly (if you have committer privilege).
   
   Need to update docs? 
   
   - [ ] `doc-required` 
     
     (If you need help on updating docs, create a doc issue)
     
   - [x] `no-need-doc` 
     
     (Please explain why)
     
   - [ ] `doc` 
     
     (If this PR contains doc changes)
   
   
   


-- 
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: commits-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] AnonHxy commented on pull request #15033: [improvement][client-java] Avoid too large memory preallocation for batch message.

Posted by GitBox <gi...@apache.org>.
AnonHxy commented on PR #15033:
URL: https://github.com/apache/pulsar/pull/15033#issuecomment-1250176322

   This patch  broken `org.apache.pulsar.client.impl.BatchMessageContainerImplTest.recoveryAfterOom` test. Please help fix it , thanks @tjiuming 


-- 
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: commits-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] AnonHxy commented on pull request #15033: [improvement][client-java] Avoid too large memory preallocation for batch message.

Posted by GitBox <gi...@apache.org>.
AnonHxy commented on PR #15033:
URL: https://github.com/apache/pulsar/pull/15033#issuecomment-1250422496

   > > This patch broken `org.apache.pulsar.client.impl.BatchMessageContainerImplTest.recoveryAfterOom` test. Please help fix it , thanks @tjiuming
   > 
   > @AnonHxy Yes, I've fixed, PTAL
   
   Also have some checkstyle failure. Please make sure that CI could be pass in your own fork repo first~  @tjiuming 


-- 
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: commits-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] tjiuming commented on pull request #15033: [client-producer] avoid too large memory preallocation for batch message.

Posted by GitBox <gi...@apache.org>.
tjiuming commented on PR #15033:
URL: https://github.com/apache/pulsar/pull/15033#issuecomment-1214365957

   /pulsarbot run-failure-checks


-- 
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: commits-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] AnonHxy commented on pull request #15033: [improvement][client-java] Avoid too large memory preallocation for batch message.

Posted by GitBox <gi...@apache.org>.
AnonHxy commented on PR #15033:
URL: https://github.com/apache/pulsar/pull/15033#issuecomment-1257728382

   >  Is there a significant throughput difference between these two?
   
   OK.  @Jason918 
   1. First I setup up a standalone cluster using apache-pulsar-2.10.1
   2. Then started a producer like below to publish messages, starting with command `~/houxiaoyu/jdk-17.0.4/bin/java -Xms2000m -Xmx2000m -XX:MaxDirectMemorySize=4000m -jar pulsar-buffer.jar`:
   ```
     public static void main(String[] args) throws Exception {
           String topic = "hxy-test-mem";
           byte[] smallMsg = new byte[64];
           PulsarClient client = PulsarClient.builder()
                   .serviceUrl("pulsar://127.0.0.1:6650")
                   .memoryLimit(2000, SizeUnit.MEGA_BYTES)  // memory limit 2000M
                   .build();
           Producer<byte[]> producer = client.newProducer()
                   .topic(topic)
                   .enableBatching(true)
                   .create();
           for (int i = 0; i < Integer.MAX_VALUE; ++i) {
               producer.sendAsync(smallMsg);
           }
       }
   ```
   3.  I have compared the producer stats three times, and all of them came to the same conclusion that throughput will be descended when using `compositeBuffer`.  Here is the details logs:
      *  first time test:
     
         buffer:
   <img width="1919" alt="image" src="https://user-images.githubusercontent.com/10233437/192234832-02ed6234-1d1d-4b30-9daf-812895577cb5.png">
        compositeBuffer:
   <img width="1919" alt="image" src="https://user-images.githubusercontent.com/10233437/192235247-75d5a8bc-c7d6-4127-9e21-a546f36ec29b.png">
   
     * second time test
     
       buffer:
   <img width="1919" alt="image" src="https://user-images.githubusercontent.com/10233437/192236074-930351a5-9798-4b24-ae54-93b8280b03a1.png">
       compositeBuffer:
   <img width="1918" alt="image" src="https://user-images.githubusercontent.com/10233437/192236240-279ec3dd-f4d3-4ae9-96a2-38623cd2c7ce.png">
   
      * third time test
      
       buffer:
   <img width="1919" alt="image" src="https://user-images.githubusercontent.com/10233437/192236536-4f8c25dd-54cb-4be2-a3e6-2324221dd873.png">
   
      compositeBuffer:   
   <img width="1919" alt="image" src="https://user-images.githubusercontent.com/10233437/192236617-ff4a96e5-fb7a-4fec-8ec4-9b228da2a43f.png">
   
   
   
   
   
   
   


-- 
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: commits-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] github-actions[bot] commented on pull request #15033: [improvement][client-java] Avoid too large memory preallocation for batch message.

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #15033:
URL: https://github.com/apache/pulsar/pull/15033#issuecomment-1249995475

   @tjiuming Please provide a correct documentation label for your PR.
   Instructions see [Pulsar Documentation Label Guide](https://docs.google.com/document/d/1Qw7LHQdXWBW9t2-r-A7QdFDBwmZh6ytB4guwMoXHqc0).


-- 
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: commits-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] tjiuming commented on pull request #15033: [improvement][client-java] Avoid too large memory preallocation for batch message.

Posted by GitBox <gi...@apache.org>.
tjiuming commented on PR #15033:
URL: https://github.com/apache/pulsar/pull/15033#issuecomment-1252683428

   @merlimat 
   Yes, there is allocator synchronization, but in `ProducerImpl#sendAsync(Message<?> message, SendCallback callback)`:
      ```
               try {
               synchronized (this) {
                 ......
               }
           } catch (PulsarClientException e) {
                ......
           } catch (Throwable t) {
               ......
           }
      ``` 
    which means for a Producer, only 1 thread could send message at the same time, I think even there is allocator synchronization, it wouldn't have a significant large impact.
   
   > Scattered write on the socket from many small buffers
   
   For this part, I don't think it will make a big difference. Yes, read bytes from `CompositeByteBuf` is truly slower than 1 single large buffer, because it need to calculate the `readerIndex`, but when write it to channel, `CompositeByteBuf` will transfer to `ByteBuffer[]`, it would not too much slower than `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: commits-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] AnonHxy commented on pull request #15033: [client-producer] avoid too large memory preallocation for batch message.

Posted by GitBox <gi...@apache.org>.
AnonHxy commented on PR #15033:
URL: https://github.com/apache/pulsar/pull/15033#issuecomment-1249146241

   Please rebase the lastest master branch and push it force . `pulsarbot run-failure-checks` will not help now @tjiuming 


-- 
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: commits-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] tjiuming commented on pull request #15033: [client-producer] avoid too large memory preallocation for batch message.

Posted by GitBox <gi...@apache.org>.
tjiuming commented on PR #15033:
URL: https://github.com/apache/pulsar/pull/15033#issuecomment-1249121068

   /pulsarbot run-failure-checks


-- 
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: commits-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] tjiuming commented on pull request #15033: [improvement][client-java] Avoid too large memory preallocation for batch message.

Posted by GitBox <gi...@apache.org>.
tjiuming commented on PR #15033:
URL: https://github.com/apache/pulsar/pull/15033#issuecomment-1252553172

   @merlimat @AnonHxy @Jason918 
   For `CompositeByteBuf`, if the number of components in the buffer not exceeds `maxComponent`, it wound not resize/mem_copy. But if the capital is not enough and need a new component, all the components will merge(copy) into a new component.
   Here I use the default value of `maxComponent`, which is 16. Because I'm not sure what exact value to set.
   But on the whole, the buffer will be allocated lazily, which means fewer wasted memory. To some extent, it achieves the purpose of saving memory.
   To reduce mem_copy/resize, maybe we can calculate the certain value of `maxComponent` with `maxBatchSize`, is it required?


-- 
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: commits-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] AnonHxy commented on pull request #15033: [improvement][client-java] Avoid too large memory preallocation for batch message.

Posted by GitBox <gi...@apache.org>.
AnonHxy commented on PR #15033:
URL: https://github.com/apache/pulsar/pull/15033#issuecomment-1252065209

   > @AnonHxy Can we do a quick test to validate this? Focus on the case with large number of messages in one batch.
   
   Sure


-- 
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: commits-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] tjiuming commented on pull request #15033: [client-producer] avoid too large memory preallocation for batch message.

Posted by GitBox <gi...@apache.org>.
tjiuming commented on PR #15033:
URL: https://github.com/apache/pulsar/pull/15033#issuecomment-1201498089

   /pulsarbot run-failure-checks


-- 
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: commits-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] AnonHxy commented on a diff in pull request #15033: [client-producer] avoid too large memory preallocation for batch message.

Posted by GitBox <gi...@apache.org>.
AnonHxy commented on code in PR #15033:
URL: https://github.com/apache/pulsar/pull/15033#discussion_r972987262


##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageContainerImpl.java:
##########
@@ -69,8 +69,7 @@ public boolean add(MessageImpl<?> msg, SendCallback callback) {
                 messageMetadata.setSequenceId(msg.getSequenceId());
                 lowestSequenceId = Commands.initBatchMessageMetadata(messageMetadata, msg.getMessageBuilder());
                 this.firstCallback = callback;
-                batchedMessageMetadataAndPayload = PulsarByteBufAllocator.DEFAULT
-                        .buffer(Math.min(maxBatchSize, ClientCnx.getMaxMessageSize()));
+                batchedMessageMetadataAndPayload = PulsarByteBufAllocator.DEFAULT.compositeBuffer();

Review Comment:
   Could you please do your final comment @merlimat 



-- 
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: commits-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] AnonHxy merged pull request #15033: [improvement][client-java] Avoid too large memory preallocation for batch message.

Posted by GitBox <gi...@apache.org>.
AnonHxy merged PR #15033:
URL: https://github.com/apache/pulsar/pull/15033


-- 
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: commits-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] Jason918 commented on pull request #15033: [improvement][client-java] Avoid too large memory preallocation for batch message.

Posted by GitBox <gi...@apache.org>.
Jason918 commented on PR #15033:
URL: https://github.com/apache/pulsar/pull/15033#issuecomment-1257591497

   @AnonHxy Is there a significant throughput difference between these two? 


-- 
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: commits-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] tjiuming commented on a diff in pull request #15033: [client-producer] avoid too large memory preallocation for batch message.

Posted by GitBox <gi...@apache.org>.
tjiuming commented on code in PR #15033:
URL: https://github.com/apache/pulsar/pull/15033#discussion_r882051495


##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageContainerImpl.java:
##########
@@ -69,8 +69,7 @@ public boolean add(MessageImpl<?> msg, SendCallback callback) {
                 messageMetadata.setSequenceId(msg.getSequenceId());
                 lowestSequenceId = Commands.initBatchMessageMetadata(messageMetadata, msg.getMessageBuilder());
                 this.firstCallback = callback;
-                batchedMessageMetadataAndPayload = PulsarByteBufAllocator.DEFAULT
-                        .buffer(Math.min(maxBatchSize, ClientCnx.getMaxMessageSize()));
+                batchedMessageMetadataAndPayload = PulsarByteBufAllocator.DEFAULT.compositeBuffer();

Review Comment:
   Buffer expansions will happens in some situations, but for `CompositeByteBuf`, it wouldn't takes too much costs.
   `mem_copy` operations triggered when `CompositeByteBuf#consolidate0`, which means after write more than 1000000 bytes into buffer, `mem_copy` will be triggered first time by default. I believe it's acceptable in the `client side`



-- 
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: commits-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] AnonHxy commented on pull request #15033: [client-producer] avoid too large memory preallocation for batch message.

Posted by GitBox <gi...@apache.org>.
AnonHxy commented on PR #15033:
URL: https://github.com/apache/pulsar/pull/15033#issuecomment-1249022047

   @tjiuming  Please help fix the CI failure


-- 
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: commits-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] AnonHxy closed pull request #15033: [improvement][client-java] Avoid too large memory preallocation for batch message.

Posted by GitBox <gi...@apache.org>.
AnonHxy closed pull request #15033: [improvement][client-java] Avoid too large memory preallocation for batch message.
URL: https://github.com/apache/pulsar/pull/15033


-- 
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: commits-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] AnonHxy commented on pull request #15033: [improvement][client-java] Avoid too large memory preallocation for batch message.

Posted by GitBox <gi...@apache.org>.
AnonHxy commented on PR #15033:
URL: https://github.com/apache/pulsar/pull/15033#issuecomment-1257745082

   And here is the attachment of Flame Graph
   [buffer.html.zip](https://github.com/apache/pulsar/files/9644855/buffer.html.zip)
   [compositeBuffer.html.zip](https://github.com/apache/pulsar/files/9644857/compositeBuffer.html.zip)
   


-- 
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: commits-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] tjiuming commented on pull request #15033: [client-producer] avoid too large memory preallocation for batch message.

Posted by GitBox <gi...@apache.org>.
tjiuming commented on PR #15033:
URL: https://github.com/apache/pulsar/pull/15033#issuecomment-1172867842

   @codelipenghui Please help merge the PR


-- 
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: commits-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] AnonHxy commented on pull request #15033: [improvement][client-java] Avoid too large memory preallocation for batch message.

Posted by GitBox <gi...@apache.org>.
AnonHxy commented on PR #15033:
URL: https://github.com/apache/pulsar/pull/15033#issuecomment-1251762196

   Hi @merlimat , `compositeBuffer` will not lead to memory copies until the `componentCount`  reached the `maxNumComponents`, which default value is 16. 
   
    If we want to avoid any memory, I think we could set the valule of `maxNumComponents`  equals to `batchingMaxMessages`,  which defalut is 1000. WDYT


-- 
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: commits-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] AnonHxy commented on pull request #15033: [improvement][client-java] Avoid too large memory preallocation for batch message.

Posted by GitBox <gi...@apache.org>.
AnonHxy commented on PR #15033:
URL: https://github.com/apache/pulsar/pull/15033#issuecomment-1257222675

   I have tested this feature using standalone cluster.  It seems that  this PR may bring a performance regression. The reason is that:
   When using `ByteBufAllocator#buffer(int)`, the Bytebuf writing took only a little cpu time:
   <img width="1919" alt="image" src="https://user-images.githubusercontent.com/10233437/192152190-853d90a2-1cff-45c3-be94-eadba98914fd.png">
   
   And when we using `ByteBufAllocator#compositeBuffer`, the Bytebuf wrte took more cpu time:
   <img width="1919" alt="image" src="https://user-images.githubusercontent.com/10233437/192152247-2edf193a-02c5-408a-a86e-bc0a4a029eab.png">
   
   I think another solution is that we can introduce a way to shrink the `maxBatchSize`. For example,  if current batch size is less that `3/4 * maxBatchSize` we will let `maxBatchSize = 3/4 * maxBatchSize`
   
   @merlimat @tjiuming @Jason918 @codelipenghui 
   
   


-- 
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: commits-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] github-actions[bot] commented on pull request #15033: [client-producer] avoid too large memory preallocation for batch message.

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #15033:
URL: https://github.com/apache/pulsar/pull/15033#issuecomment-1132374124

   The pr had no activity for 30 days, mark with Stale label.


-- 
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: commits-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] merlimat commented on a diff in pull request #15033: [client-producer] avoid too large memory preallocation for batch message.

Posted by GitBox <gi...@apache.org>.
merlimat commented on code in PR #15033:
URL: https://github.com/apache/pulsar/pull/15033#discussion_r882028551


##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageContainerImpl.java:
##########
@@ -69,8 +69,7 @@ public boolean add(MessageImpl<?> msg, SendCallback callback) {
                 messageMetadata.setSequenceId(msg.getSequenceId());
                 lowestSequenceId = Commands.initBatchMessageMetadata(messageMetadata, msg.getMessageBuilder());
                 this.firstCallback = callback;
-                batchedMessageMetadataAndPayload = PulsarByteBufAllocator.DEFAULT
-                        .buffer(Math.min(maxBatchSize, ClientCnx.getMaxMessageSize()));
+                batchedMessageMetadataAndPayload = PulsarByteBufAllocator.DEFAULT.compositeBuffer();

Review Comment:
   This approach could result in many buffer expansions when the average buffer size is larger than the "default" size of allocated buffers.



-- 
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: commits-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] tjiuming commented on pull request #15033: [client-producer] avoid too large memory preallocation for batch message.

Posted by GitBox <gi...@apache.org>.
tjiuming commented on PR #15033:
URL: https://github.com/apache/pulsar/pull/15033#issuecomment-1137753302

   @codelipenghui seems the PR is able to merge


-- 
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: commits-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] merlimat commented on pull request #15033: [improvement][client-java] Avoid too large memory preallocation for batch message.

Posted by GitBox <gi...@apache.org>.
merlimat commented on PR #15033:
URL: https://github.com/apache/pulsar/pull/15033#issuecomment-1252530844

   @AnonHxy If we are writing bytes into the composite buffer, it either needs to have space into its own last buffer or to allocate a buffer and copy data into 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: commits-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] merlimat commented on pull request #15033: [improvement][client-java] Avoid too large memory preallocation for batch message.

Posted by GitBox <gi...@apache.org>.
merlimat commented on PR #15033:
URL: https://github.com/apache/pulsar/pull/15033#issuecomment-1252613717

   > Here I use the default value of maxComponent, which is 16. Because I was not sure what exact value to set.
   > But on the whole, the buffer will be allocated lazily, which means fewer wasted memory. To some extent, it achieves the purpose of saving memory.
   
   Allocating many small buffers will have a significant large impact compared to allocating 1 single large buffer: 
    1. Netty allocator synchronization
    2. Scattered write on the socket from many small buffers
   


-- 
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: commits-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] Jason918 commented on pull request #15033: [improvement][client-java] Avoid too large memory preallocation for batch message.

Posted by GitBox <gi...@apache.org>.
Jason918 commented on PR #15033:
URL: https://github.com/apache/pulsar/pull/15033#issuecomment-1251789057

   @AnonHxy Can we do a quick test to validate this? Focus on the case with large number of messages in one batch.


-- 
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: commits-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] github-actions[bot] commented on pull request #15033: [client-producer] avoid too large memory preallocation for batch message.

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #15033:
URL: https://github.com/apache/pulsar/pull/15033#issuecomment-1166399374

   The pr had no activity for 30 days, mark with Stale label.


-- 
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: commits-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] Jason918 commented on pull request #15033: [client-producer] avoid too large memory preallocation for batch message.

Posted by GitBox <gi...@apache.org>.
Jason918 commented on PR #15033:
URL: https://github.com/apache/pulsar/pull/15033#issuecomment-1173272136

   > @codelipenghui Please help merge the PR
   
   Please fix the CI failure


-- 
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: commits-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] tjiuming commented on pull request #15033: [client-producer] avoid too large memory preallocation for batch message.

Posted by GitBox <gi...@apache.org>.
tjiuming commented on PR #15033:
URL: https://github.com/apache/pulsar/pull/15033#issuecomment-1173343740

   /pulsarbot run-failure-checks


-- 
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: commits-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] merlimat commented on pull request #15033: [improvement][client-java] Avoid too large memory preallocation for batch message.

Posted by GitBox <gi...@apache.org>.
merlimat commented on PR #15033:
URL: https://github.com/apache/pulsar/pull/15033#issuecomment-1251465417

   @tjiuming @Jason918 @AnonHxy 
   
   > Allocate small memory for batch message first, and let it grow(with no memory resizes and memory copies).
   
   How can you say there are no memory resizes or copies? 
   
   I'm still not convinced that this change is not bringing a significant performance regression.
   
   
   


-- 
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: commits-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] tjiuming commented on pull request #15033: [improvement][client-java] Avoid too large memory preallocation for batch message.

Posted by GitBox <gi...@apache.org>.
tjiuming commented on PR #15033:
URL: https://github.com/apache/pulsar/pull/15033#issuecomment-1250311640

   > This patch broken `org.apache.pulsar.client.impl.BatchMessageContainerImplTest.recoveryAfterOom` test. Please help fix it , thanks @tjiuming
   
   @AnonHxy Yes, I've fixed, PTAL


-- 
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: commits-unsubscribe@pulsar.apache.org

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