hbase-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Feng Honghua (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HBASE-8755) A new write thread model for HLog to improve the overall HBase write throughput
Date Fri, 22 Nov 2013 03:55:57 GMT

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

Feng Honghua commented on HBASE-8755:
-------------------------------------

[~stack] : I fixed the downgrade for 5 threads and below are the test result, the write throughput
capacity after tuning is *about 3.5 times* of the one before tuning while keeping the perf
is almost equal when thread number is <= 5. (for protobuf/test compatibility reason I based
on hbase trunk 0.97 r1516083)

{code}
  for i in 1 3 5 10 25 50 100 200; do
    for j in 1; do
        ./bin/hbase org.apache.hadoop.hbase.regionserver.wal.HLogPerformanceEvaluation -path
/user/h_fenghonghua/new-thread-v2/ -verify -threads "${i}" -iterations 1000000 -keySize 50
-valueSize 100 &> log-patch"${i}"."${j}".txt;
        grep "Summary: " log-patch"${i}"."${j}".txt
    done;
done
{code}

||threads||time-without-patch||ops-without-patch||time-with-patch||ops-with-patch||ops-diff||
|1|582|1716|630|1586|-7.5%|
|3|943|3179|951|3153|-0.8%|
|5|820|6091|847|5899|-3.1%|
|10|1141|8760|983|10166|+16%|
|25|1920|13019|1286|19426|+49.2%|
|50|3334|14995|1627|30715|+104.8%|
|100|5312|18824|1925|51943|+185%|
|200|11022|18144|3229|61922|+241.2%|

[~jmspaggi] : I attached patches for latest trunk (HBASE-8755-trunk-v4.patch) and for the
last 0.96 (HBASE-8755-0.96-v0.patch). Thanks very much if you can run similar comparison perf
tests, and any issue please feel free to raise :-)

> 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
>         Attachments: 8755trunkV2.txt, HBASE-8755-0.94-V0.patch, HBASE-8755-0.94-V1.patch,
HBASE-8755-0.96-v0.patch, HBASE-8755-trunk-V0.patch, HBASE-8755-trunk-V1.patch, HBASE-8755-trunk-v4.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)

Mime
View raw message