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

Yu Li commented on HBASE-16698:
-------------------------------

Thanks for the question [~enis], the above numbers are against branch-1 codes, 
not master.

For master branch, the issue description only talks in theory and no real 
testing done. Let me add the testing and will upload data later.

It's confirmed we won't pull this in for branch-1.3 until 1.3.0 got released 
and 1.3.1 comes out. For branch-1.2, [~stack] could you help confirm sir? 
Thanks.

Will resolve this JIRA if perf data against master branch is good as expected 
and decision for branch-1.2 is made. :-)

> Performance issue: handlers stuck waiting for CountDownLatch inside 
> WALKey#getWriteEntry under high writing workload
> --------------------------------------------------------------------------------------------------------------------
>
>                 Key: HBASE-16698
>                 URL: https://issues.apache.org/jira/browse/HBASE-16698
>             Project: HBase
>          Issue Type: Improvement
>          Components: Performance
>    Affects Versions: 1.2.3
>            Reporter: Yu Li
>            Assignee: Yu Li
>             Fix For: 2.0.0
>
>         Attachments: HBASE-16698.branch-1.patch, 
> HBASE-16698.branch-1.v2.patch, HBASE-16698.branch-1.v2.patch, 
> HBASE-16698.patch, HBASE-16698.v2.patch, hadoop0495.et2.jstack
>
>
> As titled, on our production environment we observed 98 out of 128 handlers 
> get stuck waiting for the CountDownLatch {{seqNumAssignedLatch}} inside 
> {{WALKey#getWriteEntry}} under a high writing workload.
> After digging into the problem, we found that the problem is mainly caused by 
> advancing mvcc in the append logic. Below is some detailed analysis:
> Under current branch-1 code logic, all batch puts will call 
> {{WALKey#getWriteEntry}} after appending edit to WAL, and 
> {{seqNumAssignedLatch}} is only released when the relative append call is 
> handled by RingBufferEventHandler (see {{FSWALEntry#stampRegionSequenceId}}). 
> Because currently we're using a single event handler for the ringbuffer, the 
> append calls are handled one by one (actually lot's of our current logic 
> depending on this sequential dealing logic), and this becomes a bottleneck 
> under high writing workload.
> The worst part is that by default we only use one WAL per RS, so appends on 
> all regions are dealt with in sequential, which causes contention among 
> different regions...
> To fix this, we could also take use of the "sequential appends" mechanism, 
> that we could grab the WriteEntry before publishing append onto ringbuffer 
> and use it as sequence id, only that we need to add a lock to make "grab 
> WriteEntry" and "append edit" a transaction. This will still cause contention 
> inside a region but could avoid contention between different regions. This 
> solution is already verified in our online environment and proved to be 
> effective.
> Notice that for master (2.0) branch since we already change the write 
> pipeline to sync before writing memstore (HBASE-15158), this issue only 
> exists for the ASYNC_WAL writes scenario.



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

Reply via email to