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 Tue, 09 Mar 2010 07:55:32 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-20100309.patch

New patch and better benchmarks:

Environment:  
8x2GHz, 7GB RAM, namenode and dfs client
8x2GHz, 7GB RAM, datanode

Streaming:
Single threaded:  60 runs over 100MB file, presumed in memory so network is chokepoint
Current DFS : 92MB/s over 60 runs
Multiplex : 97 MB/s over 60 runs
* Either random variation, or maybe larger packet size helps

Multi-threaded - 32 threads reading 100MB file, 60X each
Both around 3.25MB/s/thread, 104 MB/s aggregate
Network saturation


Random reads:
The multiplexed implementation saves about 1.5 ms, probably by avoiding extra file-opens and
buffer allocation.
 - 5 iterations of 2000 reads each, 32kb, front of file, singlethreaded
 - splits for current DFS: 5.3, 4.6, 5.0, 4.4, 6.4
 - splits for multiplex:    3.2, 3.0, 4.6, 3.3 ,3.2
 - multithreaded concurrent read speeds on a single host converged with more threads -- probably
client-side delay negotiating lots of new tcp connections 


File handle consumption:
Both "rest" at 401 open files (mostly jars)

When doing random reads across 128 threads, BlockSender spikes to the 1150, opening a blockfile,
metafile, selector, and socket for each concurrent connection.

MultiplexedBlockSender only jumps to 530, with just the socket as a per-connection resource,
blockfiles, metafiles and the single selector are shared.



I'll post a comment later with an updated description of the patch, and when I get a chance,
I'll run some more disk-bound benchmarks, I think the asynchronous approach will pay some
dividends there by letting the operating system do more of the work.  

Super brief patch notes:
eliminated silly add'l dependency on commons-math, now has no new dependencies
incorporated Zlatin's suggestions upthread to do asynchronous I/O, 1 shared selector
BlockChannelPool is shared across threads
Buffers are threadlocal so they'll tend to be re-used rather than re-allocated




> 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-918-20100211.patch,
hdfs-918-20100228.patch, hdfs-918-20100309.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