hadoop-hdfs-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Jay Booth (JIRA)" <j...@apache.org>
Subject [jira] Commented: (HDFS-918) Use single Selector and small thread pool to replace many instances of BlockSender for reads
Date Tue, 09 Feb 2010 22:33:28 GMT

    [ https://issues.apache.org/jira/browse/HDFS-918?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12831715#action_12831715
] 

Jay Booth commented on HDFS-918:
--------------------------------

Absolutely -- every packet send has to fetch the BlockChannel that it's transferring from
out of a shared pool.  Given that I'm using commons-pool for this, it's probably a straight
"synchronized" on every get, which would predictably kill performance.  Commons-pool is more
built around the idea of a nonsharable database connection anyways, so I'll write my own custom
thingy that works off of a ReadWriteLock and allows different threads to share the same BlockChannel
instance (since BlockChannel only exposes threadsafe channel operations).  

If that doesn't do it, well, I'll see if I can figure it out but I agree that this is not
performant enough yet.  I think there's an argument for committing this with equivalent performance
because it simplifies the read pipeline on the datanode side and consumes fewer resources
(selectors/filechannels), but 1/3 the throughput, no way.  On the upside, it means I won't
be adding new dependencies with this patch anymore once I rebuild that block channel pool.
  (Kinda jumping the gun here on assuming the cause but it's literally the only source of
systemwide synchronization, and things only slowed down once I added a bunch of threads, so...)

I could definitely set that random read test up, once I get the slow pool issue sorted out.
 Assuming that the pool is the issue, this setup should save a bunch of time specifically
in that scenario -- no per-request buffer allocation or file opening.

> Use single Selector and small thread pool to replace many instances of BlockSender for
reads
> --------------------------------------------------------------------------------------------
>
>                 Key: HDFS-918
>                 URL: https://issues.apache.org/jira/browse/HDFS-918
>             Project: Hadoop HDFS
>          Issue Type: Improvement
>          Components: data-node
>            Reporter: Jay Booth
>             Fix For: 0.22.0
>
>         Attachments: hdfs-918-20100201.patch, hdfs-918-20100203.patch, hdfs-multiplex.patch
>
>
> Currently, on read requests, the DataXCeiver server allocates a new thread per request,
which must allocate its own buffers and leads to higher-than-optimal CPU and memory usage
by the sending threads.  If we had a single selector and a small threadpool to multiplex request
packets, we could theoretically achieve higher performance while taking up fewer resources
and leaving more CPU on datanodes available for mapred, hbase or whatever.  This can be done
without changing any wire protocols.

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