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

2010-07-19 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: hbase-hdfs-benchmarks.ods

Benchmarked on EC2 this weekend, I set up 0.20.2-append clean, a copy with my 
multiplex patch applied, and a third copy which only ports filechannel pooling 
to the current architecture (can submit that patch later, it's at home).


All runs were with HBase block caching disabled to highlight the difference in 
filesystem access speeds.  

This is running across a decently small dataset (little less than 1GB) so all 
files are presumably in memory for the majority of test duration.

Run involved 6 clients reading 1,000,000 rows each divided over 10 mappers.  
Cluster setup was 3x EC2 High-CPU XL, 1 NN/JT/ZK/Master and 2x DN/TT/RS.  Ran 
in 3 batches of 3 runs each.  Cluster was restarted in between each batch for 
each run type because we're changing DN implementation.


Topline numbers (rest are in document):

Total Run Averages  

Testclean   poolmultiplex
random  21159050.44 19448216.89 16806247
scan436106.89   442452.54   443262.56
sequential  19298239.78 17871047.67 14987028.44

Pool is 7.5% gain, multiplex is more like 20% for random reads

Only batches 2+3 (batch 1 was a little messed up and doesn't track with others) 

Testclean   poolmultiplex
random  20555308.67 1842501716987643.33
scan426849  427277.98   448031
sequential  18665323.67 16969885.83 15102404

Pool is 10% gain, multiplex is 17% or so for random reads

Per row for random read (batches 2+3 only):
clean: 3.42ms
pool: 3.07ms
multiplex: 2.83ms


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



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

2010-07-09 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-918-branch20-append.patch

Managed to get back to this.

Rebased on branch-20-append.

Fixed resource leak issue that apurtell identified.

Runs through HBase PerformanceEvaluation on my workstation completely with 
default ulimit of 1024, no crashes.

I'm going to try and benchmark this on a real cluster this weekend and report 
results.  Happy Friday everyone

 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-918-branch20-append.patch, hdfs-918-branch20.2.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.



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

2010-03-31 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: (was: hdfs-918-branch20.2.patch)

 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-918-branch20.2.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.



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

2010-03-31 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-918-branch20.2.patch

Straightened out the block not found thing with Andrew, that was on his end, 
but then he found a resource leak that's fixed here -- I'll post a trunk patch 
which incorporates this fix and the previous fix shortly.

 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-918-branch20.2.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.



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

2010-03-31 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-918-TRUNK.patch

Trunk patch with previous fixes.  

 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-918-branch20.2.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.



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

2010-03-30 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-918-branch20.2.patch

Cleaned up a bug in the BlockChannelPool.cleanup() code, added new unit test, 
improved descriptions of new config values (useMultiplex, packetSize, 
maxOpenBlockChannels, minOpenBlockchannels (number to cleanup() to)).

This patch is for branch 20, I'll post a new one against trunk tonight.

 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-918-branch20.2.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.



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

2010-03-30 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: (was: hdfs-200+826+918-branch20.patch)

 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-918-branch20.2.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.



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

2010-03-24 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-200+826+918-branch20.patch

I heard all the cool kids are running HDFS-200 and HDFS-826 on their 0.20.2 
installations these days, so I merged HDFS-918 with them.

Also, nobody use the existing 0.20.2 patch, I'll delete now and post a new one 
tonight -- it happens to be missing a very important Thread.start() invocation.

 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-200+826+918-branch20.patch, 
 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.



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

2010-03-24 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: (was: hdfs-918-0.20.2.patch)

 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-200+826+918-branch20.patch, 
 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.



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

2010-03-22 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-918-0.20.2.patch

0.20.2 compatible patch!

A couple people mentioned that it would be much easier for them to benchmark if 
I produced an 0.20.2 compatible patch.  So here it is, it works, seems to pass 
all unit tests that I ran on it, and I even did a hadoop fs -put and hadoop fs 
-cat.  But that's the entire extent of the testing, unit tests and a 
super-simple pseudodistributed operation.

So anyone who wants to try this on some I/O bound jobs on a test 0.20.2 cluster 
and see if they have speedups, please feel free and report results.

 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-0.20.2.patch, 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.



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

2010-03-08 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-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.



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

2010-02-28 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-918-20100228.patch

New patch -- Took Zlatin's advice and utilized selectionKey.interestOps(0) to 
avoid busy waits, so we're back to a single selector and an ExecutorService.  
The ExecutorService reuses threads if possible, destroying threads that haven't 
been used in 60 seconds.  Analyzed logs and the selectorThread doesn't seem to 
busy wait ever.  Buffers are now stored in threadlocals and allocated per 
thread (they're now HeapByteBuffers since we might have some churn and most of 
our transfer is using transferTo anyways).  Still uses shared BlockChannelPool 
implemented via ReadWriteLock.  

I think this will be pretty good, will benchmark tonight.

 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-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-918) Use single Selector and small thread pool to replace many instances of BlockSender for reads

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



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

2010-01-31 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-918-20100201.patch

New patch.. 

* new configuration params:  dfs.datanode.multiplexBlockSender=true, 
dfs.datanode.multiplex.packetSize=32k, dfs.datanode.multiplex.numWorkers=3

* Packet size is tuneable, possibly allowing better performance with larger TCP 
buffers enabled

* Workers only wake up when a connection is writable

* 3 new class files, minor changes to DataXceiverServer and DataXceiver, 2 
utility classes added to DataTransferProtocol (one stolen from HDFS-881)

* Passes tests from earlier comment  plus a new one for files with lengths that 
don't match up to checksum chunk size, as well as holding up to some load on 
TestDFSIO

* Still fails all tests relying on SimulatedFSDataset

* Has a large amount of TRACE level debugging going on in 
MultiplexedBlockSender in case anybody wants to watch the output

* Adds dependencies for commons-pool and commons-math (for benchmarking code)

* Doesn't yet have benchmarks, but those should be easy now that the 
configuration is all in place

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