You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@hbase.apache.org by "stack (JIRA)" <ji...@apache.org> on 2010/02/04 00:59:28 UTC

[jira] Commented: (HBASE-2180) read performance from synchronizing hfile.fddatainputstream

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

stack commented on HBASE-2180:
------------------------------

Using pread -- its present in the code already, its just commented out on the line following the above cited by Ryan -- I see doubled throughput when 16 clients concurrently random reading out of a single regionserver so it helps.  i'll try and get some more numbers in here (I see 'wa' in top running at about the same for both cases but the regionserver is definetly working harder for the pread case using about double the CPU).

Numbers are not that good though -- about 50ms latency doing a random read when 16 concurrent clients.  This is a RS carrying 16M rows on 92 regions where there is 1 storefile only in the family and 4DNs under it.

Way back when we were looking at pread, it improved the random read latency by some small percentage IIRC, about 11%, but then scan speed slowed some... but these would have been for the case of low numbers of concurrent clients.

Its scanning 27k rows/second before the pread change using single client.  And 21k/second after.

Let me get some more numbers... up the concurrent client count and get some other points on how pread changes throughput.


> read performance from synchronizing hfile.fddatainputstream
> -----------------------------------------------------------
>
>                 Key: HBASE-2180
>                 URL: https://issues.apache.org/jira/browse/HBASE-2180
>             Project: Hadoop HBase
>          Issue Type: Bug
>            Reporter: ryan rawson
>            Assignee: ryan rawson
>             Fix For: 0.21.0
>
>
> deep in the HFile read path, there is this code:
>     synchronized (in) {
>       in.seek(pos);
>       ret = in.read(b, off, n);
>     }
> this makes it so that only 1 read per file per thread is active. this prevents the OS and hardware from being able to do IO scheduling by optimizing lots of concurrent reads. 
> We need to either use a reentrant API (pread may be partially reentrant according to Todd) or use multiple stream objects, 1 per scanner/thread.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.