Skip to content

Conversation

@Gargi-jais11
Copy link
Contributor

@Gargi-jais11 Gargi-jais11 commented Oct 17, 2025

What changes were proposed in this pull request?

Currently RpcClient has a ElasticByteBufferPool to reuse buffers during EC data read and write. ElasticByteBufferPool can save the time of buffer allocation. While this Pool doesn't have a upper limit, so in s3g case, a long lived RpcClient will accumulate all buffers allocated through this pool, which lead to high memory pressure of s3g.

Solution:
Create a new class implementing ByteBufferPool which will be a bounded version of ElasticByteBufferPool that limits the total size of buffers that can be cached in the pool.
To control the size of this pool added a new configuration :

public static final String OZONE_CLIENT_ELASTIC_BYTE_BUFFER_POOL_MAX_SIZE_GB =
      "ozone.client.elastic.byte.buffer.pool.max.size.gb";
public static final String OZONE_CLIENT_ELASTIC_BYTE_BUFFER_POOL_MAX_SIZE_GB_DEFAULT = "16GB";

In RpcClient use BoundedElasticByteBufferPool instead of ElasticByteBufferPool.

What is the link to the Apache JIRA

https://issues.apache.org/jira/browse/HDDS-13400

How was this patch tested?

Passed Existing Tests and green CI.

@ivandika3 ivandika3 added s3 S3 Gateway EC labels Oct 19, 2025
@Gargi-jais11 Gargi-jais11 marked this pull request as ready for review October 21, 2025 04:17
@Gargi-jais11
Copy link
Contributor Author

@ChenSammi could you please review this patch.

@peterxcli peterxcli requested a review from ChenSammi October 21, 2025 04:38
peterxcli

This comment was marked as outdated.

peterxcli

This comment was marked as duplicate.

Copy link
Member

@peterxcli peterxcli left a comment

Choose a reason for hiding this comment

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

LGTM!

@peterxcli
Copy link
Member

@ChenSammi Would you like to take a look? Thanks!

@Gargi-jais11
Copy link
Contributor Author

@peterxcli Please take a look, I have updated the patch.

Copy link
Member

@peterxcli peterxcli left a comment

Choose a reason for hiding this comment

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

Other looks good.

Comment on lines +476 to +477
like the S3 Gateway. Once this limit is reached, used buffers are not
put back to the pool and will be garbage collected.
Copy link
Member

Choose a reason for hiding this comment

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

used buffers are not put back to the pool and will be garbage collected.

can we help them to deallocate the buffer immediately? so we can reduce the GC pressure.

not quite understand how GC in java works

Copy link
Contributor Author

@Gargi-jais11 Gargi-jais11 Oct 23, 2025

Choose a reason for hiding this comment

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

In Java, we can't deallocate memory manually (like free() in C/C++). The only way to free memory is to remove all references to an object and let the Garbage Collector (GC) reclaim it.
When our pool is full, by returning without storing the buffer, we are doing exactly that. The buffer becomes "unreachable," and the GC will handle its deallocation.

So, I believe while we are still relying on the GC (which is unavoidable in Java), it's for a much smaller fraction of objects, which is exactly the fix we want to reduce overall s3g memory pressure.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

We can also call System.gc() to suggest that the garbage collector run immediately. However, the Java Runtime makes the final decision.
According to the Java documentation.
So immediately deallocating buffer is not allowed by java. However if needed we can use System.gc() to run garbage collector immediately.

Copy link
Member

Choose a reason for hiding this comment

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

Got it, Thanks for the detail explanation. learned a lot!

@Gargi-jais11
Copy link
Contributor Author

@peterxcli Could please re-trigger these failed checks as my CI has passed these checks.
https://github.com/Gargi-jais11/ozone/actions/runs/18738774685

@peterxcli
Copy link
Member

@peterxcli Could please re-trigger these failed checks as my CI has passed these checks. Gargi-jais11/ozone/actions/runs/18738774685

#9130 (comment)
#9130 (comment)

@Gargi-jais11
Copy link
Contributor Author

@peterxcli Could please re-trigger these failed checks as my CI has passed these checks. Gargi-jais11/ozone/actions/runs/18738774685

#9130 (comment) #9130 (comment)

I think I need to rebase my branch as the above Commit has been reverted.

@adoroszlai
Copy link
Contributor

need to rebase my branch

Please use git merge, not git rebase, to avoid force-push.

@Gargi-jais11
Copy link
Contributor Author

need to rebase my branch

Please use git merge, not git rebase, to avoid force-push.

Okay sure.

Comment on lines +60 to +75
@Override
public synchronized ByteBuffer getBuffer(boolean direct, int length) {
TreeMap<Key, ByteBuffer> tree = this.getBufferTree(direct);
Map.Entry<Key, ByteBuffer> entry = tree.ceilingEntry(new Key(length, 0L));
if (entry == null) {
// Pool is empty or has no suitable buffer. Allocate a new one.
return direct ? ByteBuffer.allocateDirect(length) : ByteBuffer.allocate(length);
}
tree.remove(entry.getKey());
ByteBuffer buffer = entry.getValue();

// Decrement the size because we are taking a buffer OUT of the pool.
currentPoolSize.addAndGet(-buffer.capacity());
buffer.clear();
return buffer;
}
Copy link
Member

Choose a reason for hiding this comment

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

Should we also count those "allocated but not released" buffer into the buffer size limit?

Just like BufferPool does:

private final LinkedList<ChunkBuffer> allocated = new LinkedList<>();
private final LinkedList<ChunkBuffer> released = new LinkedList<>();

while (allocated.size() == capacity) {
LOG.debug("Allocation needs to wait the pool is at capacity (allocated = capacity = {}).", capacity);
notFull.await();
}

I know that the original ElasticByteBufferPool doesn't do this.
Just want to make sure if we need to managed the allocated buffer, and why or why not.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I appreciate your suggestion but I think If we did count "allocated but not released" buffers toward the limit, we would be forced to change our getBuffer method to be blocking (i.e., to wait() when the limit is hit).
This would be a major, high-risk change from the original ElasticByteBufferPool's behavior, which always allocates a new buffer immediately. It could introduce performance bottlenecks or even deadlocks.

Copy link
Contributor Author

@Gargi-jais11 Gargi-jais11 Oct 29, 2025

Choose a reason for hiding this comment

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

The BufferPool is linked to is a blocking, fixed-size pool. Its purpose is to strictly limit the total number of buffers ever created (e.g., "this system will only ever use 100 buffers, total"). If you ask for buffer 101, getBuffer will wait until one is returned.

Our BoundedElasticByteBufferPool is a non-blocking, caching pool. Its purpose is to fix a memory leak from the original ElasticByteBufferPool (which grew forever) while preserving its "elastic" (non-blocking) nature.

Copy link
Member

Choose a reason for hiding this comment

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

Sounds good—let’s get this merged.

@peterxcli peterxcli merged commit 388f3d2 into apache:master Oct 29, 2025
43 checks passed
@peterxcli
Copy link
Member

Thanks @Gargi-jais11 for the patch, @adoroszlai for reviewing!

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

EC s3 S3 Gateway

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants