Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Unlimited threads for compaction reservation can oversubscribe manager memory #5177

Closed
keith-turner opened this issue Dec 13, 2024 · 3 comments · Fixed by #5185
Closed

Unlimited threads for compaction reservation can oversubscribe manager memory #5177

keith-turner opened this issue Dec 13, 2024 · 3 comments · Fixed by #5185
Labels
bug This issue has been verified to be a bug.
Milestone

Comments

@keith-turner
Copy link
Contributor

Describe the bug

When a compactor request a compaction job from the manager the following things happen.

  • The request runs in the general client thrift thread pool (note this thread pool will slowly automatically increase)
  • The request reads tablet metadata and then writes a conditional mutation to add the the compaction to tablet metadata.

When there are lots of compactors and there is a problem writing to the metadata table these threads will grow in an unlimited manner. Saw this problem occur and it was probably caused by not having the fixes for #5155 and #5168, however there are many other potential causes that could cause threads to get stuck or be slow.

Expected behavior

The number of threads concurrently executing compaction reservation is somehow constrained. This must be done w/o blocking other manager functionality. A simple way to achieve this goal would be to add a semaphore around this functionality, however this would cause general thrift threads that execute all manager functionality to block which could cause other problems but maybe that is ok since the manager thread pool always grows.

One possible way to achieve this goal would be to use #5018 and in the async code for executing compaction reservation runs in a limited thread pool. #5018 was created for performance reason, but it can also easily satisfy this goal of protecting manager memory.

Another possible way to achieve this goal would be run another thrift server w/ its own port for getting compaction jobs and limit the thread pool size for this.

@keith-turner keith-turner added the bug This issue has been verified to be a bug. label Dec 13, 2024
@keith-turner keith-turner added this to the 4.0.0 milestone Dec 13, 2024
@cshannon
Copy link
Contributor

cshannon commented Dec 13, 2024

The Jetty solution (or another async solution like gRPC) would be interesting to try because you'd get the performance benefits as well. As you pointed out, we'd need to make sure to limit things by passing in a thread pool when there is a job available here and that should prevent trying to reserve too many jobs at once.

Another thing that would help solve this problem would be #4978 because if those changes were made then the compaction coordinator would no longer be handling the job reservations as that responsibility would be moved to the compactor itself

@keith-turner
Copy link
Contributor Author

Another thing that would help solve this problem would be #4978 because if those changes were made then the compaction coordinator would no longer be handling the job reservations as that responsibility would be moved to the compactor itself

That is a good point. I made a comment on 4978 about async processing for tserver conditional mutation processing. I think that would make that change work really smoothly, but may not be required for it work smoothly.

@cshannon
Copy link
Contributor

That is a good point. I made a comment on 4978 about async processing for tserver conditional mutation processing. I think that would make that change work really smoothly, but may not be required for it work smoothly.

I think the final solution will probably be both changes working together to make it very fast and lower the required resources. if we move the compaction job request to async then that keeps the request threadpool free so we don't block other requests, it allows us to bound the number job we requests we process to at one time with a worker thread pool, and it moves the reservation to the compactor which reduces the load even further on the manager.

keith-turner added a commit to keith-turner/accumulo that referenced this issue Dec 14, 2024
Added threads pools to execute compaction reservation request in order
to limit memory and cpu used by executing reservations.  Request queued up
for the pool could still potentially use a lot of memory.  Did two
things to control memory of things in the queue.  First only allow a
compactor process to have one reservation processing at time.  Second
made the data related to a resevation request a soft reference which
should allow it be garbage collected if memory gets low while it sitting
in the queue.  Once the request starts executing it obtains a strong
refrence to the data so it can no longer be garbage collected.

fixes apache#5177
keith-turner added a commit that referenced this issue Dec 15, 2024
Added threads pools to execute compaction reservation request in order
to limit memory and cpu used by executing reservations.  Request queued up
for the pool could still potentially use a lot of memory.  Did two
things to control memory of things in the queue.  First only allow a
compactor process to have one reservation processing at time.  Second
made the data related to a resevation request a soft reference which
should allow it be garbage collected if memory gets low while it sitting
in the queue.  Once the request starts executing it obtains a strong
refrence to the data so it can no longer be garbage collected.

fixes #5177
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug This issue has been verified to be a bug.
Projects
None yet
Development

Successfully merging a pull request may close this issue.

2 participants