You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by "Liu Shaohui (JIRA)" <ji...@apache.org> on 2013/10/11 04:43:59 UTC

[jira] [Commented] (HBASE-8755) A new write thread model for HLog to improve the overall HBase write throughput

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

Liu Shaohui commented on HBASE-8755:
------------------------------------

[~stack] [~fenghh][~yehangjun]
We redo the same HogPE comparision test using hbase trunk. Results are followed.

Test env:
hdfs: cdh 4.1.0, five datanode, each node has 12 sata disks.
hbase: hbase trunk 0.97 r1516083 (for in r1516084, trunk update the protobuf to 2.5 which is incompatible with protobuf 2.4 used in cdh 4.1.0)
HLogPE is run on one of these datanodes, so one replica of hlog's block will be at local datanode.

||Thread number ||Time without Patch(s) || Ops without Patch || Time with Patch(s) ||Ops with Patch || Time diff %|| Ops diff % ||
|1|580.309|1723.22|624.709|1600.745|-7.65|-7.11|
|1|591.177|1691.541|631.34|1583.932|-6.79|-6.36|
|1|591.948|1689.338|634.518|1575.999|-7.19|-6.71|
|5|794.034|6296.959|1201.563|4161.247|-51.32|-33.92|
|5|781.033|6401.778|1191.776|4195.419|-52.59|-34.46|
|5|805.597|6206.577|1187.179|4211.665|-47.37|-32.14|
|50|3222.659|15515.139|1815.586|27539.316|43.66|77.50|
|50|3191.131|15668.426|1821.956|27443.033|42.91|75.15|
|50|3222.407|15516.352|1817.754|27506.473|43.59|77.27|
|75|4517.149|16603.393|2024.359|37048.766|55.19|123.14|
|75|4498.987|16670.42|2016.899|37185.797|55.17|123.06|
|75|4554.122|16468.598|2037.155|36816.051|55.27|123.55|
|100|5186.292|19281.598|2147.581|46564.016|58.59|141.49|
|100|5181.344|19300.012|2135.768|46821.563|58.78|142.60|
|100|5189.396|19270.064|2143.529|46652.039|58.69|142.10|

> A new write thread model for HLog to improve the overall HBase write throughput
> -------------------------------------------------------------------------------
>
>                 Key: HBASE-8755
>                 URL: https://issues.apache.org/jira/browse/HBASE-8755
>             Project: HBase
>          Issue Type: Improvement
>          Components: Performance, wal
>            Reporter: Feng Honghua
>            Assignee: stack
>            Priority: Critical
>             Fix For: 0.96.1
>
>         Attachments: 8755trunkV2.txt, HBASE-8755-0.94-V0.patch, HBASE-8755-0.94-V1.patch, HBASE-8755-trunk-V0.patch, HBASE-8755-trunk-V1.patch
>
>
> In current write model, each write handler thread (executing put()) will individually go through a full 'append (hlog local buffer) => HLog writer append (write to hdfs) => HLog writer sync (sync hdfs)' cycle for each write, which incurs heavy race condition on updateLock and flushLock.
> The only optimization where checking if current syncTillHere > txid in expectation for other thread help write/sync its own txid to hdfs and omitting the write/sync actually help much less than expectation.
> Three of my colleagues(Ye Hangjun / Wu Zesheng / Zhang Peng) at Xiaomi proposed a new write thread model for writing hdfs sequence file and the prototype implementation shows a 4X improvement for throughput (from 17000 to 70000+). 
> I apply this new write thread model in HLog and the performance test in our test cluster shows about 3X throughput improvement (from 12150 to 31520 for 1 RS, from 22000 to 70000 for 5 RS), the 1 RS write throughput (1K row-size) even beats the one of BigTable (Precolator published in 2011 says Bigtable's write throughput then is 31002). I can provide the detailed performance test results if anyone is interested.
> The change for new write thread model is as below:
>  1> All put handler threads append the edits to HLog's local pending buffer; (it notifies AsyncWriter thread that there is new edits in local buffer)
>  2> All put handler threads wait in HLog.syncer() function for underlying threads to finish the sync that contains its txid;
>  3> An single AsyncWriter thread is responsible for retrieve all the buffered edits in HLog's local pending buffer and write to the hdfs (hlog.writer.append); (it notifies AsyncFlusher thread that there is new writes to hdfs that needs a sync)
>  4> An single AsyncFlusher thread is responsible for issuing a sync to hdfs to persist the writes by AsyncWriter; (it notifies the AsyncNotifier thread that sync watermark increases)
>  5> An single AsyncNotifier thread is responsible for notifying all pending put handler threads which are waiting in the HLog.syncer() function
>  6> No LogSyncer thread any more (since there is always AsyncWriter/AsyncFlusher threads do the same job it does)



--
This message was sent by Atlassian JIRA
(v6.1#6144)