[ https://issues.apache.org/jira/browse/HADOOP-1134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12500493 ]
Doug Cutting commented on HADOOP-1134: -------------------------------------- > What is 'length'? The total size of the packet in bytes. Having this up front might make it easier to, e.g., write an NIO-based datanode that uses async io. Ideally we could re-write datanode to be async without modifying the on-the-wire protocol. > Yes, there is one DATA_CHUNK for every bytesPerChecksum. Okay. So I take it that when you write 'XXX ...' you mean '<XXX>*' in BNF, right? That wasn't clear to me. > So, is it the 16 bytes for every bytesPerChecksum you want to save? It wouldn't hurt to save those, but, moreover, I don't see the use case for transmitting the start and length with each checksum, rather it seems like it only makes sense once per request, no? So why not factor it to the OP-level? > There is no reason to think higher level knows best what to buffer and how to > buffer. The primary method for opening a file is: FSDataInputStream FileSystem#open(Path, bufferSize); I do not expect this patch to change that. So the FileSystem implementation creates buffers for applications. The FileSystem implementation also computes checksums. When, e.g., passed a 10MB bufferSize, the FileSystem implementation should attempt to read-ahead and cache 10MB chunks of data: that's what the application is asking it to do. But, in such cases, the FileSystem implementation should also try to arrange to checksum that data as it is delivered to the application from the stream, rather than as it is read into the stream's internal buffer. Do you disagree with this? I expect this patch to implement this by sharing as much code as is reasonable with ChecksumFileSystem. > DistributedFileSystem does not a ChecksumFileSystem any more. Should it be? No, but ChecksumFileSystem's stream implementations should be made public and reusable. Both ChecksumFileSystem and DistributedFileSystem should build on common checksumming input and output stream abstract base classes. These classes should have abstract methods to write and read checksum data. ChecksumFileSystem can extend them to read and write from separate checksum file streams, while DistributedFileSystem can extend them to access checksum data from DFSClient. > Process of making these share the code would result in quite a few changes to > ChecksumFileSystem. May be that should be a different Jira? No, I think it should be a part of this issue. We need shared checksumming stream classes. ChecksumFileSystem and DistributedFileSystem should both be converted to use them. We can't know whether they're really generic and sharable until they're sucessfully used in more than one place, so I think this is properly included in this issue. However, if you want to make it a separate issue, then this issue should be made dependent on that issue. > Block level CRCs in HDFS > ------------------------ > > Key: HADOOP-1134 > URL: https://issues.apache.org/jira/browse/HADOOP-1134 > Project: Hadoop > Issue Type: New Feature > Components: dfs > Reporter: Raghu Angadi > Assignee: Raghu Angadi > Attachments: bc-no-upgrade-05302007.patch, > DfsBlockCrcDesign-05305007.htm > > > Currently CRCs are handled at FileSystem level and are transparent to core > HDFS. See recent improvement HADOOP-928 ( that can add checksums to a given > filesystem ) regd more about it. Though this served us well there a few > disadvantages : > 1) This doubles namespace in HDFS ( or other filesystem implementations ). In > many cases, it nearly doubles the number of blocks. Taking namenode out of > CRCs would nearly double namespace performance both in terms of CPU and > memory. > 2) Since CRCs are transparent to HDFS, it can not actively detect corrupted > blocks. With block level CRCs, Datanode can periodically verify the checksums > and report corruptions to namnode such that name replicas can be created. > We propose to have CRCs maintained for all HDFS data in much the same way as > in GFS. I will update the jira with detailed requirements and design. This > will include same guarantees provided by current implementation and will > include a upgrade of current data. > -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.