You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by "Andrew Purtell (JIRA)" <ji...@apache.org> on 2015/04/09 09:06:14 UTC

[jira] [Comment Edited] (HBASE-13420) RegionEnvironment.offerExecutionLatency Blocks Threads under Heavy Load

    [ https://issues.apache.org/jira/browse/HBASE-13420?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14486865#comment-14486865 ] 

Andrew Purtell edited comment on HBASE-13420 at 4/9/15 7:06 AM:
----------------------------------------------------------------

Here's what I get with a JMH benchmark on a server with 16 cores, with queue size bounded to 100 entries where supported (ConcurrentLinkedQueue does not), measuring time to offer(), 3 forks, 20 warmup iterations per fork, 10 measured iterations each fork, clearing the queue and running GC after each iteration. (Note this microbenchmark calls offer() many more times than we’d see in practice, up to 1M invocations per iteration.)

Measurements in microseconds.

||type||threads||avg us/op||p99 us/op|max us/op||
|ArrayBlockingQueue|1|0.156|0.211|6.648|
| |4|1.355|39.872|87.648|
| |8|3.125|94.464|152.832|
| |16|5.053|201.984|274.944|
| |32|9.285|406.528|488.96|
| |64|18.448|811.008|914.432|
|BoundedConcurrentLinkedQueue|1|0.148|0.202|6.816|
| |4|0.508|0.835|39.872|
| |8|0.762|1.558|40.256|
| |16|0.957|1.992|397.696|
| |32|2.047|2.808|23986.176|
| |64|4.007|4.616|55967.744|
|ConcurrentLinkedQueue|1|0.177|0.324|11.344|
| |4|0.951|3.204|252.416|
| |8|1.65|6.44|347.104|
| |16|2.742|11.424|2953.216|
| |32|5.467|11.2|39976.96|
| |64|10.797|10.896|71827.456|

Sampled timings for offer() at percentiles < 1.0 for BoundedConcurrentLinkedQueue are much better than that for ArrayBlockingQueue. 

(Bounded)ConcurrentLinkedQueue measurements have a long tail which lengthens as concurrency is increased. The worst case concurrency we can expect for coprocessors is the configured size of the RPC handler pool.


was (Author: apurtell):
Here's what I get with a JMH benchmark on a server with 16 cores, with queue size bounded to 100 entries where supported (ConcurrentLinkedQueue does not), measuring time to offer(), 3 forks, 20 warmup iterations per fork, 10 measured iterations each fork, clearing the queue and running GC after each iteration. (Note this microbenchmark calls offer() many more times than we’d see in practice, up to 1M invocations per iteration.)

Measurements in microseconds.

||type||threads||avg us/op||p99 us/op|max us/op|
|ArrayBlockingQueue|1|0.156|0.211|6.648|
| |4|1.355|39.872|87.648|
| |8|3.125|94.464|152.832|
| |16|5.053|201.984|274.944|
| |32|9.285|406.528|488.96|
| |64|18.448|811.008|914.432|
|BoundedConcurrentLinkedQueue|1|0.148|0.202|6.816|
| |4|0.508|0.835|39.872|
| |8|0.762|1.558|40.256|
| |16|0.957|1.992|397.696|
| |32|2.047|2.808|23986.176|
| |64|4.007|4.616|55967.744|
|ConcurrentLinkedQueue|1|0.177|0.324|11.344|
| |4|0.951|3.204|252.416|
| |8|1.65|6.44|347.104|
| |16|2.742|11.424|2953.216|
| |32|5.467|11.2|39976.96|
| |64|10.797|10.896|71827.456|

Sampled timings for offer() at percentiles < 1.0 for BoundedConcurrentLinkedQueue are much better than that for ArrayBlockingQueue. 

(Bounded)ConcurrentLinkedQueue measurements have a long tail which lengthens as concurrency is increased. The worst case concurrency we can expect for coprocessors is the configured size of the RPC handler pool.

> RegionEnvironment.offerExecutionLatency Blocks Threads under Heavy Load
> -----------------------------------------------------------------------
>
>                 Key: HBASE-13420
>                 URL: https://issues.apache.org/jira/browse/HBASE-13420
>             Project: HBase
>          Issue Type: Improvement
>            Reporter: John Leach
>            Assignee: Andrew Purtell
>         Attachments: HBASE-13420.patch, HBASE-13420.txt, offerExecutionLatency.tiff
>
>   Original Estimate: 3h
>  Remaining Estimate: 3h
>
> The ArrayBlockingQueue blocks threads for 20s during a performance run focusing on creating numerous small scans.  
> I see a buffer size of (100)
>     private final BlockingQueue<Long> coprocessorTimeNanos = new ArrayBlockingQueue<Long>(
>         LATENCY_BUFFER_SIZE);
> and then I see a drain coming from
>          MetricsRegionWrapperImpl with 45 second executor
>          HRegionMetricsWrapperRunable
>          RegionCoprocessorHost#getCoprocessorExecutionStatistics()   
>          RegionCoprocessorHost#getExecutionLatenciesNanos()
> Am I missing something?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)