Hey Brian, This is really interesting stuff. I'm curious - have you tried these same experiments using the Java API? I'm wondering whether this is FUSE-specific or inherent to all HDFS reads. I'll try to reproduce this over here as well.
This smells sort of nagle-related to me... if you get a chance, you may want to edit DFSClient.java and change TCP_WINDOW_SIZE to 256 * 1024, and see if the magic number jumps up to 256KB. If so, I think it should be a pretty easy bugfix. -Todd On Sun, Apr 12, 2009 at 9:41 PM, Brian Bockelman <[email protected]>wrote: > Ok, here's something perhaps even more strange. I removed the "seek" part > out of my timings, so I was only timing the "read" instead of the "seek + > read" as in the first case. I also turned the read-ahead down to 1-byte > (aka, off). > > The jump *always* occurs at 128KB, exactly. > > I'm a bit befuddled. I know we say that HDFS is optimized for large, > sequential reads, not random reads - but it seems that it's one bug-fix away > from being a good general-purpose system. Heck if I can find what's causing > the issues though... > > Brian > > > > > > On Apr 12, 2009, at 8:53 PM, Brian Bockelman wrote: > > Hey all, >> >> I was doing some research on I/O patterns of our applications, and I >> noticed the attached pattern. In case if the mail server strips out >> attachments, I also uploaded it: >> >> http://t2.unl.edu/store/Hadoop_64KB_ra.png >> http://t2.unl.edu/store/Hadoop_1024KB_ra.png >> >> This was taken using the FUSE mounts of Hadoop; the first one was with a >> 64KB read-ahead and the second with a 1MB read-ahead. This was taken from a >> 2GB file and randomly 'seek'ed in the file. This was performed 20 times for >> each read size, advancing in 4KB increments. Each blue dot is the read time >> of one experiment; the red dot is the median read time for the read size. >> The graphs show the absolute read time. >> >> There's very interesting behavior - it seems that there is a change in >> behavior around reads of size of 800KB. The time for the reads go down >> significantly when you read *larger* files. I thought this was just an >> artifact of the 64KB read-ahead I set in FUSE, so I upped the read-ahead >> significantly, to 1MB. In this case, the difference between the the small >> read sizes and large read sizes are *very* pronounced. If it was an >> artifact from FUSE, I'd expect the place where the change occurred would be >> a function of the readahead-size. >> >> Anyone out there who knows the code have any ideas? What could I be doing >> wrong? >> >> Brian >> >> <Hadoop_64KB_ra.png> >> >> <Hadoop_1024KB_ra.png> >> > > >
