[jira] Commented: (HDFS-1377) Quota bug for partial blocks allows quotas to be violated
[ https://issues.apache.org/jira/browse/HDFS-1377?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12969857#action_12969857 ] Raghu Angadi commented on HDFS-1377: +1. Thanks Eli. The warning in contentSummary() is also useful. Quota bug for partial blocks allows quotas to be violated -- Key: HDFS-1377 URL: https://issues.apache.org/jira/browse/HDFS-1377 Project: Hadoop HDFS Issue Type: Bug Components: name-node Affects Versions: 0.20.1, 0.20.2, 0.21.0, 0.22.0, 0.23.0 Reporter: Eli Collins Assignee: Eli Collins Priority: Blocker Fix For: 0.20.3, 0.21.1, 0.22.0, 0.23.0 Attachments: hdfs-1377-1.patch, hdfs-1377-b20-1.patch, hdfs-1377-b20-2.patch, hdfs-1377-b20-3.patch, HDFS-1377.patch There's a bug in the quota code that causes them not to be respected when a file is not an exact multiple of the block size. Here's an example: {code} $ hadoop fs -mkdir /test $ hadoop dfsadmin -setSpaceQuota 384M /test $ ls dir/ | wc -l # dir contains 101 files 101 $ du -ms dir# each is 3mb 304 dir $ hadoop fs -put dir /test $ hadoop fs -count -q /test none inf 402653184 -5505024002 101 317718528 hdfs://haus01.sf.cloudera.com:10020/test $ hadoop fs -stat %o %r /test/dir/f30 134217728 3# three 128mb blocks {code} INodeDirectoryWithQuota caches the number of bytes consumed by it's children in {{diskspace}}. The quota adjustment code has a bug that causes {{diskspace}} to get updated incorrectly when a file is not an exact multiple of the block size (the value ends up being negative). This causes the quota checking code to think that the files in the directory consumes less space than they actually do, so the verifyQuota does not throw a QuotaExceededException even when the directory is over quota. However the bug isn't visible to users because {{fs count -q}} reports the numbers generated by INode#getContentSummary which adds up the sizes of the blocks rather than use the cached INodeDirectoryWithQuota#diskspace value. In FSDirectory#addBlock the disk space consumed is set conservatively to the full block size * the number of replicas: {code} updateCount(inodes, inodes.length-1, 0, fileNode.getPreferredBlockSize()*fileNode.getReplication(), true); {code} In FSNameSystem#addStoredBlock we adjust for this conservative estimate by subtracting out the difference between the conservative estimate and what the number of bytes actually stored was: {code} //Updated space consumed if required. INodeFile file = (storedBlock != null) ? storedBlock.getINode() : null; long diff = (file == null) ? 0 : (file.getPreferredBlockSize() - storedBlock.getNumBytes()); if (diff 0 file.isUnderConstruction() cursize storedBlock.getNumBytes()) { ... dir.updateSpaceConsumed(path, 0, -diff*file.getReplication()); {code} We do the same in FSDirectory#replaceNode when completing the file, but at a file granularity (I believe the intent here is to correct for the cases when there's a failure replicating blocks and recovery). Since oldnode is under construction INodeFile#diskspaceConsumed will use the preferred block size (vs of Block#getNumBytes used by newnode) so we will again subtract out the difference between the full block size and what the number of bytes actually stored was: {code} long dsOld = oldnode.diskspaceConsumed(); ... //check if disk space needs to be updated. long dsNew = 0; if (updateDiskspace (dsNew = newnode.diskspaceConsumed()) != dsOld) { try { updateSpaceConsumed(path, 0, dsNew-dsOld); ... {code} So in the above example we started with diskspace at 384mb (3 * 128mb) and then we subtract 375mb (to reflect only 9mb raw was actually used) twice so for each file the diskspace for the directory is - 366mb (384mb minus 2 * 375mb). Which is why the quota gets negative and yet we can still write more files. So a directory with lots of single block files (if you have multiple blocks on the final partial block ends up subtracting from the diskspace used) ends up having a quota that's way off. I think the fix is to in FSDirectory#replaceNode not have the diskspaceConsumed calculations differ when the old and new INode have the same blocks. I'll work on a patch which also adds a quota test for blocks that are not multiples of the block size and warns in INodeDirectory#computeContentSummary if the computed size does not reflect the cached value. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-1377) Quota bug for partial blocks allows quotas to be violated
[ https://issues.apache.org/jira/browse/HDFS-1377?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12965378#action_12965378 ] Raghu Angadi commented on HDFS-1377: Thanks Eli. will review the patch tonight (tomorrow night at the latest). Quota bug for partial blocks allows quotas to be violated -- Key: HDFS-1377 URL: https://issues.apache.org/jira/browse/HDFS-1377 Project: Hadoop HDFS Issue Type: Bug Components: name-node Affects Versions: 0.20.1, 0.20.2, 0.21.0, 0.22.0, 0.23.0 Reporter: Eli Collins Assignee: Eli Collins Priority: Blocker Fix For: 0.20.3, 0.21.1, 0.22.0, 0.23.0 Attachments: hdfs-1377-1.patch, hdfs-1377-b20-1.patch, hdfs-1377-b20-2.patch, hdfs-1377-b20-3.patch, HDFS-1377.patch There's a bug in the quota code that causes them not to be respected when a file is not an exact multiple of the block size. Here's an example: {code} $ hadoop fs -mkdir /test $ hadoop dfsadmin -setSpaceQuota 384M /test $ ls dir/ | wc -l # dir contains 101 files 101 $ du -ms dir# each is 3mb 304 dir $ hadoop fs -put dir /test $ hadoop fs -count -q /test none inf 402653184 -5505024002 101 317718528 hdfs://haus01.sf.cloudera.com:10020/test $ hadoop fs -stat %o %r /test/dir/f30 134217728 3# three 128mb blocks {code} INodeDirectoryWithQuota caches the number of bytes consumed by it's children in {{diskspace}}. The quota adjustment code has a bug that causes {{diskspace}} to get updated incorrectly when a file is not an exact multiple of the block size (the value ends up being negative). This causes the quota checking code to think that the files in the directory consumes less space than they actually do, so the verifyQuota does not throw a QuotaExceededException even when the directory is over quota. However the bug isn't visible to users because {{fs count -q}} reports the numbers generated by INode#getContentSummary which adds up the sizes of the blocks rather than use the cached INodeDirectoryWithQuota#diskspace value. In FSDirectory#addBlock the disk space consumed is set conservatively to the full block size * the number of replicas: {code} updateCount(inodes, inodes.length-1, 0, fileNode.getPreferredBlockSize()*fileNode.getReplication(), true); {code} In FSNameSystem#addStoredBlock we adjust for this conservative estimate by subtracting out the difference between the conservative estimate and what the number of bytes actually stored was: {code} //Updated space consumed if required. INodeFile file = (storedBlock != null) ? storedBlock.getINode() : null; long diff = (file == null) ? 0 : (file.getPreferredBlockSize() - storedBlock.getNumBytes()); if (diff 0 file.isUnderConstruction() cursize storedBlock.getNumBytes()) { ... dir.updateSpaceConsumed(path, 0, -diff*file.getReplication()); {code} We do the same in FSDirectory#replaceNode when completing the file, but at a file granularity (I believe the intent here is to correct for the cases when there's a failure replicating blocks and recovery). Since oldnode is under construction INodeFile#diskspaceConsumed will use the preferred block size (vs of Block#getNumBytes used by newnode) so we will again subtract out the difference between the full block size and what the number of bytes actually stored was: {code} long dsOld = oldnode.diskspaceConsumed(); ... //check if disk space needs to be updated. long dsNew = 0; if (updateDiskspace (dsNew = newnode.diskspaceConsumed()) != dsOld) { try { updateSpaceConsumed(path, 0, dsNew-dsOld); ... {code} So in the above example we started with diskspace at 384mb (3 * 128mb) and then we subtract 375mb (to reflect only 9mb raw was actually used) twice so for each file the diskspace for the directory is - 366mb (384mb minus 2 * 375mb). Which is why the quota gets negative and yet we can still write more files. So a directory with lots of single block files (if you have multiple blocks on the final partial block ends up subtracting from the diskspace used) ends up having a quota that's way off. I think the fix is to in FSDirectory#replaceNode not have the diskspaceConsumed calculations differ when the old and new INode have the same blocks. I'll work on a patch which also adds a quota test for blocks that are not multiples of the block size and warns in INodeDirectory#computeContentSummary if the computed size does not reflect the cached value. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-1377) Quota bug for partial blocks allows quotas to be violated
[ https://issues.apache.org/jira/browse/HDFS-1377?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12933651#action_12933651 ] Raghu Angadi commented on HDFS-1377: I am still refreshing my memory of this NN internals... I think the patch is correct. I too am not sure if this causes problem else where. looks like it fixes more that it might break. This patch fixes bug the case where replaceNode(oldINode, newINode) is called under normal close().. where though oldINode is a INodeUnderConstruction only by its Java type, but not logically (it is already been removed from the active leases etc). What about the other way around where newINode is an INodeUnderConstruction? as described by Eli below : bq. So the question becomes do oldnode and newnode ever have different blocks? On branch 20 I don't think that's the case. The only caller where it seems that it could potentially be the case is loadFilesUnderConstruction, in which case the under construction INode may have an extra block, but then wouldn't that have been accounted for via addStoredBlock already? Not sure how it manifests in real life, but here a file is going from INodeFile to INodeFIleUnderConstruction. So the actual space consumed should be rounded upwards to the block boundery. Add stored block is not called at this time. How about calling replaceNode(..., updateDiskspace = false) in finalizeFileUnderconstruction(), since we know that oldNode is not under construction anymore? btw, do we separate test file for this? TestQuota.java is already supposed to test violations (FWIW, it does test some violations). Quota bug for partial blocks allows quotas to be violated -- Key: HDFS-1377 URL: https://issues.apache.org/jira/browse/HDFS-1377 Project: Hadoop HDFS Issue Type: Bug Components: name-node Affects Versions: 0.20.1, 0.20.2, 0.21.0, 0.22.0, 0.23.0 Reporter: Eli Collins Assignee: Eli Collins Priority: Blocker Fix For: 0.20.3, 0.21.1, 0.22.0, 0.23.0 Attachments: hdfs-1377-b20-1.patch, hdfs-1377-b20-2.patch, HDFS-1377.patch There's a bug in the quota code that causes them not to be respected when a file is not an exact multiple of the block size. Here's an example: {code} $ hadoop fs -mkdir /test $ hadoop dfsadmin -setSpaceQuota 384M /test $ ls dir/ | wc -l # dir contains 101 files 101 $ du -ms dir# each is 3mb 304 dir $ hadoop fs -put dir /test $ hadoop fs -count -q /test none inf 402653184 -5505024002 101 317718528 hdfs://haus01.sf.cloudera.com:10020/test $ hadoop fs -stat %o %r /test/dir/f30 134217728 3# three 128mb blocks {code} INodeDirectoryWithQuota caches the number of bytes consumed by it's children in {{diskspace}}. The quota adjustment code has a bug that causes {{diskspace}} to get updated incorrectly when a file is not an exact multiple of the block size (the value ends up being negative). This causes the quota checking code to think that the files in the directory consumes less space than they actually do, so the verifyQuota does not throw a QuotaExceededException even when the directory is over quota. However the bug isn't visible to users because {{fs count -q}} reports the numbers generated by INode#getContentSummary which adds up the sizes of the blocks rather than use the cached INodeDirectoryWithQuota#diskspace value. In FSDirectory#addBlock the disk space consumed is set conservatively to the full block size * the number of replicas: {code} updateCount(inodes, inodes.length-1, 0, fileNode.getPreferredBlockSize()*fileNode.getReplication(), true); {code} In FSNameSystem#addStoredBlock we adjust for this conservative estimate by subtracting out the difference between the conservative estimate and what the number of bytes actually stored was: {code} //Updated space consumed if required. INodeFile file = (storedBlock != null) ? storedBlock.getINode() : null; long diff = (file == null) ? 0 : (file.getPreferredBlockSize() - storedBlock.getNumBytes()); if (diff 0 file.isUnderConstruction() cursize storedBlock.getNumBytes()) { ... dir.updateSpaceConsumed(path, 0, -diff*file.getReplication()); {code} We do the same in FSDirectory#replaceNode when completing the file, but at a file granularity (I believe the intent here is to correct for the cases when there's a failure replicating blocks and recovery). Since oldnode is under construction INodeFile#diskspaceConsumed will use the preferred block size (vs of Block#getNumBytes used by newnode) so we will again subtract out the difference between the full block size and what the number of bytes actually stored was: {code}
[jira] Commented: (HDFS-1414) HDFS federation : fix unit test cases
[ https://issues.apache.org/jira/browse/HDFS-1414?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12915422#action_12915422 ] Raghu Angadi commented on HDFS-1414: If the existing upgrade test fails, that implies HDFS can not be upgraded from that version. If the upgrade from hadoop-0.14 is supported than we should not need to modify. First we need decide what the latest version that this HDFS can upgrade from. Then we should create an image with that version (say it is hadoop-0.21) HDFS federation : fix unit test cases - Key: HDFS-1414 URL: https://issues.apache.org/jira/browse/HDFS-1414 Project: Hadoop HDFS Issue Type: Sub-task Reporter: Tanping Wang Attachments: HDFS1414-branchHDFS1052.1.patch -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-918) Use single Selector and small thread pool to replace many instances of BlockSender for reads
[ https://issues.apache.org/jira/browse/HDFS-918?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12844620#action_12844620 ] Raghu Angadi commented on HDFS-918: --- RE: Netty, I'm not very knowledgeable about it beyond the Cliff's Notes version, but my code dealing with the Selector is pretty small - the main loop is under 75 lines, and java.util.concurrent does most of the heavy lifting Jay, I think is ok to ignore Netty for this jira. it could be re-factored later. I think it is very important to have separate pools for each partition. This would be the case if I were using a fixed-size thread pool and a LinkedBlockingQueue - but I'm not, see Executors.newCachedThreadPool(), hmm.. does it mean that if you have thousand clients and the load is disk bound, we end up with 1000 threads? 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] Commented: (HDFS-918) Use single Selector and small thread pool to replace many instances of BlockSender for reads
[ https://issues.apache.org/jira/browse/HDFS-918?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12844278#action_12844278 ] Raghu Angadi commented on HDFS-918: --- Thanks Jay. This is a well known limitation on DataNode and I think an important problem to fix. Also +1 for fixing the reads first. I just skimmed through the patch. will read it more thoroughly. There are various performance gains mentioned in the Jira with thread pool even for common case with small number of readers.. I don't see how that is possible. Any benchmarks would be great. Thread pools do have resource benefits (threads, epoll descriptors etc) even in normal case. The way I see the problem and the solution : - Problem : Datanode should gracefully handle thousands of simultaneous readers. Implementation : - Use a thread pool to serve data. -- I think it is better to use something like Netty rather than worry about all the selector handling as in this patch. -- I might implement a version by reusing the existing code. will update in couple of weeks. - Most of the datanode installations have multiple disks (some even in double digits). I think it is very important to have separate pools for each partition. Otherwise, each disk will be accessed only as much as the slowest disk (when DN has enough load). That would be a huge loss of available disk bandwidth in the exact scenarios this feature is supposed help. 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] Commented: (HDFS-729) fsck option to list only corrupted files
[ https://issues.apache.org/jira/browse/HDFS-729?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12806459#action_12806459 ] Raghu Angadi commented on HDFS-729: --- As I understand, list of files returned in one call will not have duplicates. 500 is a lot.. note that these are files with 'hard-corruption', ie, HDFS could not repair them. Once a cluster has so many corrupt files, I would think there would be a lot more urgent things to worry about than finding rest of the corrupt files. In practice, most likely reason for such a scenario would be a large number of datanodes go missing. fsck option to list only corrupted files Key: HDFS-729 URL: https://issues.apache.org/jira/browse/HDFS-729 Project: Hadoop HDFS Issue Type: Improvement Reporter: dhruba borthakur Assignee: dhruba borthakur Attachments: badFiles.txt, badFiles2.txt, corruptFiles.txt An option to fsck to list only corrupted files will be very helpful for frequent monitoring. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-755) Read multiple checksum chunks at once in DFSInputStream
[ https://issues.apache.org/jira/browse/HDFS-755?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12793113#action_12793113 ] Raghu Angadi commented on HDFS-755: --- User code should use buffering for application specific reasons. May be 'bufferSize' argument for FSInputStream is flawed to start with. My impression is that main purpose of this patch is to reduce a copy. keeping the large buffer prohibits that. Even when a sequencefile has very small records (avg 1k?), I think it might not have net negative effect. system calls are fairly cheap. There might not be a net negative effect on fairly small reads. Do you see FSInputChecker or DFSClient evolve to dynamically decide if a buffer should be used in near future? +1 for the patch itself. btw, I ran 'time bin/hadoop fs -cat 1gbfile /dev/null', with NN, DN, and the client on the same machine, but not been able to see improvement. will verify if I am really running the patch. Read multiple checksum chunks at once in DFSInputStream --- Key: HDFS-755 URL: https://issues.apache.org/jira/browse/HDFS-755 Project: Hadoop HDFS Issue Type: Improvement Components: hdfs client Affects Versions: 0.22.0 Reporter: Todd Lipcon Assignee: Todd Lipcon Attachments: alldata-hdfs.tsv, benchmark-8-256.png, benchmark.png, hdfs-755.txt, hdfs-755.txt, hdfs-755.txt, hdfs-755.txt, hdfs-755.txt HADOOP-3205 adds the ability for FSInputChecker subclasses to read multiple checksum chunks in a single call to readChunk. This is the HDFS-side use of that new feature. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-729) fsck option to list only corrupted files
[ https://issues.apache.org/jira/browse/HDFS-729?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12789653#action_12789653 ] Raghu Angadi commented on HDFS-729: --- Hi Dhruba, Patch looks good. What would be an alternative for badFile? A corruptFile, might not imply one of the blocks no good replica. But in general, corrupt file implies something that could not be recovered by filesystem. Between these two, my vote is for corruptFile. The current API is fine : one minor nit is that even when it returns less than numExpectedFiles it does not imply there aren't any more. In practice, it is probably good enough not to have this limit and always return up to (100 or 500 files).. This would simplify the interface. Your choice. I will double check if all the blocks in 'pri 2' bucket includes all the and only the blocks with no good replica left. fsck option to list only corrupted files Key: HDFS-729 URL: https://issues.apache.org/jira/browse/HDFS-729 Project: Hadoop HDFS Issue Type: Improvement Reporter: dhruba borthakur Assignee: dhruba borthakur Attachments: badFiles.txt, badFiles2.txt, corruptFiles.txt An option to fsck to list only corrupted files will be very helpful for frequent monitoring. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-755) Read multiple checksum chunks at once in DFSInputStream
[ https://issues.apache.org/jira/browse/HDFS-755?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12786788#action_12786788 ] Raghu Angadi commented on HDFS-755: --- The patch looks good except that BlockReader.in seems to buffered with a large buffer. Most of the data might still be going through extra copy. I haven't run the patch yet... Read multiple checksum chunks at once in DFSInputStream --- Key: HDFS-755 URL: https://issues.apache.org/jira/browse/HDFS-755 Project: Hadoop HDFS Issue Type: Improvement Components: hdfs client Affects Versions: 0.22.0 Reporter: Todd Lipcon Assignee: Todd Lipcon Attachments: hdfs-755.txt, hdfs-755.txt, hdfs-755.txt, hdfs-755.txt, hdfs-755.txt HADOOP-3205 adds the ability for FSInputChecker subclasses to read multiple checksum chunks in a single call to readChunk. This is the HDFS-side use of that new feature. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-767) Job failure due to BlockMissingException
[ https://issues.apache.org/jira/browse/HDFS-767?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12784390#action_12784390 ] Raghu Angadi commented on HDFS-767: --- I just briefly looked at it. Essentially, you are randomizing the retry times without actually increasing the number of retries (retry interval is increased). In that case, we will still see failures if the fetches take longer than a few seconds (a few seconds is quite possible, if you have a lot of threads reading from the disk, each client will take longer to read same amount of data). +1 for the patch, as a work around for some situations. Job failure due to BlockMissingException Key: HDFS-767 URL: https://issues.apache.org/jira/browse/HDFS-767 Project: Hadoop HDFS Issue Type: Improvement Reporter: Ning Zhang Assignee: Ning Zhang Attachments: HDFS-767.patch If a block is request by too many mappers/reducers (say, 3000) at the same time, a BlockMissingException is thrown because it exceeds the upper limit (I think 256 by default) of number of threads accessing the same block at the same time. The DFSClient wil catch that exception and retry 3 times after waiting for 3 seconds. Since the wait time is a fixed value, a lot of clients will retry at about the same time and a large portion of them get another failure. After 3 retries, there are about 256*4 = 1024 clients got the block. If the number of clients are more than that, the job will fail. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-767) Job failure due to BlockMissingException
[ https://issues.apache.org/jira/browse/HDFS-767?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12780765#action_12780765 ] Raghu Angadi commented on HDFS-767: --- I wasn't aware of limit on number of accessors for single block. Anyone knows reason behind such a restriction? Job failure due to BlockMissingException Key: HDFS-767 URL: https://issues.apache.org/jira/browse/HDFS-767 Project: Hadoop HDFS Issue Type: Improvement Reporter: Ning Zhang Assignee: Ning Zhang Attachments: HDFS-767.patch If a block is request by too many mappers/reducers (say, 3000) at the same time, a BlockMissingException is thrown because it exceeds the upper limit (I think 256 by default) of number of threads accessing the same block at the same time. The DFSClient wil catch that exception and retry 3 times after waiting for 3 seconds. Since the wait time is a fixed value, a lot of clients will retry at about the same time and a large portion of them get another failure. After 3 retries, there are about 256*4 = 1024 clients got the block. If the number of clients are more than that, the job will fail. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-767) Job failure due to BlockMissingException
[ https://issues.apache.org/jira/browse/HDFS-767?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12780813#action_12780813 ] Raghu Angadi commented on HDFS-767: --- There is a global limit on number of threads in DataNode. is that what you are referring to?. But there is no separate limit on a single block AFAIK. It is better not to have these limits (I should really get to implementing async IO for readers soon and I plan to. also there is a proposal to read data over RPC). For now, I think you should increase the limit. 256 is too small for most machines.. many large clusters have this limit set to 2k or more. Job failure due to BlockMissingException Key: HDFS-767 URL: https://issues.apache.org/jira/browse/HDFS-767 Project: Hadoop HDFS Issue Type: Improvement Reporter: Ning Zhang Assignee: Ning Zhang Attachments: HDFS-767.patch If a block is request by too many mappers/reducers (say, 3000) at the same time, a BlockMissingException is thrown because it exceeds the upper limit (I think 256 by default) of number of threads accessing the same block at the same time. The DFSClient wil catch that exception and retry 3 times after waiting for 3 seconds. Since the wait time is a fixed value, a lot of clients will retry at about the same time and a large portion of them get another failure. After 3 retries, there are about 256*4 = 1024 clients got the block. If the number of clients are more than that, the job will fail. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-777) A zero size file is created when SpaceQuota exceeded
[ https://issues.apache.org/jira/browse/HDFS-777?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12780353#action_12780353 ] Raghu Angadi commented on HDFS-777: --- To restate a comment in HDFS-172. The file is created by the implementation of -put command (not as a side effect of SpaceQuotaExcceed exception). I would think this is a bug about how -put (internally copyFromLocal) deals with errors. A zero size file is created when SpaceQuota exceeded Key: HDFS-777 URL: https://issues.apache.org/jira/browse/HDFS-777 Project: Hadoop HDFS Issue Type: Bug Affects Versions: 0.20.1 Environment: Debian GNU/Linux 5.0 hadoop-0.20.1 java version 1.6.0_12 Java(TM) SE Runtime Environment (build 1.6.0_12-b04) Java HotSpot(TM) Server VM (build 11.2-b01, mixed mode) Reporter: freestyler The issue can be reproduced by the following steps: $ cd hadoop $ bin/hadoop fs -mkdir /tmp $ bin/hadoop dfsadmin -setSpaceQuota 1m /tmp $ bin/hadoop fs -count -q /tmp none inf 1048576 10485761 0 0 hdfs://debian:9000/tmp $ ls -l hadoop-0.20.1-core.jar -rw-r--r-- 1 freestyler freestyler 2682112 2009-09-02 04:59 hadoop-0.20.1-core.jar $ bin/hadoop fs -put hadoop-0.20.1-core.jar /tmp/test.jar {quote} 09/11/19 12:09:35 WARN hdfs.DFSClient: DataStreamer Exception: org.apache.hadoop.hdfs.protocol.DSQuotaExceededException: org.apache.hadoop.hdfs.protocol.DSQuotaExceededException: The DiskSpace quota of /tmp is exceeded: quota=1048576 diskspace consumed=128.0m at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method) at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:39) at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:27) at java.lang.reflect.Constructor.newInstance(Constructor.java:513) at org.apache.hadoop.ipc.RemoteException.instantiateException(RemoteException.java:96) at org.apache.hadoop.ipc.RemoteException.unwrapRemoteException(RemoteException.java:58) at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.locateFollowingBlock(DFSClient.java:2906) at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.nextBlockOutputStream(DFSClient.java:2786) at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.access$2000(DFSClient.java:2076) at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSClient.java:2262) Caused by: org.apache.hadoop.ipc.RemoteException: org.apache.hadoop.hdfs.protocol.DSQuotaExceededException: The DiskSpace quota of /tmp is exceeded: quota=1048576 diskspace consumed=128.0m at org.apache.hadoop.hdfs.server.namenode.INodeDirectoryWithQuota.verifyQuota(INodeDirectoryWithQuota.java:156) at org.apache.hadoop.hdfs.server.namenode.INodeDirectoryWithQuota.updateNumItemsInTree(INodeDirectoryWithQuota.java:127) at org.apache.hadoop.hdfs.server.namenode.FSDirectory.updateCount(FSDirectory.java:859) at org.apache.hadoop.hdfs.server.namenode.FSDirectory.addBlock(FSDirectory.java:265) at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.allocateBlock(FSNamesystem.java:1436) at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getAdditionalBlock(FSNamesystem.java:1285) at org.apache.hadoop.hdfs.server.namenode.NameNode.addBlock(NameNode.java:422) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25) at java.lang.reflect.Method.invoke(Method.java:597) at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:508) at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:959) at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:955) at java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:396) at
[jira] Commented: (HDFS-763) DataBlockScanner reporting of bad blocks is slightly misleading
[ https://issues.apache.org/jira/browse/HDFS-763?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12777605#action_12777605 ] Raghu Angadi commented on HDFS-763: --- +1. totalErrors shown on 'blockScannerReport' now becomes same as number of verification failures, rather than all the errors seen. DataBlockScanner reporting of bad blocks is slightly misleading --- Key: HDFS-763 URL: https://issues.apache.org/jira/browse/HDFS-763 Project: Hadoop HDFS Issue Type: Bug Components: data-node Affects Versions: 0.20.1 Reporter: dhruba borthakur Assignee: dhruba borthakur Attachments: scanErrors.txt, scanErrors.txt, scanErrors.txt The Datanode generates a report of the period block scanning that verifies crcs. It reports something like the following: Scans since restart : 192266 Scan errors since restart : 33 Transient scan errors : 0 The statement saying that there were 33 errors is slightly midleading because these are not crc mismatches, rather the block was being deleted when the crc verification was about to happen. I propose that DataBlockScanner.totalScanErrors is not updated if the dataset.getFile(block) is null, i.e. the block is now deleted from the datanode. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-770) SocketTimeoutException: timeout while waiting for channel to be ready for read
[ https://issues.apache.org/jira/browse/HDFS-770?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12777612#action_12777612 ] Raghu Angadi commented on HDFS-770: --- From the datanode log : 2009-11-13 06:18:21,965 DEBUG org.apache.hadoop.ipc.RPC: Call: sendHeartbeat 14 2009-11-13 06:19:38,081 DEBUG org.apache.hadoop.ipc.Client: IPC Client (47) connection to dfs.hadoop.tsukku.solatis/127.0.0.1:9000 from hadoop: closed Note that there is no activity on DataNode for 77 seconds. There are number of possibilities, common one being GC. we haven't seen GC taking this long DN though. Assuming DN went to sleep for some reason, rest of the behaviour is expected. If you do expect such delays, what you need to increase is the read timeout for responder thread in DFSOutputStream (there is a config for generic read timeout that applies to sockets in many contexts). SocketTimeoutException: timeout while waiting for channel to be ready for read -- Key: HDFS-770 URL: https://issues.apache.org/jira/browse/HDFS-770 Project: Hadoop HDFS Issue Type: Bug Components: contrib/libhdfs, data-node, hdfs client, name-node Affects Versions: 0.20.1 Environment: Ubuntu Linux 8.04 Reporter: Leon Mergen Attachments: client.txt, datanode.txt, namenode.txt We're having issues with timeouts occurring in our client: for some reason, a timeout of 63000 milliseconds is triggered while writing HDFS data. Since we currently have a single-server setup, this results in our client terminating with a All datanodes are bad IOException. We're running all services, including the client, on our single server, so it cannot be a network error. The load on the client is extremely low during this period: only a few kilobytes a minute were being written around the time the error occured. After browsing a bit online, a lot of people talk about setting dfs.datanode.socket.write.timeout to 0 as a solution for this problem. Due to the low load of our system during this period, however, I do feel this is a real error and a timeout that should not be occurring. I have attached 3 logs of the namenode, datanode and client. It could be that this is related to http://issues.apache.org/jira/browse/HDFS-693 Any pointers on how I can assist to resolve this issue will be greatly appreciated. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-763) DataBlockScanner reporting of bad blocks is slightly misleading
[ https://issues.apache.org/jira/browse/HDFS-763?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=1215#action_1215 ] Raghu Angadi commented on HDFS-763: --- I don't think this needs an extra unit test. That stat affected here is only for display purposes and also not related to stats reported to stats servers like simon. DataBlockScanner reporting of bad blocks is slightly misleading --- Key: HDFS-763 URL: https://issues.apache.org/jira/browse/HDFS-763 Project: Hadoop HDFS Issue Type: Bug Components: data-node Affects Versions: 0.20.1 Reporter: dhruba borthakur Assignee: dhruba borthakur Attachments: scanErrors.txt, scanErrors.txt, scanErrors.txt The Datanode generates a report of the period block scanning that verifies crcs. It reports something like the following: Scans since restart : 192266 Scan errors since restart : 33 Transient scan errors : 0 The statement saying that there were 33 errors is slightly midleading because these are not crc mismatches, rather the block was being deleted when the crc verification was about to happen. I propose that DataBlockScanner.totalScanErrors is not updated if the dataset.getFile(block) is null, i.e. the block is now deleted from the datanode. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-763) DataBlockScanner reporting of bad blocks is slightly misleading
[ https://issues.apache.org/jira/browse/HDFS-763?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12776689#action_12776689 ] Raghu Angadi commented on HDFS-763: --- +1. yes. it should be incremented only for real errors. DataBlockScanner reporting of bad blocks is slightly misleading --- Key: HDFS-763 URL: https://issues.apache.org/jira/browse/HDFS-763 Project: Hadoop HDFS Issue Type: Bug Components: data-node Affects Versions: 0.20.1 Reporter: dhruba borthakur Assignee: dhruba borthakur The Datanode generates a report of the period block scanning that verifies crcs. It reports something like the following: Scans since restart : 192266 Scan errors since restart : 33 Transient scan errors : 0 The statement saying that there were 33 errors is slightly midleading because these are not crc mismatches, rather the block was being deleted when the crc verification was about to happen. I propose that DataBlockScanner.totalScanErrors is not updated if the dataset.getFile(block) is null, i.e. the block is now deleted from the datanode. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-732) HDFS files are ending up truncated
[ https://issues.apache.org/jira/browse/HDFS-732?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12771012#action_12771012 ] Raghu Angadi commented on HDFS-732: --- 0.20 seems to be setting 'closed' to true inside a finally. It would be better to fix the behaviour to be equivalent to 0.21. That said, I think contract of close is not the real issue here. Why isn't error from first close() not treated as hard error? bq. Even when the close call fails, DFS client does not go by itself and has to continue to provide consistent results. Do you mean DFS client does not serve other streams properly after this error? bq. Besides that, what is the purpose of recovering a file aborted during close? What is a use case for that? This changed quite some time back. This is the normal expected behaviour of most filesystems. A user's process or machine might die in the middle of writing and there is no use of throwing the data that is already written away. Christian, do you expect the actual error on datanodes while writing? I would be concerned about pipeline error detection whenever I see failure on all the three datanodes. Multiple bugs were fixed in this area. Please include any stacktrace around the messages in datanode logs (third datanode log would be very useful, but looks like you were not able to recover it). partial data recovered after such a failure is as expected. I agree, it would be better to make second invocation of close() return error as well and it would be good practice for app not to ignore error from the first close(). HDFS files are ending up truncated -- Key: HDFS-732 URL: https://issues.apache.org/jira/browse/HDFS-732 Project: Hadoop HDFS Issue Type: Bug Components: name-node Affects Versions: 0.20.1 Reporter: Christian Kunz We recently started to use hadoop-0.20.1 in our production environment (less than 2 weeks ago) and already had 3 instances of truncated files, more than we had for months using hadoop-0.18.3. Writing is done using libhdfs, although it rather seems to be a problem on the server side. I will post some relevant logs (they are too large to be put into the description) -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-738) Improve the disk utilization of HDFS
[ https://issues.apache.org/jira/browse/HDFS-738?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12770558#action_12770558 ] Raghu Angadi commented on HDFS-738: --- DN always picked disks in round-robin. Are you using vanilla HDFS? There were proposals to make it randomly pick a partition in the past but it was not committed for precisely the same reason mentioned above ( https://issues.apache.org/jira/browse/HDFS-325?focusedCommentId=12560037page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#action_12560037 ) +1 for smarter disk handler. It may not improve simple cases like the 24 writers on 16 disks test you mentioned, but in practice it should help more (especially with readers and un-even disk performance). Improve the disk utilization of HDFS Key: HDFS-738 URL: https://issues.apache.org/jira/browse/HDFS-738 Project: Hadoop HDFS Issue Type: Improvement Components: data-node Reporter: Zheng Shao HDFS data node currently assigns writers to disks randomly. This is good if there are a large number of readers/writers on a single data node, but might create a lot of contentions if there are only 4 readers/writers on a 4-disk node. A better way is to introduce a base class DiskHandler, for registering all disk operations (read/write), as well as getting the best disk for writing new blocks. A good strategy of the DiskHandler would be to distribute the load of the writes to the disks with more free spaces as well as less recent activities. There can be many strategies. This could help improve the HDFS multi-threaded write throughput a lot - we are seeing 25MB/s/disk on a 4-disk/node 4-node cluster (replication is already considered) given 8 concurrent writers (24 writers considering replication). I believe we can improve that to 2x. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-729) fsck option to list only corrupted files
[ https://issues.apache.org/jira/browse/HDFS-729?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12769356#action_12769356 ] Raghu Angadi commented on HDFS-729: --- Is this a regular fsck with less output? That might still be prohibitively long and expensive for regular poll. Name node already has a list of all the corrupt/missing/underreplicated blocks. It might be better to list those blocks and files they belong to (a user friendly -metaSave). Would be nice to have jsp as well. fsck option to list only corrupted files Key: HDFS-729 URL: https://issues.apache.org/jira/browse/HDFS-729 Project: Hadoop HDFS Issue Type: Improvement Reporter: dhruba borthakur Assignee: dhruba borthakur An option to fsck to list only corrupted files will be very helpful for frequent monitoring. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-729) fsck option to list only corrupted files
[ https://issues.apache.org/jira/browse/HDFS-729?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12769480#action_12769480 ] Raghu Angadi commented on HDFS-729: --- Using -metaSave as the tell me what is broke option is not obvious. (I'm not even user what -metaSave is supposed to mean!) I'd rather have '-listCorruptedFiles' and '-listCorruptedBlocks' or a -listCorrupted that takes an option of files or blocks with the default being files. yes. this is what I meant. metaSave was mentioned only as a quick reference (for a few that used it in the past). fsck option to list only corrupted files Key: HDFS-729 URL: https://issues.apache.org/jira/browse/HDFS-729 Project: Hadoop HDFS Issue Type: Improvement Reporter: dhruba borthakur Assignee: dhruba borthakur An option to fsck to list only corrupted files will be very helpful for frequent monitoring. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-127) DFSClient block read failures cause open DFSInputStream to become unusable
[ https://issues.apache.org/jira/browse/HDFS-127?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12757371#action_12757371 ] Raghu Angadi commented on HDFS-127: --- Raghu Finally, I am mainly proposing an explicit policy in a code comment. [...] Stack I agree that [...] etc., and that the 'failure' policy be explicitly stated. Is something blocking from adding a description of what 'failures' is meant for? Otherwise, IMHO it is just a magic fix that could lead to similar problems in future.. b/c it is hard for developers to keep and review contracts that they don't know.. that is the reason why this bug first appeared. Not a -1 from me. DFSClient block read failures cause open DFSInputStream to become unusable -- Key: HDFS-127 URL: https://issues.apache.org/jira/browse/HDFS-127 Project: Hadoop HDFS Issue Type: Bug Reporter: Igor Bolotin Attachments: 4681.patch We are using some Lucene indexes directly from HDFS and for quite long time we were using Hadoop version 0.15.3. When tried to upgrade to Hadoop 0.19 - index searches started to fail with exceptions like: 2008-11-13 16:50:20,314 WARN [Listener-4] [] DFSClient : DFS Read: java.io.IOException: Could not obtain block: blk_5604690829708125511_15489 file=/usr/collarity/data/urls-new/part-0/20081110-163426/_0.tis at org.apache.hadoop.hdfs.DFSClient$DFSInputStream.chooseDataNode(DFSClient.java:1708) at org.apache.hadoop.hdfs.DFSClient$DFSInputStream.blockSeekTo(DFSClient.java:1536) at org.apache.hadoop.hdfs.DFSClient$DFSInputStream.read(DFSClient.java:1663) at java.io.DataInputStream.read(DataInputStream.java:132) at org.apache.nutch.indexer.FsDirectory$DfsIndexInput.readInternal(FsDirectory.java:174) at org.apache.lucene.store.BufferedIndexInput.refill(BufferedIndexInput.java:152) at org.apache.lucene.store.BufferedIndexInput.readByte(BufferedIndexInput.java:38) at org.apache.lucene.store.IndexInput.readVInt(IndexInput.java:76) at org.apache.lucene.index.TermBuffer.read(TermBuffer.java:63) at org.apache.lucene.index.SegmentTermEnum.next(SegmentTermEnum.java:131) at org.apache.lucene.index.SegmentTermEnum.scanTo(SegmentTermEnum.java:162) at org.apache.lucene.index.TermInfosReader.scanEnum(TermInfosReader.java:223) at org.apache.lucene.index.TermInfosReader.get(TermInfosReader.java:217) at org.apache.lucene.index.SegmentTermDocs.seek(SegmentTermDocs.java:54) ... The investigation showed that the root of this issue is that we exceeded # of xcievers in the data nodes and that was fixed by changing configuration settings to 2k. However - one thing that bothered me was that even after datanodes recovered from overload and most of client servers had been shut down - we still observed errors in the logs of running servers. Further investigation showed that fix for HADOOP-1911 introduced another problem - the DFSInputStream instance might become unusable once number of failures over lifetime of this instance exceeds configured threshold. The fix for this specific issue seems to be trivial - just reset failure counter before reading next block (patch will be attached shortly). This seems to be also related to HADOOP-3185, but I'm not sure I really understand necessity of keeping track of failed block accesses in the DFS client. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-516) Low Latency distributed reads
[ https://issues.apache.org/jira/browse/HDFS-516?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12756214#action_12756214 ] Raghu Angadi commented on HDFS-516: --- When you get a change please point me to the streaming test/benchmark. bq. After I get those, my roadmap for this is to add checksum support and better DatanodeInfo caching. User groups would come after that. Unless you want to add checksums for better comparison, I don't think it is every essential. You need not spend much time on getting feature parity with HDFS. For more users to benefit from your work, I think it is better to extract the features that are complementary to HDFS. and we can work on getting those into HDFS. Low Latency distributed reads - Key: HDFS-516 URL: https://issues.apache.org/jira/browse/HDFS-516 Project: Hadoop HDFS Issue Type: New Feature Reporter: Jay Booth Priority: Minor Attachments: hdfs-516-20090912.patch Original Estimate: 168h Remaining Estimate: 168h I created a method for low latency random reads using NIO on the server side and simulated OS paging with LRU caching and lookahead on the client side. Some applications could include lucene searching (term-doc and doc-offset mappings are likely to be in local cache, thus much faster than nutch's current FsDirectory impl and binary search through record files (bytes at 1/2, 1/4, 1/8 marks are likely to be cached) -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-599) Improve Namenode robustness by prioritizing datanode heartbeats over client requests
[ https://issues.apache.org/jira/browse/HDFS-599?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12756267#action_12756267 ] Raghu Angadi commented on HDFS-599: --- true, essentially we need a simple way to find outliers. Average is not robust for this - if you have a couple of dead datanodes they'll skew the mean up. Dead nodes don't contribute to avg. Avg is updated only when there is a heartBeat. Also avg need not be just the last heartBeat. Some weight could be given to previous heartBeat intervals. something like : new_contribution = current_delay/2 + prev_contribution/2; prev_contribution = new_contribution; Improve Namenode robustness by prioritizing datanode heartbeats over client requests Key: HDFS-599 URL: https://issues.apache.org/jira/browse/HDFS-599 Project: Hadoop HDFS Issue Type: Improvement Components: name-node Reporter: dhruba borthakur Assignee: dhruba borthakur The namenode processes RPC requests from clients that are reading/writing to files as well as heartbeats/block reports from datanodes. Sometime, because of various reasons (Java GC runs, inconsistent performance of NFS filer that stores HDFS transacttion logs, etc), the namenode encounters transient slowness. For example, if the device that stores the HDFS transaction logs becomes sluggish, the Namenode's ability to process RPCs slows down to a certain extent. During this time, the RPCs from clients as well as the RPCs from datanodes suffer in similar fashion. If the underlying problem becomes worse, the NN's ability to process a heartbeat from a DN is severly impacted, thus causing the NN to declare that the DN is dead. Then the NN starts replicating blocks that used to reside on the now-declared-dead datanode. This adds extra load to the NN. Then the now-declared-datanode finally re-establishes contact with the NN, and sends a block report. The block report processing on the NN is another heavyweight activity, thus casing more load to the already overloaded namenode. My proposal is tha the NN should try its best to continue processing RPCs from datanodes and give lesser priority to serving client requests. The Datanode RPCs are integral to the consistency and performance of the Hadoop file system, and it is better to protect it at all costs. This will ensure that NN recovers from the hiccup much faster than what it does now. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-516) Low Latency distributed reads
[ https://issues.apache.org/jira/browse/HDFS-516?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12755627#action_12755627 ] Raghu Angadi commented on HDFS-516: --- bq. somehow, from 213 seconds to 112 seconds to stream 1GB from a remote HDFS file. This is 5MBps for HDFS and 9MBps for RadFS. Assuming 9MBps is probably 100Mbps network limit (is it?), 5MBps is too low for any FS. Since both reads are from the same physical files, this may not be hardware related. Could you check what is causing this delay? This might be affecting other benchmarks as well. Checking netstat on the client while this read is going on might help. Regd reads in RAD fs, does client fetch 32KB each time (single RPC) or does it pipeline (multiple requests for single client's stream)? @Todd, I essentially see this as POC of what could/should be improved in HDFS for addressing latency issues. Contrib makes sense, but I would not expect this to go to production in this form and should be marked 'Experimental'. The benchmarks also help greatly in setting priorities for features. I don't think this needs a branch since it does not touch core at all. Low Latency distributed reads - Key: HDFS-516 URL: https://issues.apache.org/jira/browse/HDFS-516 Project: Hadoop HDFS Issue Type: New Feature Reporter: Jay Booth Priority: Minor Attachments: hdfs-516-20090912.patch Original Estimate: 168h Remaining Estimate: 168h I created a method for low latency random reads using NIO on the server side and simulated OS paging with LRU caching and lookahead on the client side. Some applications could include lucene searching (term-doc and doc-offset mappings are likely to be in local cache, thus much faster than nutch's current FsDirectory impl and binary search through record files (bytes at 1/2, 1/4, 1/8 marks are likely to be cached) -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-516) Low Latency distributed reads
[ https://issues.apache.org/jira/browse/HDFS-516?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12755033#action_12755033 ] Raghu Angadi commented on HDFS-516: --- Hi Jay, will go through the patch. I hope a few others get a chance to look at it as well. Since it is contrib, it certainly makes it easier to include in trunk. I am not sure about 0.21 timeline. Low Latency distributed reads - Key: HDFS-516 URL: https://issues.apache.org/jira/browse/HDFS-516 Project: Hadoop HDFS Issue Type: New Feature Reporter: Jay Booth Priority: Minor Attachments: hdfs-516-20090912.patch Original Estimate: 168h Remaining Estimate: 168h I created a method for low latency random reads using NIO on the server side and simulated OS paging with LRU caching and lookahead on the client side. Some applications could include lucene searching (term-doc and doc-offset mappings are likely to be in local cache, thus much faster than nutch's current FsDirectory impl and binary search through record files (bytes at 1/2, 1/4, 1/8 marks are likely to be cached) -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-503) Implement erasure coding as a layer on HDFS
[ https://issues.apache.org/jira/browse/HDFS-503?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12753349#action_12753349 ] Raghu Angadi commented on HDFS-503: --- This seems pretty useful. since this is done outside HDFS, it is simpler for users to start experimenting. Say a file has 5 blocks with replication of 3 : total 15 blocks With this tool, replication could be reduced to 2, with one block for parity : total 10 + 2 blocks This is a savings of 20% space. Is this math correct? Detecting when to 'unRaid' : * The patch does this using a wrapper filesystem over HDFS. ** This requires file to be read by the client. ** More often than not, HDFS knows about irrecoverable blocks much before a client reads. ** this only semi-transparent to the users since they have to use the new filesystem. * Another completely transparent alternative could be to make 'RaidNode' ping NameNode for missing blocks. ** NameNode already knows about blocks that don't have any known good replica. And fetching that list is cheap. ** RaidNode could check if the corrupt/missing block belongs to any of its files. ** Rest of RaidNode pretty much remains the same as this patch. Implement erasure coding as a layer on HDFS --- Key: HDFS-503 URL: https://issues.apache.org/jira/browse/HDFS-503 Project: Hadoop HDFS Issue Type: New Feature Reporter: dhruba borthakur Assignee: dhruba borthakur Attachments: raid1.txt The goal of this JIRA is to discuss how the cost of raw storage for a HDFS file system can be reduced. Keeping three copies of the same data is very costly, especially when the size of storage is huge. One idea is to reduce the replication factor and do erasure coding of a set of blocks so that the over probability of failure of a block remains the same as before. Many forms of error-correcting codes are available, see http://en.wikipedia.org/wiki/Erasure_code. Also, recent research from CMU has described DiskReduce https://opencirrus.org/system/files/Gibson-OpenCirrus-June9-09.ppt. My opinion is to discuss implementation strategies that are not part of base HDFS, but is a layer on top of HDFS. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Updated: (HDFS-173) Recursively deleting a directory with millions of files makes NameNode unresponsive for other commands until the deletion completes
[ https://issues.apache.org/jira/browse/HDFS-173?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Raghu Angadi updated HDFS-173: -- Hadoop Flags: [Reviewed] Recursively deleting a directory with millions of files makes NameNode unresponsive for other commands until the deletion completes --- Key: HDFS-173 URL: https://issues.apache.org/jira/browse/HDFS-173 Project: Hadoop HDFS Issue Type: Bug Reporter: Suresh Srinivas Assignee: Suresh Srinivas Attachments: HDFS-173.1.patch, HDFS-173.2.patch, HDFS-173.3.patch, HDFS-173.patch Delete a directory with millions of files. This could take several minutes (observed 12 mins for 9 million files). While the operation is in progress FSNamesystem lock is held and the requests from clients are not handled until deletion completes. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-559) Work out the memory consumption of NN artifacts on a compressed pointer JVM
[ https://issues.apache.org/jira/browse/HDFS-559?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12747650#action_12747650 ] Raghu Angadi commented on HDFS-559: --- Thats why sizeof(BlockInfo) is only a partial picture. It does not include other objects associated with each BlockInfo. I wonder where the 8 byte difference comes from for BlockInfo between 32-bit and 64-bit-compressed-oop. Some special alignment requirements? Work out the memory consumption of NN artifacts on a compressed pointer JVM --- Key: HDFS-559 URL: https://issues.apache.org/jira/browse/HDFS-559 Project: Hadoop HDFS Issue Type: Improvement Components: name-node Affects Versions: 0.21.0 Environment: 64-bit and 32 bit JVMs, Java6u14 and jdk7 betas, with -XX compressed oops enabled/disabled Reporter: Steve Loughran Assignee: Steve Loughran Priority: Minor Following up HADOOP-1687, it would be nice to know the size of datatypes in under the java16u14 JVM, which offers compressed pointers. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-15) All replicas of a block end up on only 1 rack
[ https://issues.apache.org/jira/browse/HDFS-15?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12747719#action_12747719 ] Raghu Angadi commented on HDFS-15: -- +1. Looks good. one nit : the check needs to consider blocks with single replication.. so that they don't end up on needed replication list. All replicas of a block end up on only 1 rack - Key: HDFS-15 URL: https://issues.apache.org/jira/browse/HDFS-15 Project: Hadoop HDFS Issue Type: Bug Reporter: Hairong Kuang Assignee: Jitendra Nath Pandey Priority: Critical Attachments: HDFS-15.4.patch, HDFS-15.patch, HDFS-15.patch.2, HDFS-15.patch.3 HDFS replicas placement strategy guarantees that the replicas of a block exist on at least two racks when its replication factor is greater than one. But fsck still reports that the replicas of some blocks end up on one rack. The cause of the problem is that decommission and corruption handling only check the block's replication factor but not the rack requirement. When an over-replicated block loses a replica due to decomission, corruption, or heartbeat lost, namenode does not take any action to guarantee that remaining replicas are on different racks. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-15) All replicas of a block end up on only 1 rack
[ https://issues.apache.org/jira/browse/HDFS-15?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12745640#action_12745640 ] Raghu Angadi commented on HDFS-15: -- I am in favor of not having another list. I looked at the patch briefly, not sure if having extra set reduces new code.. Are you implying newer proposal has less changes? It is better to have 'neededReplicationList' to have all the block that need replication. May be 'UnderReplicatedBlocks' could be renamed. All replicas of a block end up on only 1 rack - Key: HDFS-15 URL: https://issues.apache.org/jira/browse/HDFS-15 Project: Hadoop HDFS Issue Type: Bug Reporter: Hairong Kuang Assignee: Jitendra Nath Pandey Priority: Critical Attachments: HDFS-15.patch HDFS replicas placement strategy guarantees that the replicas of a block exist on at least two racks when its replication factor is greater than one. But fsck still reports that the replicas of some blocks end up on one rack. The cause of the problem is that decommission and corruption handling only check the block's replication factor but not the rack requirement. When an over-replicated block loses a replica due to decomission, corruption, or heartbeat lost, namenode does not take any action to guarantee that remaining replicas are on different racks. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-173) Recursively deleting a directory with millions of files makes NameNode unresponsive for other commands until the deletion completes
[ https://issues.apache.org/jira/browse/HDFS-173?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12744660#action_12744660 ] Raghu Angadi commented on HDFS-173: --- First the target directory to be deleted will be removed from the directory tree, to prevent further changes in that directory. Just trying to see if there are some consistency issues : What about other references to files under the tree. E.g. if a file is still being written, there are references to it from lease manager, that could be an inconsistent view of the namesystem. I think this one can be handled What about log entry? Is the deletion log written at the beginning or the when the whole deletion is completed? (At the beginning seems more accurate).. Currently can recursive deletes fail for any reason in the middle (and are those rolled back?)? What is is the suspect for taking so long? Is it mainly the memory allocations? Recursively deleting a directory with millions of files makes NameNode unresponsive for other commands until the deletion completes --- Key: HDFS-173 URL: https://issues.apache.org/jira/browse/HDFS-173 Project: Hadoop HDFS Issue Type: Bug Reporter: Suresh Srinivas Assignee: Suresh Srinivas Delete a directory with millions of files. This could take several minutes (observed 12 mins for 9 million files). While the operation is in progress FSNamesystem lock is held and the requests from clients are not handled until deletion completes. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-516) Low Latency distributed reads
[ https://issues.apache.org/jira/browse/HDFS-516?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12738565#action_12738565 ] Raghu Angadi commented on HDFS-516: --- Jay, random read is an (increasingly more) important feature for HDFS to support. Currently latency is the biggest draw back. See HDFS-236. It is good to see your work on this. You could also run simple benchmark in HDFS-236 that does simple random read on a file and does not depend on a sequence file. From your architecture description this reduces the latency through following improvements : * Connection caching (Through RPC). * File Channel caching on Server * Local cache on the client. These are complementary to existing datanode. I might be a lot more simpler to add these features to existing implementation rather than requiring a user to choose an implementation based on the access. As such you will have to re-implement many features (BlockLocations on client, CRC verification, effcient bulk transfers AVRO-24, etc ) Low Latency distributed reads - Key: HDFS-516 URL: https://issues.apache.org/jira/browse/HDFS-516 Project: Hadoop HDFS Issue Type: New Feature Reporter: Jay Booth Priority: Minor Attachments: radfs.patch Original Estimate: 168h Remaining Estimate: 168h I created a method for low latency random reads using NIO on the server side and simulated OS paging with LRU caching and lookahead on the client side. Some applications could include lucene searching (term-doc and doc-offset mappings are likely to be in local cache, thus much faster than nutch's current FsDirectory impl and binary search through record files (bytes at 1/2, 1/4, 1/8 marks are likely to be cached) -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-487) HDFS should expose a fileid to uniquely identify a file
[ https://issues.apache.org/jira/browse/HDFS-487?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12732045#action_12732045 ] Raghu Angadi commented on HDFS-487: --- Dhruba, can you describe _how_ fileid helps with pluggable block placement. Many use cases are mentioned here but I can't find description of how. 'distcp -update' is not expected to be fool-proof (just like rsync). Even with fileid, should distcp store fileids in previous update (or source and destination files are expected to have same fileid?)? HDFS should expose a fileid to uniquely identify a file --- Key: HDFS-487 URL: https://issues.apache.org/jira/browse/HDFS-487 Project: Hadoop HDFS Issue Type: New Feature Reporter: dhruba borthakur Assignee: dhruba borthakur Attachments: fileid1.txt HDFS should expose a id that uniquely identifies a file. This helps in developing applications that work correctly even when files are moved from one directory to another. A typical use-case is to make the Pluggable Block Placement Policy (HDFS-385) use fileid instead of filename. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Updated: (HDFS-463) CreateEditsLog utility broken due to FSImage URL scheme check
[ https://issues.apache.org/jira/browse/HDFS-463?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Raghu Angadi updated HDFS-463: -- Tags: CreateEditLog FSImage Resolution: Fixed Hadoop Flags: [Reviewed] Status: Resolved (was: Patch Available) I just committed this. Thanks Suresh. CreateEditsLog utility broken due to FSImage URL scheme check - Key: HDFS-463 URL: https://issues.apache.org/jira/browse/HDFS-463 Project: Hadoop HDFS Issue Type: Bug Components: tools Affects Versions: 0.21.0 Reporter: Suresh Srinivas Assignee: Suresh Srinivas Fix For: 0.21.0 Attachments: HDFS-463.patch, HDFS-463.patch, HDFS-463.patch CreateEditsLog creates editslog using FSImage class. With new URI scheme test added, the utility fails. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-487) HDFS should expose a fileid to uniquely identify a file
[ https://issues.apache.org/jira/browse/HDFS-487?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12731617#action_12731617 ] Raghu Angadi commented on HDFS-487: --- Given that there is mapping from id to the file, I am not sure if it really works like an id. Can you explain the use case bit more? (I haven't read the patch for pluggable block placement) When I first read the title of the jira, thought this might be about file handles... HDFS should expose a fileid to uniquely identify a file --- Key: HDFS-487 URL: https://issues.apache.org/jira/browse/HDFS-487 Project: Hadoop HDFS Issue Type: New Feature Reporter: dhruba borthakur Assignee: dhruba borthakur Attachments: fileid1.txt HDFS should expose a id that uniquely identifies a file. This helps in developing applications that work correctly even when files are moved from one directory to another. A typical use-case is to make the Pluggable Block Placement Policy (HDFS-385) use fileid instead of filename. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-487) HDFS should expose a fileid to uniquely identify a file
[ https://issues.apache.org/jira/browse/HDFS-487?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12731620#action_12731620 ] Raghu Angadi commented on HDFS-487: --- Given that there is mapping [...] should be 'there is NO mapping [...]' HDFS should expose a fileid to uniquely identify a file --- Key: HDFS-487 URL: https://issues.apache.org/jira/browse/HDFS-487 Project: Hadoop HDFS Issue Type: New Feature Reporter: dhruba borthakur Assignee: dhruba borthakur Attachments: fileid1.txt HDFS should expose a id that uniquely identifies a file. This helps in developing applications that work correctly even when files are moved from one directory to another. A typical use-case is to make the Pluggable Block Placement Policy (HDFS-385) use fileid instead of filename. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-463) CreateEditsLog utility broken due to FSImage URL scheme check
[ https://issues.apache.org/jira/browse/HDFS-463?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12731115#action_12731115 ] Raghu Angadi commented on HDFS-463: --- Looks like the fix might just be replacing 'new URI(edits_dir)' with 'new File(edits_dir).getAbsoluteFile().toURI()'. CreateEditsLog utility broken due to FSImage URL scheme check - Key: HDFS-463 URL: https://issues.apache.org/jira/browse/HDFS-463 Project: Hadoop HDFS Issue Type: Bug Components: tools Affects Versions: 0.21.0 Reporter: Suresh Srinivas Assignee: Suresh Srinivas Fix For: 0.21.0 Attachments: HDFS-463.patch, HDFS-463.patch CreateEditsLog creates editslog using FSImage class. With new URI scheme test added, the utility fails. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-463) CreateEditsLog utility broken due to FSImage URL scheme check
[ https://issues.apache.org/jira/browse/HDFS-463?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12731141#action_12731141 ] Raghu Angadi commented on HDFS-463: --- Does it imply CreateEditLog works with non-file URIs?... CreateEditsLog utility broken due to FSImage URL scheme check - Key: HDFS-463 URL: https://issues.apache.org/jira/browse/HDFS-463 Project: Hadoop HDFS Issue Type: Bug Components: tools Affects Versions: 0.21.0 Reporter: Suresh Srinivas Assignee: Suresh Srinivas Fix For: 0.21.0 Attachments: HDFS-463.patch, HDFS-463.patch CreateEditsLog creates editslog using FSImage class. With new URI scheme test added, the utility fails. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-463) CreateEditsLog utility broken due to FSImage URL scheme check
[ https://issues.apache.org/jira/browse/HDFS-463?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12731148#action_12731148 ] Raghu Angadi commented on HDFS-463: --- Right, you need the first hunk in your patch as well.. CreateEditsLog utility broken due to FSImage URL scheme check - Key: HDFS-463 URL: https://issues.apache.org/jira/browse/HDFS-463 Project: Hadoop HDFS Issue Type: Bug Components: tools Affects Versions: 0.21.0 Reporter: Suresh Srinivas Assignee: Suresh Srinivas Fix For: 0.21.0 Attachments: HDFS-463.patch, HDFS-463.patch CreateEditsLog creates editslog using FSImage class. With new URI scheme test added, the utility fails. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-397) Incorporate storage directories into EditLogFileInput/Output streams
[ https://issues.apache.org/jira/browse/HDFS-397?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12730464#action_12730464 ] Raghu Angadi commented on HDFS-397: --- Regd 1) : What about 'STORAGE_JSPOOL_DIR' which points to a different directory from 'current'. JSPOOL_FILE also 'happens' to be same but intention of original author might have been to be able change. We should remove these constants if these are not sued. bq. I assumed it's correct since it passed the unit tests correctly. hmm.. I don't think that is sufficient or wise. When I program I would like to know what I am doing and why it is correct at every line. Have a look for instance to line 1497 and following. Is it for the latest patch attached there? I am asking this mainly so that I don't have go through the big patch for HDFS-311. Incorporate storage directories into EditLogFileInput/Output streams Key: HDFS-397 URL: https://issues.apache.org/jira/browse/HDFS-397 Project: Hadoop HDFS Issue Type: Sub-task Reporter: Luca Telloli Assignee: Luca Telloli Attachments: HADOOP-6001.patch, HADOOP-6001.patch, HADOOP-6001.patch, HDFS-397.patch -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-445) pread() fails when cached block locations are no longer valid
[ https://issues.apache.org/jira/browse/HDFS-445?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12729808#action_12729808 ] Raghu Angadi commented on HDFS-445: --- +1. This is an important fix for clients that keep a file open for a long time (like HBase). pread() fails when cached block locations are no longer valid - Key: HDFS-445 URL: https://issues.apache.org/jira/browse/HDFS-445 Project: Hadoop HDFS Issue Type: Bug Reporter: Kan Zhang Assignee: Kan Zhang Attachments: 445-06.patch, 445-08.patch when cached block locations are no longer valid (e.g., datanodes restart on different ports), pread() will fail, whereas normal read() still succeeds through re-fetching of block locations from namenode (up to a max number of times). -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Updated: (HDFS-439) HADOOP-5961 is incorrectly committed.
[ https://issues.apache.org/jira/browse/HDFS-439?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Raghu Angadi updated HDFS-439: -- Resolution: Fixed Status: Resolved (was: Patch Available) I just committed this. HADOOP-5961 is incorrectly committed. - Key: HDFS-439 URL: https://issues.apache.org/jira/browse/HDFS-439 Project: Hadoop HDFS Issue Type: Bug Affects Versions: 0.21.0 Reporter: Raghu Angadi Assignee: Raghu Angadi Priority: Blocker Fix For: 0.21.0 Attachments: HDFS-439.patch I committed HADOOP-5961 even though one of the test failures was caused by the patch. I didn't check properly. I will attach a patch. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-473) TestHdfsProxy fails in Linux
[ https://issues.apache.org/jira/browse/HDFS-473?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12729818#action_12729818 ] Raghu Angadi commented on HDFS-473: --- HDFS-439 is committed. I think this jira can be closed. TestHdfsProxy fails in Linux Key: HDFS-473 URL: https://issues.apache.org/jira/browse/HDFS-473 Project: Hadoop HDFS Issue Type: Bug Components: contrib/hdfsproxy Environment: Linux hostname 2.6.9-55.ELsmp #1 SMP Fri Apr 20 16:36:54 EDT 2007 x86_64 x86_64 x86_64 GNU/Linux Reporter: Tsz Wo (Nicholas), SZE {noformat} test-junit: [junit] Running org.apache.hadoop.hdfsproxy.TestHdfsProxy [junit] Tests run: 1, Failures: 0, Errors: 1, Time elapsed: 4.397 sec [junit] Test org.apache.hadoop.hdfsproxy.TestHdfsProxy FAILED [junit] Running org.apache.hadoop.hdfsproxy.TestProxyUgiManager [junit] Tests run: 3, Failures: 0, Errors: 0, Time elapsed: 4.219 sec BUILD FAILED /home/tsz/hadoop/latest/build.xml:1022: The following error occurred while executing this line: /home/tsz/hadoop/latest/src/contrib/build.xml:48: The following error occurred while executing this line: /home/tsz/hadoop/latest/src/contrib/hdfsproxy/build.xml:224: Tests failed! {noformat} -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-297) Implement a pure Java CRC32 calculator
[ https://issues.apache.org/jira/browse/HDFS-297?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12728440#action_12728440 ] Raghu Angadi commented on HDFS-297: --- +1 for committing. pretty significant improvement in CPU. Implement a pure Java CRC32 calculator -- Key: HDFS-297 URL: https://issues.apache.org/jira/browse/HDFS-297 Project: Hadoop HDFS Issue Type: Improvement Reporter: Owen O'Malley Assignee: Todd Lipcon Attachments: crc32-results.txt, hadoop-5598-evil.txt, hadoop-5598-hybrid.txt, hadoop-5598.txt, hadoop-5598.txt, hdfs-297.txt, PureJavaCrc32.java, PureJavaCrc32.java, PureJavaCrc32.java, TestCrc32Performance.java, TestCrc32Performance.java, TestCrc32Performance.java, TestPureJavaCrc32.java We've seen a reducer writing 200MB to HDFS with replication = 1 spending a long time in crc calculation. In particular, it was spending 5 seconds in crc calculation out of a total of 6 for the write. I suspect that it is the java-jni border that is causing us grief. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.