hbase-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "stack (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HBASE-8755) A new write thread model for HLog to improve the overall HBase write throughput
Date Sat, 07 Sep 2013 15:54:57 GMT

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

stack commented on HBASE-8755:
------------------------------

Staring in on taking a look at this.  I tried HLogPerformanceEvaluation.  Going via ycsb would
seem to add noise and indirection.  I ran with 1, 5, and 50 threads w/ sizes that are like
Honghua's (key of 50 and value of 150).

I ran the test like this on trunk using localfs:

{code}
$ for i in 1 5 50; do  for j in 1 2 3; do ./bin/hbase org.apache.hadoop.hbase.regionserver.wal.HLogPerformanceEvaluation
 -verify -threads "${i}" -iterations 1000000 -nocleanup -verbose -keySize 50 -valueSize 100
&> /tmp/log-patch"${i}"."${j}".txt; done; done
{code}

Needs fixes over in HBASE-9460 for above to work on localfs.

With localfs, the patch is twice as slow.  localfs does not support sync so this is probably
what makes the difference -- the extra threads do better w/ the dfsclient's stutter around
sync.  Let me try up on hdfs next.

Below are numbers.  I ran each test three times: i.e. three times without patch with one thread,
then three times with patch and one thread, etc.

Each thread did 1M puts.  The table is times for the test to complete in seconds so less is
better.

||Thread Count||WithoutPatch||WithPatch|| 
|1|4.895|29.088|
|1|4.856|29.544|
|1|4.901|29.326|
|5|24.173|53.974|
|5|24.013|55.976|
|5|23.390|55.858|
|50|253.773|454.147|
|50|247.095|443.215|
|50|254.044|449.043|

                
> 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 is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

Mime
View raw message