hbase-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Vladimir Rodionov (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HBASE-7336) HFileBlock.readAtOffset does not work well with multiple threads
Date Tue, 15 Jul 2014 22:01:08 GMT

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

Vladimir Rodionov commented on HBASE-7336:
------------------------------------------

DFSInputStream class is heavily synchronized (at least in HDFS 2.2) and regardless of a read
op type (stream, positional) all readers will be waiting on a single lock eventually.  This
is what I see in my local tests. 

1 scanner - 14 sec
2 scanners - 36 sec (!!!) 
4 scanners - too long to be true. 

I have no explanation yet, but something is wrong here.

> HFileBlock.readAtOffset does not work well with multiple threads
> ----------------------------------------------------------------
>
>                 Key: HBASE-7336
>                 URL: https://issues.apache.org/jira/browse/HBASE-7336
>             Project: HBase
>          Issue Type: Sub-task
>          Components: Performance
>            Reporter: Lars Hofhansl
>            Assignee: Lars Hofhansl
>            Priority: Critical
>             Fix For: 0.94.4, 0.95.0
>
>         Attachments: 7336-0.94.txt, 7336-0.96.txt
>
>
> HBase grinds to a halt when many threads scan along the same set of blocks and neither
read short circuit is nor block caching is enabled for the dfs client ... disabling the block
cache makes sense on very large scans.
> It turns out that synchronizing in istream in HFileBlock.readAtOffset is the culprit.



--
This message was sent by Atlassian JIRA
(v6.2#6252)

Mime
View raw message