[
https://issues.apache.org/jira/browse/HDFS-918?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12976346#action_12976346
]
Todd Lipcon commented on HDFS-918:
----------------------------------
Just took a swing through this patch. A few thoughts:
- Ed: this is only on the read side, so number of xceivers for outputting
thousands of partitions on a single DN won't be helped here. This patch helps
the situation where multiple readers access the same file concurrently, but
currently open it separately for each accessor.
- Jay: I think the approach of splitting into smaller patches is good. The
hdfs-918-pool.patch is pretty manageable, but it seems it could trivially be
split in two - the first half just refactoring BlockSender to use the new
ReadChannel interface, and the second half actually introducing the pool.
Regarding thoughts on the channel pooling patch itself:
- Did you do any experimentation of just adding a small LRU cache of the
DataChecksum info for each block? I'm curious what percent of the latency
improvements is due to avoiding open/close vs avoiding the extra "read" of the
checksum header at the top of the meta file.
- The pool holds its write lock while calling open() on blocks. It seems this
lock only needs to be per-block-id rather than across the whole pool. My worry
is that, if one disk is heavily loaded or broken in some way, the open() call
could stall several seconds, and then all other reads even on other volumes
could be held up.
But, generally, I think this is promising! I think we could realistically get
this in for 0.23.
> 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
> Assignee: Jay Booth
> Fix For: 0.22.0
>
> Attachments: hbase-hdfs-benchmarks.ods, hdfs-918-20100201.patch,
> hdfs-918-20100203.patch, hdfs-918-20100211.patch, hdfs-918-20100228.patch,
> hdfs-918-20100309.patch, hdfs-918-branch20-append.patch,
> hdfs-918-branch20.2.patch, hdfs-918-pool.patch, hdfs-918-TRUNK.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.