You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@activemq.apache.org by bgutjahr <gi...@git.apache.org> on 2017/03/28 14:43:11 UTC

[GitHub] activemq-artemis pull request #1144: ARTEMIS-1078: Improved thread pool

GitHub user bgutjahr opened a pull request:

    https://github.com/apache/activemq-artemis/pull/1144

    ARTEMIS-1078: Improved thread pool

    The improved thread pool make sure that only those threads that are currently waiting in queue poll or take calls are considered idle and ready to run a queued task. If not enough threads are idle, new threads will be created - up to the maximum number.
    
    The case where this affected is was in HA backup failover, when client reconnect tasks went into an endless loop due to an unlimited number of reconnect attempts, preventing the task to handle the failover from being executed due to the missed thread creation. This fix will help as long as the thread pool's maximum number of threads has not been reached, or some of the currently running threads will not hang in such endless loops and can run the failover task without significant delay. I would suggest to recheck the looping reconnect code. Maybe it can be rewritten to submit a new task to try the next reconnect, such that other queued tasks could also be given a chance to be executed.
    
    I created the fix on the 1.x branch. I guess this will also affect the master branch.


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/bgutjahr/activemq-artemis improved-thread-pool

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/activemq-artemis/pull/1144.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #1144
    
----
commit 07edd3968f668aacf44f8f984a2c35d98552af99
Author: Bernd Gutjahr <be...@hpe.com>
Date:   2017-03-28T13:54:58Z

    ARTEMIS-1078 Improved thread pool to consider only threads waiting for the queue to get new tasks as idle.
    
    The thread pool maintained a counter of active threads, but that counter was increased too late in the beforeExecute method. Submitting a task created a new thread. If now a second task was submitter before the new thread had started to execute it's task, the second task was queued without creating a 2nd thread. So the second task was only executed after the first task had been completed - even if the thread pool's maximum number of thread had not been reached.
    
    This fix now only count those threads that are currently waiting in the queue's poll or take methods as idle threads, and creates new threads unless there are enough idle threads to pick up all queued tasks.
    
    	modified:   artemis-commons/src/main/java/org/apache/activemq/artemis/utils/ActiveMQThreadPoolExecutor.java

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] activemq-artemis pull request #1144: ARTEMIS-1078: Improved thread pool

Posted by franz1981 <gi...@git.apache.org>.
Github user franz1981 commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1144#discussion_r108658925
  
    --- Diff: artemis-commons/src/main/java/org/apache/activemq/artemis/utils/ActiveMQThreadPoolExecutor.java ---
    @@ -46,32 +45,70 @@
        private static class ThreadPoolQueue extends LinkedBlockingQueue<Runnable> {
     
           private ActiveMQThreadPoolExecutor executor = null;
    +      private int idleThreads = 0;
     
           public void setExecutor(ActiveMQThreadPoolExecutor executor) {
              this.executor = executor;
           }
     
           @Override
    -      public boolean offer(Runnable runnable) {
    +      public synchronized boolean offer(Runnable runnable) {
    --- End diff --
    
    @bgutjahr Great, np to wait, after the changes I'll try to built a little benchmark on it to look if there will be any improvements/regressions that in a full broker test couldn't be seen :+1: 
    This component is used pretty everywhere so I think it could worth to be sure of it


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] activemq-artemis pull request #1144: ARTEMIS-1078: Improved thread pool

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/activemq-artemis/pull/1144


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] activemq-artemis pull request #1144: ARTEMIS-1078: Improved thread pool

Posted by franz1981 <gi...@git.apache.org>.
Github user franz1981 commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1144#discussion_r108615515
  
    --- Diff: artemis-commons/src/main/java/org/apache/activemq/artemis/utils/ActiveMQThreadPoolExecutor.java ---
    @@ -46,32 +45,70 @@
        private static class ThreadPoolQueue extends LinkedBlockingQueue<Runnable> {
     
           private ActiveMQThreadPoolExecutor executor = null;
    +      private int idleThreads = 0;
     
           public void setExecutor(ActiveMQThreadPoolExecutor executor) {
              this.executor = executor;
           }
     
           @Override
    -      public boolean offer(Runnable runnable) {
    +      public synchronized boolean offer(Runnable runnable) {
    --- End diff --
    
    @bgutjahr @clebertsuconic Today I'll take a look at the code and will do some test on it :)
    @bgutjahr Do you've tested it against the original with some benchmark tool like [JMH](http://openjdk.java.net/projects/code-tools/jmh/)?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] activemq-artemis pull request #1144: ARTEMIS-1078: Improved thread pool

Posted by clebertsuconic <gi...@git.apache.org>.
Github user clebertsuconic commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1144#discussion_r108526825
  
    --- Diff: artemis-commons/src/main/java/org/apache/activemq/artemis/utils/ActiveMQThreadPoolExecutor.java ---
    @@ -46,32 +45,70 @@
        private static class ThreadPoolQueue extends LinkedBlockingQueue<Runnable> {
     
           private ActiveMQThreadPoolExecutor executor = null;
    +      private int idleThreads = 0;
     
           public void setExecutor(ActiveMQThreadPoolExecutor executor) {
              this.executor = executor;
           }
     
           @Override
    -      public boolean offer(Runnable runnable) {
    +      public synchronized boolean offer(Runnable runnable) {
    --- End diff --
    
    @franz1981 I feel like you're the man to look into this...
    
    can you look into this one?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] activemq-artemis pull request #1144: ARTEMIS-1078: Improved thread pool

Posted by bgutjahr <gi...@git.apache.org>.
Github user bgutjahr commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1144#discussion_r108618664
  
    --- Diff: artemis-commons/src/main/java/org/apache/activemq/artemis/utils/ActiveMQThreadPoolExecutor.java ---
    @@ -46,32 +45,70 @@
        private static class ThreadPoolQueue extends LinkedBlockingQueue<Runnable> {
     
           private ActiveMQThreadPoolExecutor executor = null;
    +      private int idleThreads = 0;
     
           public void setExecutor(ActiveMQThreadPoolExecutor executor) {
              this.executor = executor;
           }
     
           @Override
    -      public boolean offer(Runnable runnable) {
    +      public synchronized boolean offer(Runnable runnable) {
    --- End diff --
    
    I put the code into our product and tested the specific issue that a HA backup server is occasionally not taking over. So far, our product tests have not found new issues and I couldn't reproduce the HA backup failover issue. I did not run any performance tests.
    BTW: I have rechecked the code and found a solution without any synchronization, just using an AtomicInteger. I'm currently testing my new changed and want to also discuss my changes with a collegue. I suggest to wait reviewing and testing my changes until I committed these changes later today.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] activemq-artemis issue #1144: ARTEMIS-1078: Improved thread pool

Posted by franz1981 <gi...@git.apache.org>.
Github user franz1981 commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1144
  
    @bgutjahr These are the preliminary results from the benchmark [here](https://github.com/franz1981/activemq-artemis/tree/jmh_improved_thread_pool), see the [ThreadPoolBenchmark](https://github.com/franz1981/activemq-artemis/blob/e4b815281dc14c540821bb672e2bd0e5592f5133/tests/extra-tests/src/test/java/org/apache/activemq/artemis/tests/extras/benchmarks/pools/ThreadPoolBenchmark.java) if you want to reproduce the test or verify if is fair enough.
    The type **new** is your version of the pool, **old** is the original while **baseline** is the JDK fixed thread executor service.
    The **execute** is the throughput of the producer thread to submit a task to the pool, while
    **thread_1**, **thread_2** and **thread_3** are the throughput to execute the tasks on the given pooled thread (3 in total).
    As is visible by the results the task submission is slower than the original while the pooled threads are faster to consume, altought with an higher variance: maybe is dependent by a synchronized operation or any queuing effect.
    ```
    Benchmark                                                   (type)   Mode  Cnt          Score          Error   Units
    ThreadPoolBenchmark.tpt:execute                           baseline  thrpt    5    4095676.139 �   248391.951   ops/s
    ThreadPoolBenchmark.tpt:thread_1                          baseline  thrpt    5     436573.539 �    78582.478   ops/s
    ThreadPoolBenchmark.tpt:thread_2                          baseline  thrpt    5     441567.641 �   111930.503   ops/s
    ThreadPoolBenchmark.tpt:thread_3                          baseline  thrpt    5     420329.280 �    78979.112   ops/s
    
    ThreadPoolBenchmark.tpt:execute                                old  thrpt    5    5273631.677 �  2606806.562   ops/s
    ThreadPoolBenchmark.tpt:thread_1                               old  thrpt    5     622163.951 �   422993.497   ops/s
    ThreadPoolBenchmark.tpt:thread_2                               old  thrpt    5     582100.488 �   193436.998   ops/s
    ThreadPoolBenchmark.tpt:thread_3                               old  thrpt    5     640375.952 �   190881.796   ops/s
    
    ThreadPoolBenchmark.tpt:execute                                new  thrpt    5    3628826.588 �  2837645.169   ops/s
    ThreadPoolBenchmark.tpt:thread_1                               new  thrpt    5    1057520.974 �  3212217.630   ops/s
    ThreadPoolBenchmark.tpt:thread_2                               new  thrpt    5    1028641.600 �  2629406.160   ops/s
    ThreadPoolBenchmark.tpt:thread_3                               new  thrpt    5    1026665.278 �  2753389.072   ops/s
    ```
    In the next days I'll take a look to the code too and I'll build a concurrency sanity test to find out if everything is working as expected.
    Anyway god job, for me is more balanced than the original version :+1: 



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] activemq-artemis issue #1144: ARTEMIS-1078: Improved thread pool

Posted by franz1981 <gi...@git.apache.org>.
Github user franz1981 commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1144
  
    @bgutjahr Great! I'll build up a simple bench to find how it works then, thanks :+1: 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] activemq-artemis pull request #1144: ARTEMIS-1078: Improved thread pool

Posted by bgutjahr <gi...@git.apache.org>.
Github user bgutjahr commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1144#discussion_r108594658
  
    --- Diff: artemis-commons/src/main/java/org/apache/activemq/artemis/utils/ActiveMQThreadPoolExecutor.java ---
    @@ -46,32 +45,70 @@
        private static class ThreadPoolQueue extends LinkedBlockingQueue<Runnable> {
     
           private ActiveMQThreadPoolExecutor executor = null;
    +      private int idleThreads = 0;
     
           public void setExecutor(ActiveMQThreadPoolExecutor executor) {
              this.executor = executor;
           }
     
           @Override
    -      public boolean offer(Runnable runnable) {
    +      public synchronized boolean offer(Runnable runnable) {
    --- End diff --
    
    the synchronized is necessary to make sure the idleThreads doesn't change, which is done it the related synchronized blocks in poll and take. I also needed to make sure that no 2 offers occur in parallel. Otherwise, each offer could see one remaining idle thread and queue it's task. But only 1 task has to be be queued, for the other task a new thread has to be created. I might be able to reduce the synchronization time, but I can't avoid it. I don't think that the additional synchronization will be recognizable compared to the runtime of the tasks to be executed. The underlying LinkedBlockingQueue also uses a ReentrantLock (which I could't access). So I'm not sure if trying to optimize on that level will be recognizable and would be worth the effort. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] activemq-artemis issue #1144: ARTEMIS-1078: Improved thread pool

Posted by dudaerich <gi...@git.apache.org>.
Github user dudaerich commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1144
  
    +1 for this improvement.
    
    I've reported issue with ThreadPoolExecutor in [ARTEMIS-968](https://issues.apache.org/jira/browse/ARTEMIS-968). Part of the JIRA is [test](https://github.com/dudaerich/activemq-artemis/blob/ActiveMQThreadPoolExecutorTest/tests/unit-tests/src/test/java/org/apache/activemq/artemis/tests/unit/util/ActiveMQThreadPoolExecutorTest.java) which reproduces the issue in old implementation. Could you add the test to this PR? I tried to execute it and it passed.
    
    Thanks


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] activemq-artemis issue #1144: ARTEMIS-1078: Improved thread pool

Posted by bgutjahr <gi...@git.apache.org>.
Github user bgutjahr commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1144
  
    I replaced my code with a new implemented. I could not get rid of using synchronization, since the timeout-handle block in poll has to be synchronized with offer. But I made a couple of improvements:
    1. I used a private lock object to synchronize on
    2. I added comments to explain how the code works
    3. I fixed a race condition that had been caused by using queue.size(), which changed within super.poll/take calls by now thread-safely maintaining the delta beween queue tasks and idle threads.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] activemq-artemis pull request #1144: ARTEMIS-1078: Improved thread pool

Posted by clebertsuconic <gi...@git.apache.org>.
Github user clebertsuconic commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1144#discussion_r108526952
  
    --- Diff: artemis-commons/src/main/java/org/apache/activemq/artemis/utils/ActiveMQThreadPoolExecutor.java ---
    @@ -46,32 +45,70 @@
        private static class ThreadPoolQueue extends LinkedBlockingQueue<Runnable> {
     
           private ActiveMQThreadPoolExecutor executor = null;
    +      private int idleThreads = 0;
     
           public void setExecutor(ActiveMQThreadPoolExecutor executor) {
              this.executor = executor;
           }
     
           @Override
    -      public boolean offer(Runnable runnable) {
    +      public synchronized boolean offer(Runnable runnable) {
    --- End diff --
    
    @bgutjahr are you sure the synchronized on offer won't cause contentions?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---