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

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

Thanks for chiming in boss [~stack] :-)

bq. Yes, intentionally one-by-one in a single thread... How is it a bottleneck
Since there's only one WAL per RS, and single event handler for the ringbuffer, 
it makes contention among writes on different regions. And I forgot to attach 
the jstack when issue happens online (oops...), let me upload it. (the version 
is our modified 1.1.2, so lines may not match, but enough to show the issue I 
guess)

bq. ...Would need to set the ringbuffer initial sequence to be that of the most 
recent edit for the region...Would be interested to hear/see what you are 
thinking
Oh I meant to call {{writeEntry = mvcc.begin();}} and set it into {{WALKey}} 
before publishing the append to ringbuffer, the lock and ringbuffer's 
sequential mechanism could make sure writes with lower mvcc/sequenceId written 
into WAL first. Please check the patch for more details and let me know your 
thoughts sir.

bq. I see an added reentrant lock. Otherwise, all else is the same?
In the patch we call {{writeEntry = mvcc.begin();}} and set it into {{WALKey}} 
before publishing the append to ringbuffer, so we won't block on waiting for 
the CountDownLatch. Previously the CountDownLatch will be released one by one 
due to ringbuffer sequential handling, so writes on different regions will 
race. Please check the attached jstack. :-)

> 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.1.6, 1.2.3
>            Reporter: Yu Li
>            Assignee: Yu Li
>         Attachments: HBASE-16698.patch
>
>
> 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