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] Updated: (HDFS-918) Use single Selector and small thread pool to replace many instances of BlockSender for reads
Date Wed, 03 Feb 2010 06:13:19 GMT

     [ https://issues.apache.org/jira/browse/HDFS-918?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]

Jay Booth updated HDFS-918:
---------------------------

    Attachment: hdfs-918-20100203.patch

New patch.  Streamlined MultiplexedBlockSender, we now have one selector per worker thread
and no BlockingQueues, writeable connections are handled inline by each thread as they're
available.  

Includes a utility class to read a file with a bunch of threads and time them.

Ran some ad hoc jobs on my laptop and got similar performance to existing BlockSender, slightly
faster for single file and slightly slower for 15 competing localhost threads..  which is
exactly the opposite of what I boldly predicted.   I read somewhere that linux thread scheduling
for Java is disabled because it requires root, so it ignores priority -- if that's the case,
maybe running more threads is actually an advantage when all the readers are local and you're
directly competing with them for CPU -- you compete more effectively for limited resources
with more threads.

I'm gonna try and write an MR job to run some different scenarios on a cluster soon (thundering
herd, steady medium, large number of idles, individual read)..  I think the architecture here
is more suited to large numbers of connections so if it did ok under a small number, then
great.  I'll be pretty busy for the next month or so but will try to get this running in a
cluster at some point and report some more interesting numbers.

> 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