hadoop-common-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Todd Lipcon (JIRA)" <j...@apache.org>
Subject [jira] Commented: (HADOOP-3205) FSInputChecker and FSOutputSummer should allow better access to user buffer
Date Tue, 03 Nov 2009 05:00:59 GMT

    [ https://issues.apache.org/jira/browse/HADOOP-3205?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12772878#action_12772878
] 

Todd Lipcon commented on HADOOP-3205:
-------------------------------------

Been looking at this ticket tonight. I'm not sure exactly what you're getting it. As I am
understanding it, the wrapping looks something like:

User Reader -> FSInputChecker -> FSInputChecker subclass -> BufferedInputStream ->
Underlying source

e.g:
{noformat}
        java.io.FileInputStream.readBytes(FileInputStream.java:Unknown line)
        java.io.FileInputStream.read(FileInputStream.java:199)
        org.apache.hadoop.fs.RawLocalFileSystem$TrackingFileInputStream.read(RawLocalFileSystem.jav
a:90)
        org.apache.hadoop.fs.RawLocalFileSystem$LocalFSFileInputStream.read(RawLocalFileSystem.java
:143)
        java.io.BufferedInputStream.fill(BufferedInputStream.java:218)
        java.io.BufferedInputStream.read1(BufferedInputStream.java:258)
        java.io.BufferedInputStream.read(BufferedInputStream.java:317)
        java.io.DataInputStream.read(DataInputStream.java:132)
        org.apache.hadoop.fs.FSInputChecker.readFully(FSInputChecker.java:385)
        org.apache.hadoop.fs.ChecksumFileSystem$ChecksumFSInputChecker.readChunk(ChecksumFileSystem.java:224)
        org.apache.hadoop.fs.FSInputChecker.readChecksumChunk(FSInputChecker.java:238)
        org.apache.hadoop.fs.FSInputChecker.read1(FSInputChecker.java:190)
        org.apache.hadoop.fs.FSInputChecker.read(FSInputChecker.java:158)
        java.io.DataInputStream.read(DataInputStream.java:83)
        org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:72)
{noformat}

The user's buffer size passed in to fs.open(...) controls the size of the BufferedInputStream
that wraps the underlying input stream (ie raw file or socket). The FSInputChecker does indeed
call read() on that BufferedInputStream once for every 512 bytes (directly into the user buffer),
but in my profiling this doesn't seem to be a CPU hog, since it only results in one syscall
to the underlying stream for every io.file.buffer.size.

As a test of the CPU overhead, I put an 800M file (checksummed) in /dev/shm and profiled hadoop
fs -cat with io.file.buffer.size=64K. This obviously stresses the CPU hogs and syscall overhead
without any actual disk involved. The top consumers are:

{noformat}
   1 61.17% 61.17%    4363 300617 org.apache.hadoop.fs.FSInputChecker.readChecksumChunk
   2 13.11% 74.28%     935 300618 java.io.FileInputStream.readBytes
   3  7.71% 82.00%     550 300632 java.io.DataInputStream.read
   4  5.02% 87.02%     358 300600 java.io.FileOutputStream.writeBytes
   5  3.76% 90.77%     268 300657 java.io.DataInputStream.readFully
   6  1.67% 92.44%     119 300631 java.io.DataInputStream.readFully
{noformat}

The particular line of readChecksumChunk that's consuming the time is line 241 (sum.update)
- this indicates that the overhead here is just from checksumming and not from memory copies.
The one possible gain I could see here would be to revert to a JNI implementation of CRC32
that can do multiple checksum chunks at once - we found that JNI was slow due to a constant
overhead "jumping the gap" to C for small sizes, but we can probably get 50% checksum speedup
for some buffers. This was originally rejected in HADOOP-6148 due to the complexity of maintaining
two different CRC32 implementations.

Are you suggesting here that we could do away with the internal buffer and assume that users
are always going to do large reads? Doesn't that violate the contract of fs.open taking a
buffer size?

> FSInputChecker and FSOutputSummer should allow better access to user buffer
> ---------------------------------------------------------------------------
>
>                 Key: HADOOP-3205
>                 URL: https://issues.apache.org/jira/browse/HADOOP-3205
>             Project: Hadoop Common
>          Issue Type: Bug
>          Components: fs
>            Reporter: Raghu Angadi
>            Assignee: Raghu Angadi
>
> Implementations of FSInputChecker and FSOutputSummer like DFS do not have access to full
user buffer. At any time DFS can access only up to 512 bytes even though user usually reads
with a much larger buffer (often controlled by io.file.buffer.size). This requires implementations
to double buffer data if an implementation wants to read or write larger chunks of data from
underlying storage.
> We could separate changes for FSInputChecker and FSOutputSummer into two separate jiras.

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


Mime
View raw message