Skip to content

[fix][client]Fix Producer exceeds PulsarClient memory limit when sending fail timeout using CompressionType.NONE#17663

Closed
AnonHxy wants to merge 2 commits intoapache:masterfrom
AnonHxy:fix_exceeded_memory_producer
Closed

[fix][client]Fix Producer exceeds PulsarClient memory limit when sending fail timeout using CompressionType.NONE#17663
AnonHxy wants to merge 2 commits intoapache:masterfrom
AnonHxy:fix_exceeded_memory_producer

Conversation

@AnonHxy
Copy link
Contributor

@AnonHxy AnonHxy commented Sep 15, 2022

Fixes #17662

Motivation

  • Fixes [Bug] Producer exceeds PulsarClient memory limit when sending fail timeout using CompressionType.NONE #17662

  • The root cause is that:

    1. After we ever sent a big message, e.g, 2M bytes, the AbstractBatchMessageContainer#maxBatchSize will never be less than 2M bytes(see line171) in the following batches, even if the message size we added is 1 bytes only.

      }
      // Update the current max batch size using the uncompressed size, which is what we need in any case to
      // accumulate the batch content
      maxBatchSize = Math.max(maxBatchSize, uncompressedSize);
      return compressedPayload;
      }

    2. So when we add the first message in the following batches, we will always allocate a byte buffer greater or equals to 2M, see line102

      messageMetadata.setSequenceId(msg.getSequenceId());
      lowestSequenceId = Commands.initBatchMessageMetadata(messageMetadata, msg.getMessageBuilder());
      this.firstCallback = callback;
      batchedMessageMetadataAndPayload = allocator.buffer(
      Math.min(maxBatchSize, ClientCnx.getMaxMessageSize()));
      if (msg.getMessageBuilder().hasTxnidMostBits() && currentTxnidMostBits == -1) {

    3. When we use CompressionCodecNone to encode the 2M bytes buffer it just retain the buffer, which means the 2M bytes buffer will be added to ProducerImpl#pendingMessages and will be release until timeout(30s by default). Note that we sent 1 bytes message(or batched, but it should much less that 2M), but retain 2M bytes buffer, so it will exceed PulsarClient memory limit.

Modifications

Copy readable bytes to a new Bytebuf if necessary, like CompressionCodecLZ4 or other CompressionCodec implement

Verifying this change

  • Make sure that the change passes the CI checks.

Documentation

  • doc-required
    (Your PR needs to update docs and you will update later)

  • doc-not-needed
    (Please explain why)

  • doc
    (Your PR contains doc changes)

  • doc-complete
    (Docs have been already added)

@github-actions github-actions bot added the doc-not-needed Your PR changes do not impact docs label Sep 15, 2022
@AnonHxy AnonHxy self-assigned this Sep 15, 2022
@AnonHxy AnonHxy changed the title [fix][client]Fix Producer exceeds PulsarClient memory limit when sending fail timeout [WIP][fix][client]Fix Producer exceeds PulsarClient memory limit when sending fail timeout Sep 15, 2022
@AnonHxy AnonHxy changed the title [WIP][fix][client]Fix Producer exceeds PulsarClient memory limit when sending fail timeout [fix][client]Fix Producer exceeds PulsarClient memory limit when sending fail timeout Sep 15, 2022
target.writeBytes(raw);
raw.resetReaderIndex();
return target;
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do you mean: The MemoryLimitController manages memory according to the real size of message, but the message is allocated a bigger memory by BatchMessageContainerImpl?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Right.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

Copy link
Member

@lhotari lhotari left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good work.

I just wonder if the title of the PR should mention that this applies to CompressionType.NONE .

@AnonHxy AnonHxy changed the title [fix][client]Fix Producer exceeds PulsarClient memory limit when sending fail timeout [fix][client]Fix Producer exceeds PulsarClient memory limit when sending fail timeout using CompressionType.NONE Sep 15, 2022
@AnonHxy
Copy link
Contributor Author

AnonHxy commented Sep 15, 2022

Good work.

I just wonder if the title of the PR should mention that this applies to CompressionType.NONE .

Sure

@AnonHxy AnonHxy added the type/bug The PR fixed a bug or issue reported a bug label Sep 15, 2022
ByteBuf target = PulsarByteBufAllocator.DEFAULT.buffer(readableBytes, readableBytes);
target.writeBytes(raw);
raw.resetReaderIndex();
return target;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think this is the right solution because it would cause a memory copy for all the batches.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1.

  1. We should take the whole buffer size into account in MemoryLimitController. Because it's allocated by our producer.
  2. There is a memory waste problem in producer batch container, as the max buffer size only grows.

ByteBuf target = PulsarByteBufAllocator.DEFAULT.buffer(readableBytes, readableBytes);
target.writeBytes(raw);
raw.resetReaderIndex();
return target;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1.

  1. We should take the whole buffer size into account in MemoryLimitController. Because it's allocated by our producer.
  2. There is a memory waste problem in producer batch container, as the max buffer size only grows.

@Jason918
Copy link
Contributor

@AnonHxy Please take a look at this PR #15033. It tries to solve the memory waste problem.

@AnonHxy
Copy link
Contributor Author

AnonHxy commented Sep 27, 2022

closed this pr becase it' not the best solution

@AnonHxy AnonHxy closed this Sep 27, 2022
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

doc-not-needed Your PR changes do not impact docs type/bug The PR fixed a bug or issue reported a bug

Projects

None yet

Development

Successfully merging this pull request may close these issues.

[Bug] Producer exceeds PulsarClient memory limit when sending fail timeout using CompressionType.NONE

5 participants