[ https://issues.apache.org/jira/browse/HDFS-918?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12831692#action_12831692 ]
Jay Booth commented on HDFS-918: -------------------------------- BENCHMARK UPDATE: For single threaded reads of 100MB, the new implementation is about the same, a hair faster, probably because I have the (now tuneable) packet size cranked up to 512kb. DFS came in around 90MB/s, my new implementation with the higher packet sizes averaged around 95MB/s. (30 runs each over a 100MB file on a machine with 7G ram, so we're probably reading from FS cache for most of those reads) For multithreaded reads, the new implementation seems to have some kind of synchronization chokepoint, probably in the commons-pool which I'm using to grab the file handles needed to process each packet. Commons-pool predates Java 5 so they're probably using good-ol synchronized when something based around a ReadWriteLock could be better for our needs here. I'll write something up when I get a chance. Numbers were: 20 threads, DFS, 5MB/s/thread -- multiplexing, 1.5 MB/s/thread. In both cases, the first couple threads to start and last couple to finish ran way fast because they weren't contending for disk or network resources -- then when contention started, multiplexing peaked out lower. The block fetching is the only system wide point of synchronization (aside from a very brief one to do round-robin arithmetic to pick the right Selector when adding new threads). Adjusting the number of SelectorThreads for the Multiplexing implementation didn't appreciably change results, so I have to figure it's locking on that BlockChannelPool rather than the very brief lock on the way into a given SelectorThread. Also, I cleaned up the code a bit and integrated with SimulatedFSDataset so that it seemed all tests were passing. I'll run a full ant test and report whether it fails. I'll post the newest patch later tonight or tomorrow morning along with an updated architecture description, estimate I should get some time to throw together a more performant BlockChannelPool over the next week or two. > 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.