-
Notifications
You must be signed in to change notification settings - Fork 3.3k
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
HBASE-25547: Thread pools should release unused resources #2922
Conversation
🎊 +1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Should we also make the type of threads queue configurable? So far, inside ExecutorService.Executor we are tightly creating an instance of LinkedBlockingQueue. My understanding is that with unbounded queues, maxPoolSize has no effect.
hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorService.java
Outdated
Show resolved
Hide resolved
hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorService.java
Outdated
Show resolved
Hide resolved
return maxPoolSize; | ||
} | ||
|
||
public ExecutorConfig setMaxPoolSize(int maxPoolSize) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
What if the maxPoolSize is set to less than the corePoolSize? Would it be worth to check and automatically set to same as corePoolSize?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
ThreadPoolConstructor throws in that case.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks @wchevreuil for your comments, got a couple of questions before I push a new rev.
Should we also make the type of threads queue configurable?
Just to be sure, you mean ExecutorType enum? Ya thats a good idea.
My understanding is that with unbounded queues, maxPoolSize has no effect.
Not sure I follow. maxPoolSize is the max number of threads not the size of backing workQueue (javadoc). Mind rephrasing the question?
My understanding is that maxPoolSize works as a cap mostly when queueing wasn't possible, for example, because the queue was full. Revisited ThreadPoolExecutor docs:
"
...
- If fewer than corePoolSize threads are running, the Executor always prefers adding a new thread rather than queuing.
- If corePoolSize or more threads are running, the Executor always prefers queuing a request rather than adding a new thread.
- If a request cannot be queued, a new thread is created unless this would exceed maximumPoolSize, in which case, the task will be rejected.
...
There are three general strategies for queuing:
...
Unbounded queues. Using an unbounded queue (for example a LinkedBlockingQueue without a predefined capacity) will cause new tasks to wait in the queue when all corePoolSize threads are busy. Thus, no more than corePoolSize threads will ever be created. (And the value of the maximumPoolSize therefore doesn't have any effect.)
...
"
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Agree and we use unbounded queue:
// work queue to use - unbounded queue
final BlockingQueue<Runnable> q = new LinkedBlockingQueue<>();
Looks like our implementation never needed to worry about maxPoolSize so far, interesting!
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks @wchevreuil for your comments, got a couple of questions before I push a new rev.
Should we also make the type of threads queue configurable?
Just to be sure, you mean ExecutorType enum? Ya thats a good idea.
My understanding is that with unbounded queues, maxPoolSize has no effect.
Not sure I follow. maxPoolSize is the max number of threads not the size of backing workQueue (javadoc). Mind rephrasing the question?
return maxPoolSize; | ||
} | ||
|
||
public ExecutorConfig setMaxPoolSize(int maxPoolSize) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
ThreadPoolConstructor throws in that case.
hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorService.java
Outdated
Show resolved
Hide resolved
hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorService.java
Outdated
Show resolved
Hide resolved
hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorService.java
Show resolved
Hide resolved
hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorService.java
Outdated
Show resolved
Hide resolved
return maxPoolSize; | ||
} | ||
|
||
public ExecutorConfig setMaxPoolSize(int maxPoolSize) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Agree and we use unbounded queue:
// work queue to use - unbounded queue
final BlockingQueue<Runnable> q = new LinkedBlockingQueue<>();
Looks like our implementation never needed to worry about maxPoolSize so far, interesting!
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Skimmed. Looks really nice.
keepAliveTimeInMillis, TimeUnit.MILLISECONDS, q); | ||
config.getCorePoolSize(), config.getMaxPoolSize(), | ||
config.getKeepAliveTimeMillis(), TimeUnit.MILLISECONDS, q); | ||
this.threadPoolExecutor.allowCoreThreadTimeOut(config.allowCoreThreadTimeout()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Great
dd54ffb
to
492ec19
Compare
hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorService.java
Show resolved
Hide resolved
hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorService.java
Show resolved
Hide resolved
aa551b4
to
a8b6fd7
Compare
🎊 +1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
Enabled timeout of core threads for some low priority thread pools to reclaim unused resources.
a8b6fd7
to
881e4d9
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@wchevreuil @virajjasani Any more comments? Rebased on the latest tip fwiw.
🎊 +1 overall
This message was automatically generated. |
So far, we are starting off with terminating core worker threads if no tasks arrive in keepAlive time only for EventTypes:
|
Instead moved it to the actual method. Copy-pasting it to multiple places seems redundant.
JVM thread creation is expensive. It involves initializing the stack memory, creating a corresponding native thread etc.. We may not want to do it for all thread pools for performance sake. We can add other thread pools over time as we see fit but turning it on by default might have some unintended and hard to track down performance issues. |
🎊 +1 overall
This message was automatically generated. |
Oh Great, you are right. Thread creation should be considered only if necessary for tasks like merge regions (recent change), where we are certain that it might happen for less than 1% of total cluster's available times. We are good with default value false. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
+1, pending QA
/** | ||
* Allows timing out of core threads. Good to set this for non-critical thread pools for | ||
* release of unused resources. Refer to {@link ThreadPoolExecutor#allowCoreThreadTimeOut} | ||
* for additional details. | ||
*/ |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Perfect!
🎊 +1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
@bharathv When you commit, can you please mention 4 EventTypes for which this commit is going to consider releasing unused core threads in thread pools? Maybe you can mention them in Release notes as well. |
I just figured that I missed the commit that rolls the op-code into the ExecutorConfig, let me create an addendum. |
Plumbs the configuration needed to enable core thread timeout on non-critical thread pools. Currently only enabled for thread pools with op-codes RS_LOG_REPLAY_OPS, RS_PARALLEL_SEEK, MASTER_SNAPSHOT_OPERATIONS, MASTER_MERGE_OPERATIONS. Others can be added later as needed. Signed-off-by: Michael Stack <stack@apache.org> Signed-off-by: Viraj Jasani <vjasani@apache.org> Signed-off-by: Wellington Chevreuil <wchevreuil@apache.org>
Plumbs the configuration needed to enable core thread timeout on non-critical thread pools. Currently only enabled for thread pools with op-codes RS_LOG_REPLAY_OPS, RS_PARALLEL_SEEK, MASTER_SNAPSHOT_OPERATIONS, MASTER_MERGE_OPERATIONS. Others can be added later as needed. Signed-off-by: Michael Stack <stack@apache.org> Signed-off-by: Viraj Jasani <vjasani@apache.org> Signed-off-by: Wellington Chevreuil <wchevreuil@apache.org>
Plumbs the configuration needed to enable core thread timeout on non-critical thread pools. Currently only enabled for thread pools with op-codes RS_LOG_REPLAY_OPS, RS_PARALLEL_SEEK, MASTER_SNAPSHOT_OPERATIONS, MASTER_MERGE_OPERATIONS. Others can be added later as needed. Signed-off-by: Michael Stack <stack@apache.org> Signed-off-by: Viraj Jasani <vjasani@apache.org> Signed-off-by: Wellington Chevreuil <wchevreuil@apache.org> (cherry picked from commit 618236d)
* HBASE-25547: Thread pools should release unused resources (#2922) Plumbs the configuration needed to enable core thread timeout on non-critical thread pools. Currently only enabled for thread pools with op-codes RS_LOG_REPLAY_OPS, RS_PARALLEL_SEEK, MASTER_SNAPSHOT_OPERATIONS, MASTER_MERGE_OPERATIONS. Others can be added later as needed. Signed-off-by: Michael Stack <stack@apache.org> Signed-off-by: Viraj Jasani <vjasani@apache.org> Signed-off-by: Wellington Chevreuil <wchevreuil@apache.org> (cherry picked from commit 618236d) * HBASE-25547 (addendum): Roll ExecutorType into ExecutorConfig Signed-off-by: Wellington Ramos Chevreuil <wchevreuil@apache.org> (cherry picked from commit 4c822d7)
Still marked as a WIP because the defaults have not been tuned yet.
All the code is in place, I still need to identify the low priority
thread pools and tune them to release unused threads quickly.