Details

    • Type: Bug
    • Status: Patch Available
    • Priority: Minor
    • Resolution: Unresolved
    • Affects Version/s: 1.0.0, 2.0.0
    • Fix Version/s: None
    • Component/s: Client, Performance
    • Labels:
      None

      Description

      The JDK TPE creates all the threads in the pool. As a consequence, we create (by default) 256 threads even if we just need a few.

      The attached TPE create threads only if we have something in the queue.
      On a PE test with replica on, it improved the 99 latency percentile by 5%.

      Warning: there are likely some race conditions, but I'm posting it here because there is may be an implementation available somewhere we can use, or a good reason not to do that. So feedback welcome as usual.

      1. HBASE-11590.v1.patch
        51 kB
        Nicolas Liochon
      2. ExecutorServiceTest.java
        25 kB
        Nicolas Liochon
      3. UnitQueuePU.java
        3 kB
        Nicolas Liochon
      4. UnitQueueP.java
        3 kB
        Nicolas Liochon
      5. LifoThreadPoolExecutorSQP.java
        19 kB
        Nicolas Liochon
      6. tp.patch
        20 kB
        Nicolas Liochon

        Activity

        Hide
        hadoopqa Hadoop QA added a comment -
        -1 overall



        Vote Subsystem Runtime Comment
        0 reexec 0m 0s Docker mode activated.
        -1 patch 0m 7s HBASE-11590 does not apply to master. Rebase required? Wrong Branch? See https://yetus.apache.org/documentation/0.4.0/precommit-patchnames for help.



        Subsystem Report/Notes
        JIRA Issue HBASE-11590
        JIRA Patch URL https://issues.apache.org/jira/secure/attachment/12766789/HBASE-11590.v1.patch
        Console output https://builds.apache.org/job/PreCommit-HBASE-Build/7469/console
        Powered by Apache Yetus 0.4.0 http://yetus.apache.org

        This message was automatically generated.

        Show
        hadoopqa Hadoop QA added a comment - -1 overall Vote Subsystem Runtime Comment 0 reexec 0m 0s Docker mode activated. -1 patch 0m 7s HBASE-11590 does not apply to master. Rebase required? Wrong Branch? See https://yetus.apache.org/documentation/0.4.0/precommit-patchnames for help. Subsystem Report/Notes JIRA Issue HBASE-11590 JIRA Patch URL https://issues.apache.org/jira/secure/attachment/12766789/HBASE-11590.v1.patch Console output https://builds.apache.org/job/PreCommit-HBASE-Build/7469/console Powered by Apache Yetus 0.4.0 http://yetus.apache.org This message was automatically generated.
        Hide
        hadoopqa Hadoop QA added a comment -

        -1 overall. Here are the results of testing the latest attachment
        http://issues.apache.org/jira/secure/attachment/12766789/HBASE-11590.v1.patch
        against master branch at commit e7defd7d9a76f44e3089db3fe522fe400fe6dcd7.
        ATTACHMENT ID: 12766789

        +1 @author. The patch does not contain any @author tags.

        +1 tests included. The patch appears to include 18 new or modified tests.

        +1 hadoop versions. The patch compiles with all supported hadoop versions (2.4.0 2.4.1 2.5.0 2.5.1 2.5.2 2.6.0 2.6.1 2.7.0 2.7.1)

        +1 javac. The applied patch does not increase the total number of javac compiler warnings.

        +1 protoc. The applied patch does not increase the total number of protoc compiler warnings.

        +1 javadoc. The javadoc tool did not generate any warning messages.

        -1 checkstyle. The applied patch generated 1754 checkstyle errors (more than the master's current 1747 errors).

        +1 findbugs. The patch does not introduce any new Findbugs (version 2.0.3) warnings.

        +1 release audit. The applied patch does not increase the total number of release audit warnings.

        -1 lineLengths. The patch introduces the following lines longer than 100:
        + ExecutorService pool = ProcedureCoordinator.defaultPool(COORDINATOR_NODE_NAME, POOL_SIZE, KEEP_ALIVE);
        + ExecutorService pool = ProcedureCoordinator.defaultPool(COORDINATOR_NODE_NAME, POOL_SIZE, KEEP_ALIVE);

        +1 site. The mvn post-site goal succeeds with this patch.

        -1 core tests. The patch failed these unit tests:

        Test results: https://builds.apache.org/job/PreCommit-HBASE-Build/16028//testReport/
        Release Findbugs (version 2.0.3) warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/16028//artifact/patchprocess/newFindbugsWarnings.html
        Checkstyle Errors: https://builds.apache.org/job/PreCommit-HBASE-Build/16028//artifact/patchprocess/checkstyle-aggregate.html

        Console output: https://builds.apache.org/job/PreCommit-HBASE-Build/16028//console

        This message is automatically generated.

        Show
        hadoopqa Hadoop QA added a comment - -1 overall . Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12766789/HBASE-11590.v1.patch against master branch at commit e7defd7d9a76f44e3089db3fe522fe400fe6dcd7. ATTACHMENT ID: 12766789 +1 @author . The patch does not contain any @author tags. +1 tests included . The patch appears to include 18 new or modified tests. +1 hadoop versions . The patch compiles with all supported hadoop versions (2.4.0 2.4.1 2.5.0 2.5.1 2.5.2 2.6.0 2.6.1 2.7.0 2.7.1) +1 javac . The applied patch does not increase the total number of javac compiler warnings. +1 protoc . The applied patch does not increase the total number of protoc compiler warnings. +1 javadoc . The javadoc tool did not generate any warning messages. -1 checkstyle . The applied patch generated 1754 checkstyle errors (more than the master's current 1747 errors). +1 findbugs . The patch does not introduce any new Findbugs (version 2.0.3) warnings. +1 release audit . The applied patch does not increase the total number of release audit warnings. -1 lineLengths . The patch introduces the following lines longer than 100: + ExecutorService pool = ProcedureCoordinator.defaultPool(COORDINATOR_NODE_NAME, POOL_SIZE, KEEP_ALIVE); + ExecutorService pool = ProcedureCoordinator.defaultPool(COORDINATOR_NODE_NAME, POOL_SIZE, KEEP_ALIVE); +1 site . The mvn post-site goal succeeds with this patch. -1 core tests . The patch failed these unit tests: Test results: https://builds.apache.org/job/PreCommit-HBASE-Build/16028//testReport/ Release Findbugs (version 2.0.3) warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/16028//artifact/patchprocess/newFindbugsWarnings.html Checkstyle Errors: https://builds.apache.org/job/PreCommit-HBASE-Build/16028//artifact/patchprocess/checkstyle-aggregate.html Console output: https://builds.apache.org/job/PreCommit-HBASE-Build/16028//console This message is automatically generated.
        Hide
        nkeywal Nicolas Liochon added a comment -

        The patch compiles locally, but it's all I checked.
        client side: use the ForkJoin instead of ThreadPoolExecutor; remove the monitoring linked to ThreadPoolExecutor
        server side: when possible; use the interface (ExecutorService) instead of the implementation (ThreadPoolExecutor)

        Show
        nkeywal Nicolas Liochon added a comment - The patch compiles locally, but it's all I checked. client side: use the ForkJoin instead of ThreadPoolExecutor; remove the monitoring linked to ThreadPoolExecutor server side: when possible; use the interface (ExecutorService) instead of the implementation (ThreadPoolExecutor)
        Hide
        nkeywal Nicolas Liochon added a comment -

        The issue is that the ThreadPoolExecutor leaked all over the place, often for monitoring reasons.
        All lot of code depends on ThreadPoolExecutor rather than the ExecutorService...

        For example, see

        /**
         * This class will coalesce increments from a thift server if
         * hbase.regionserver.thrift.coalesceIncrement is set to true. Turning this
         * config to true will cause the thrift server to queue increments into an
         * instance of this class. The thread pool associated with this class will drain
         * the coalesced increments as the thread is able. This can cause data loss if the
         * thrift server dies or is shut down before everything in the queue is drained.
         *
         */
        public class IncrementCoalescer implements IncrementCoalescerMBean {
        // snip
          // MBean get/set methods
          public int getQueueSize() {
            return pool.getQueue().size();
          }
          public int getMaxQueueSize() {
            return this.maxQueueSize;
          }
          public void setMaxQueueSize(int newSize) {
            this.maxQueueSize = newSize;
          }
        
          public long getPoolCompletedTaskCount() {
            return pool.getCompletedTaskCount();
          }
          public long getPoolTaskCount() {
            return pool.getTaskCount();
          }
          public int getPoolLargestPoolSize() {
            return pool.getLargestPoolSize();
          }
          public int getCorePoolSize() {
            return pool.getCorePoolSize();
          }
          public void setCorePoolSize(int newCoreSize) {
            pool.setCorePoolSize(newCoreSize);
          }
          public int getMaxPoolSize() {
            return pool.getMaximumPoolSize();
          }
          public void setMaxPoolSize(int newMaxSize) {
            pool.setMaximumPoolSize(newMaxSize);
          }
        

        I'm going to limit this patch to the easy/client stuff...

        Show
        nkeywal Nicolas Liochon added a comment - The issue is that the ThreadPoolExecutor leaked all over the place, often for monitoring reasons. All lot of code depends on ThreadPoolExecutor rather than the ExecutorService... For example, see /** * This class will coalesce increments from a thift server if * hbase.regionserver.thrift.coalesceIncrement is set to true . Turning this * config to true will cause the thrift server to queue increments into an * instance of this class. The thread pool associated with this class will drain * the coalesced increments as the thread is able. This can cause data loss if the * thrift server dies or is shut down before everything in the queue is drained. * */ public class IncrementCoalescer implements IncrementCoalescerMBean { // snip // MBean get/set methods public int getQueueSize() { return pool.getQueue().size(); } public int getMaxQueueSize() { return this .maxQueueSize; } public void setMaxQueueSize( int newSize) { this .maxQueueSize = newSize; } public long getPoolCompletedTaskCount() { return pool.getCompletedTaskCount(); } public long getPoolTaskCount() { return pool.getTaskCount(); } public int getPoolLargestPoolSize() { return pool.getLargestPoolSize(); } public int getCorePoolSize() { return pool.getCorePoolSize(); } public void setCorePoolSize( int newCoreSize) { pool.setCorePoolSize(newCoreSize); } public int getMaxPoolSize() { return pool.getMaximumPoolSize(); } public void setMaxPoolSize( int newMaxSize) { pool.setMaximumPoolSize(newMaxSize); } I'm going to limit this patch to the easy/client stuff...
        Hide
        nkeywal Nicolas Liochon added a comment -

        > maybe just because it is more parsimonious in its thread use?
        That's the magic part: even of there is a single thread in the pool it's faster than the others. I didn't check if it consumes more CPU or not however.

        I will do the patch to use ForkJoin soon (hopefully today, if not next week).

        Show
        nkeywal Nicolas Liochon added a comment - > maybe just because it is more parsimonious in its thread use? That's the magic part: even of there is a single thread in the pool it's faster than the others. I didn't check if it consumes more CPU or not however. I will do the patch to use ForkJoin soon (hopefully today, if not next week).
        Hide
        stack stack added a comment -

        It is however incredible to see how faster/better it is compared to the other pools.

        It is in a different place altogether compared to the other two... in terms of timing and threads used (maybe just because it is more parsimonious in its thread use)? Man.

        Thats a very nice compare there Nicolas Liochon Sure on the FJP. Thats a substantial enough change in perf. to make the change.

        Show
        stack stack added a comment - It is however incredible to see how faster/better it is compared to the other pools. It is in a different place altogether compared to the other two... in terms of timing and threads used (maybe just because it is more parsimonious in its thread use)? Man. Thats a very nice compare there Nicolas Liochon Sure on the FJP. Thats a substantial enough change in perf. to make the change.
        Hide
        nkeywal Nicolas Liochon added a comment -

        Hey Stack

        Attached some tests comparing ThreadPoolExecutor (the one we use currently), ForkJoinPool (available in jdk1.7+) and LifoThreadPoolExecutorSQP (the one mentionned in the stackoverflow discussion) .

        • the critical use case is:
          1) do a table.batch(puts) that needs a lot of threads
          2) then do a loop { table.get(get) }

          , this needs a single thread but each call may use any of the threads in the pool, resetting the keepalive timeout => they may never expire.
          ThreadPoolExecutor is actually worse it tries to create a thread even if there are already enough threads available.

        See the code for the details, but here is the interesting case with a thread pools of 1000 threads while we need only 1 thread.

        • ForkJoinPool maxThread=1000, immediateGet=true, LOOP=2000000
        • ForkJoinPool total=68942ms
        • ForkJoinPool step1=68657ms
        • ForkJoinPool step2=284ms
        • ForkJoinPool threads: 6, 1006, 456, 6 <=== we have 456 threads instead of the ideal 7
        • ThreadPoolExecutor maxThread=1000, immediateGet=true, LOOP=2000000
        • ThreadPoolExecutor total=107449ms <=== very slow
        • ThreadPoolExecutor step1=107145ms
        • ThreadPoolExecutor step2=304ms
        • ThreadPoolExecutor threads: 6, 1006, 889, 6 <== keeps nearly all the threads -
        • LifoThreadPoolExecutorSQP maxThread=1000, immediateGet=true, LOOP=2000000
        • LifoThreadPoolExecutorSQP total=4805ms <================ quite fast
        • LifoThreadPoolExecutorSQP step1=4803ms
        • LifoThreadPoolExecutorSQP step2=1ms
        • LifoThreadPoolExecutorSQP threads: 6, 248, 8, 6 <====================== removes the threads quickly

        You may want to rerun the tests to see if you reproduce them. I included my results in the code.

        • The root issue is that we need a LIFO poll/lock but it does not exists.
        • LifoThreadPoolExecutorSQP solves this with a LIFO queues for the threads waiting for work. But it
          comes with a LGPL license, and the code is not trivial. A bug there could be difficult to find. It
          is however incredible to see how faster/better it is compared to the other pools.
        • ForkJoinPool is better then TPE. It's not as good as LifoThreadPoolExecutorSQP, but it's much
          closer to what we need. It's available in the JDK 1.7 it looks like a safe bet for HBase 1.+
          ForkJoinPool: threads are created only if there are waiting tasks. They expire after 2seconds (it's
          hardcoded in the jdk code). They are not LIFO, and the task allocation is not as fast as the one in LifoThreadPoolExecutorSQP.

        => Proposition: Let's migrate to ForkJoinPool. If someone has time to try LifoThreadPoolExecutorSQP it can be interesting in the future (if the license can be changed)...

        Show
        nkeywal Nicolas Liochon added a comment - Hey Stack Attached some tests comparing ThreadPoolExecutor (the one we use currently), ForkJoinPool (available in jdk1.7+) and LifoThreadPoolExecutorSQP (the one mentionned in the stackoverflow discussion) . the critical use case is: 1) do a table.batch(puts) that needs a lot of threads 2) then do a loop { table.get(get) } , this needs a single thread but each call may use any of the threads in the pool, resetting the keepalive timeout => they may never expire. ThreadPoolExecutor is actually worse it tries to create a thread even if there are already enough threads available. See the code for the details, but here is the interesting case with a thread pools of 1000 threads while we need only 1 thread. ForkJoinPool maxThread=1000, immediateGet=true, LOOP=2000000 ForkJoinPool total=68942ms ForkJoinPool step1=68657ms ForkJoinPool step2=284ms ForkJoinPool threads: 6, 1006, 456, 6 <=== we have 456 threads instead of the ideal 7 ThreadPoolExecutor maxThread=1000, immediateGet=true, LOOP=2000000 ThreadPoolExecutor total=107449ms <=== very slow ThreadPoolExecutor step1=107145ms ThreadPoolExecutor step2=304ms ThreadPoolExecutor threads: 6, 1006, 889, 6 <== keeps nearly all the threads - LifoThreadPoolExecutorSQP maxThread=1000, immediateGet=true, LOOP=2000000 LifoThreadPoolExecutorSQP total=4805ms <================ quite fast LifoThreadPoolExecutorSQP step1=4803ms LifoThreadPoolExecutorSQP step2=1ms LifoThreadPoolExecutorSQP threads: 6, 248, 8, 6 <====================== removes the threads quickly You may want to rerun the tests to see if you reproduce them. I included my results in the code. The root issue is that we need a LIFO poll/lock but it does not exists. LifoThreadPoolExecutorSQP solves this with a LIFO queues for the threads waiting for work. But it comes with a LGPL license, and the code is not trivial. A bug there could be difficult to find. It is however incredible to see how faster/better it is compared to the other pools. ForkJoinPool is better then TPE. It's not as good as LifoThreadPoolExecutorSQP, but it's much closer to what we need. It's available in the JDK 1.7 it looks like a safe bet for HBase 1.+ ForkJoinPool: threads are created only if there are waiting tasks. They expire after 2seconds (it's hardcoded in the jdk code). They are not LIFO, and the task allocation is not as fast as the one in LifoThreadPoolExecutorSQP. => Proposition: Let's migrate to ForkJoinPool. If someone has time to try LifoThreadPoolExecutorSQP it can be interesting in the future (if the license can be changed)...
        Hide
        stack stack added a comment -

        If we cut down the timeout, it's more or less equivalent of not having a thread pool at all.

        Well, if a timeout of 1 or 10 seconds, the pool would be in place when we need it... in times of read/write.

        No hurry Nicolas Liochon On the create of one thread too many, I'd not be too worried given we seem to currently create 255 threads too many (smile).

        Show
        stack stack added a comment - If we cut down the timeout, it's more or less equivalent of not having a thread pool at all. Well, if a timeout of 1 or 10 seconds, the pool would be in place when we need it... in times of read/write. No hurry Nicolas Liochon On the create of one thread too many, I'd not be too worried given we seem to currently create 255 threads too many (smile).
        Hide
        nkeywal Nicolas Liochon added a comment -

        If we cut down the timeout, it's more or less equivalent of not having a thread pool at all.
        One of the things I don't like in many solutions (the TPE I wrote myself included) is that we have a race condition: we may create a thread even if it's not needed.
        I'm off for 3 days, but I will try to find a reasonable solution next week.

        Show
        nkeywal Nicolas Liochon added a comment - If we cut down the timeout, it's more or less equivalent of not having a thread pool at all. One of the things I don't like in many solutions (the TPE I wrote myself included) is that we have a race condition: we may create a thread even if it's not needed. I'm off for 3 days, but I will try to find a reasonable solution next week.
        Hide
        stack stack added a comment -

        Should we down the keepalive timeout so it is seconds only? We have allowCoreThreadTimeOut(true); Core threads would run up to the max but could also go down to zero as is noted in http://stackoverflow.com/questions/19528304/how-to-get-the-threadpoolexecutor-to-increase-threads-to-max-before-queueing/19528305#19528305 Or the suggestion by Ralph H at answered Oct 23 '13 at 10:15 in the link looks simple (after executing the current reset the core thread size if not enough for current requests). There is a new answer on the end... with a GPL soln.

        Show
        stack stack added a comment - Should we down the keepalive timeout so it is seconds only? We have allowCoreThreadTimeOut(true); Core threads would run up to the max but could also go down to zero as is noted in http://stackoverflow.com/questions/19528304/how-to-get-the-threadpoolexecutor-to-increase-threads-to-max-before-queueing/19528305#19528305 Or the suggestion by Ralph H at answered Oct 23 '13 at 10:15 in the link looks simple (after executing the current reset the core thread size if not enough for current requests). There is a new answer on the end... with a GPL soln.
        Show
        nkeywal Nicolas Liochon added a comment - No... There is actually an alternative solution to this problem here: http://stackoverflow.com/questions/19528304/how-to-get-the-threadpoolexecutor-to-increase-threads-to-max-before-queueing/19528305#19528305
        Hide
        lhofhansl Lars Hofhansl added a comment -

        Is that not something we can control in ThreadPoolExecutor with corePoolSize and maximumPoolSize?

        Show
        lhofhansl Lars Hofhansl added a comment - Is that not something we can control in ThreadPoolExecutor with corePoolSize and maximumPoolSize?
        Hide
        hadoopqa Hadoop QA added a comment -

        -1 overall. Here are the results of testing the latest attachment
        http://issues.apache.org/jira/secure/attachment/12657819/tp.patch
        against trunk revision .
        ATTACHMENT ID: 12657819

        +1 @author. The patch does not contain any @author tags.

        +1 tests included. The patch appears to include 5 new or modified tests.

        +1 javac. The applied patch does not increase the total number of javac compiler warnings.

        +1 javac. The applied patch does not increase the total number of javac compiler warnings.

        +1 javadoc. The javadoc tool did not generate any warning messages.

        +1 findbugs. The patch does not introduce any new Findbugs (version 2.0.3) warnings.

        +1 release audit. The applied patch does not increase the total number of release audit warnings.

        -1 lineLengths. The patch introduces the following lines longer than 100:
        + this.batchPool = new ExecutorServiceWithQueue(Threads.newDaemonThreadFactory(toString() + "shared"), maxThreads, keepAliveTime * 1000,
        + maxThreads * conf.getInt(HConstants.HBASE_CLIENT_MAX_TOTAL_TASKS, HConstants.DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS))) ;
        + private final ConcurrentSkipListSet<Thread> availableThreads = new ConcurrentSkipListSet<Thread>(THREAD_COMPARAROR);
        + public ExecutorServiceWithQueue(ThreadFactory threadFactory, int maxThread, long threadTimeout, BlockingQueue<Runnable> tasks) {
        + public T get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException {
        + public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks) throws InterruptedException {
        + public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit) throws InterruptedException {
        + public <T> T invokeAny(Collection<? extends Callable<T>> tasks) throws InterruptedException, ExecutionException {
        + public <T> T invokeAny(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException {
        + while ((!isShutdown || !tasks.isEmpty()) && (EnvironmentEdgeManager.currentTimeMillis() < nextTimeout)) {

        +1 site. The mvn site goal succeeds with this patch.

        -1 core tests. The patch failed these unit tests:
        org.apache.hadoop.hbase.io.hfile.TestCacheConfig

        Test results: https://builds.apache.org/job/PreCommit-HBASE-Build/10187//testReport/
        Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/10187//artifact/patchprocess/newPatchFindbugsWarningshbase-prefix-tree.html
        Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/10187//artifact/patchprocess/newPatchFindbugsWarningshbase-examples.html
        Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/10187//artifact/patchprocess/newPatchFindbugsWarningshbase-common.html
        Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/10187//artifact/patchprocess/newPatchFindbugsWarningshbase-hadoop-compat.html
        Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/10187//artifact/patchprocess/newPatchFindbugsWarningshbase-client.html
        Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/10187//artifact/patchprocess/newPatchFindbugsWarningshbase-thrift.html
        Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/10187//artifact/patchprocess/newPatchFindbugsWarningshbase-protocol.html
        Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/10187//artifact/patchprocess/newPatchFindbugsWarningshbase-server.html
        Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/10187//artifact/patchprocess/newPatchFindbugsWarningshbase-hadoop2-compat.html
        Console output: https://builds.apache.org/job/PreCommit-HBASE-Build/10187//console

        This message is automatically generated.

        Show
        hadoopqa Hadoop QA added a comment - -1 overall . Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12657819/tp.patch against trunk revision . ATTACHMENT ID: 12657819 +1 @author . The patch does not contain any @author tags. +1 tests included . The patch appears to include 5 new or modified tests. +1 javac . The applied patch does not increase the total number of javac compiler warnings. +1 javac . The applied patch does not increase the total number of javac compiler warnings. +1 javadoc . The javadoc tool did not generate any warning messages. +1 findbugs . The patch does not introduce any new Findbugs (version 2.0.3) warnings. +1 release audit . The applied patch does not increase the total number of release audit warnings. -1 lineLengths . The patch introduces the following lines longer than 100: + this.batchPool = new ExecutorServiceWithQueue(Threads.newDaemonThreadFactory(toString() + " shared "), maxThreads, keepAliveTime * 1000, + maxThreads * conf.getInt(HConstants.HBASE_CLIENT_MAX_TOTAL_TASKS, HConstants.DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS))) ; + private final ConcurrentSkipListSet<Thread> availableThreads = new ConcurrentSkipListSet<Thread>(THREAD_COMPARAROR); + public ExecutorServiceWithQueue(ThreadFactory threadFactory, int maxThread, long threadTimeout, BlockingQueue<Runnable> tasks) { + public T get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { + public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks) throws InterruptedException { + public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit) throws InterruptedException { + public <T> T invokeAny(Collection<? extends Callable<T>> tasks) throws InterruptedException, ExecutionException { + public <T> T invokeAny(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { + while ((!isShutdown || !tasks.isEmpty()) && (EnvironmentEdgeManager.currentTimeMillis() < nextTimeout)) { +1 site . The mvn site goal succeeds with this patch. -1 core tests . The patch failed these unit tests: org.apache.hadoop.hbase.io.hfile.TestCacheConfig Test results: https://builds.apache.org/job/PreCommit-HBASE-Build/10187//testReport/ Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/10187//artifact/patchprocess/newPatchFindbugsWarningshbase-prefix-tree.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/10187//artifact/patchprocess/newPatchFindbugsWarningshbase-examples.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/10187//artifact/patchprocess/newPatchFindbugsWarningshbase-common.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/10187//artifact/patchprocess/newPatchFindbugsWarningshbase-hadoop-compat.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/10187//artifact/patchprocess/newPatchFindbugsWarningshbase-client.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/10187//artifact/patchprocess/newPatchFindbugsWarningshbase-thrift.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/10187//artifact/patchprocess/newPatchFindbugsWarningshbase-protocol.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/10187//artifact/patchprocess/newPatchFindbugsWarningshbase-server.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/10187//artifact/patchprocess/newPatchFindbugsWarningshbase-hadoop2-compat.html Console output: https://builds.apache.org/job/PreCommit-HBASE-Build/10187//console This message is automatically generated.

          People

          • Assignee:
            nkeywal Nicolas Liochon
            Reporter:
            nkeywal Nicolas Liochon
          • Votes:
            0 Vote for this issue
            Watchers:
            7 Start watching this issue

            Dates

            • Created:
              Updated:

              Development