[jira] Commented: (HDFS-916) Rewrite DFSOutputStream to use a single thread with NIO

2010-01-23 Thread stack (JIRA)

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

stack commented on HDFS-916:


A gallant undertaking: +1.  I volunteer testing and review.

> Rewrite DFSOutputStream to use a single thread with NIO
> ---
>
> Key: HDFS-916
> URL: https://issues.apache.org/jira/browse/HDFS-916
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: hdfs client
>Affects Versions: 0.22.0
>Reporter: Todd Lipcon
>Assignee: Todd Lipcon
>
> The DFS write pipeline code has some really hairy multi-threaded 
> synchronization. There have been a lot of bugs produced by this (HDFS-101, 
> HDFS-793, HDFS-915, tens of others) since it's very hard to understand the 
> message passing, lock sharing, and interruption properties. The reason for 
> the multiple threads is to be able to simultaneously send and receive. If 
> instead of using multiple threads, it used nonblocking IO, I think the whole 
> thing would be a lot less error prone.
> I think we could do this in two halves: one half is the DFSOutputStream. The 
> other half is BlockReceiver. I opened this JIRA first as I think it's simpler 
> (only one TCP connection to deal with, rather than an up and downstream)
> Opinions? Am I crazy? I would like to see some agreement on the idea before I 
> spend time writing code.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.



[jira] Created: (HDFS-918) Use single Selector and small thread pool to replace many instances of BlockSender for reads

2010-01-23 Thread Jay Booth (JIRA)
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


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.



[jira] Updated: (HDFS-918) Use single Selector and small thread pool to replace many instances of BlockSender for reads

2010-01-23 Thread Jay Booth (JIRA)

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

Jay Booth updated HDFS-918:
---

Attachment: hdfs-multiplex.patch

Here's a first implementation -- it works, passes TestDistributedFileSystem, 
TestDataTransferProtocol and TestPread.  However, it has a direct dependency on 
FSDataset (not FSDatasetInterface) because it needs to get ahold of files 
directly to open FileChannels.  This leads to ClassCastExceptions in all tests 
relying on SimulatedFSDataset.  Would love to hear feedback about a way to 
resolve this.  

Have not benchmarked yet, I'll post another comment with an architectural 
description.

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



[jira] Updated: (HDFS-516) Low Latency distributed reads

2010-01-23 Thread Jay Booth (JIRA)

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

Jay Booth updated HDFS-516:
---

Resolution: Won't Fix
Status: Resolved  (was: Patch Available)

This work didn't really go anywhere, I tried multithreading on the client side 
to make a request-per-packet approach viable, but it wound up being even slower 
than singlethreaded request-per-packet.  If there's interest in the client-side 
byte cache, I could create another issue with just that improvement.

> Low Latency distributed reads
> -
>
> Key: HDFS-516
> URL: https://issues.apache.org/jira/browse/HDFS-516
> Project: Hadoop HDFS
>  Issue Type: New Feature
>Reporter: Jay Booth
>Priority: Minor
> Attachments: hdfs-516-20090912.patch, radfs.odp
>
>   Original Estimate: 168h
>  Remaining Estimate: 168h
>
> I created a method for low latency random reads using NIO on the server side 
> and simulated OS paging with LRU caching and lookahead on the client side.  
> Some applications could include lucene searching (term->doc and doc->offset 
> mappings are likely to be in local cache, thus much faster than nutch's 
> current FsDirectory impl and binary search through record files (bytes at 
> 1/2, 1/4, 1/8 marks are likely to be cached)

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.