hbase-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Andrei Dragomir (JIRA)" <j...@apache.org>
Subject [jira] Commented: (HBASE-2180) read performance from synchronizing hfile.fddatainputstream
Date Thu, 04 Feb 2010 21:41:28 GMT

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

Andrei Dragomir commented on HBASE-2180:
----------------------------------------

We ran some testing that show improved performance by commenting in the change in BoundedFileInputStream,
by replacing the synchronized statement with the one commented out, that uses the PositionalReader
interface

{noformat}
    //synchronized (in) {
    //  in.seek(pos);
    //  ret = in.read(b, off, n);
    //}
    ret = in.read(pos, b, off, n);
{noformat}

> 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.


Mime
View raw message