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 Fri, 12 Feb 2010 05:13:28 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-20100211.patch

Here's my latest patch.  I replaced the dependency on commons-pool for filechannel sharing
with a ReadWriteLock-based pool where it only acquires a global lock to register newly opened
channels or while cleaning up excess channels.  I also have better integration with DataXceiverServer.

I haven't had time to benchmark and won't for a week or so, but I'm hoping for equal speed
streaming and better performance for random read.  

ARCHITECTURE

* DataXCeiverServer now reads the first byte of every request (OP)
* DataXCeivers are now instantiated with an op and inputstream
* If Op was READ_BLOCK and multiplex is enabled, dispatch thread registers with MultiplexedBlockSender
and immediately dies
* MultiplexedBlockSender.register() assigns requests across a configurable pool of SelectorThreads,
round robin
* Each SelectorThread maintains selector and buffer, continually polls for writable and writes
packets, sharing resources across requests
* Maintains current wire protocol
* Packet based model adaptable to asynchronous disk I/O in the future
* Possibly adaptable for local reads by writing packets to a WritableByteChannel wrapping
a ByteBuffer
* Easy to implement connection re-use and keep-alive parameter by dumping threads into a read
Selector and re-registering or closing as necessary



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