[jira] [Commented] (HDFS-200) In HDFS, sync() not yet guarantees data available to the new readers
[ https://issues.apache.org/jira/browse/HDFS-200?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13242382#comment-13242382 ] Uma Maheswara Rao G commented on HDFS-200: -- Hi Dhruba, Looks following code creating the problem in one special condition. {code} BlockInfo storedBlock = blocksMap.getStoredBlock(block); +if (storedBlock == null) { + // if the block with a WILDCARD generation stamp matches and the + // corresponding file is under construction, then accept this block. + // This block has a diferent generation stamp on the datanode + // because of a lease-recovery-attempt. + Block nblk = new Block(block.getBlockId()); + storedBlock = blocksMap.getStoredBlock(nblk); + if (storedBlock != null storedBlock.getINode() != null + (storedBlock.getGenerationStamp() = block.getGenerationStamp() || + storedBlock.getINode().isUnderConstruction())) { +NameNode.stateChangeLog.info(BLOCK* NameSystem.addStoredBlock: + + addStoredBlock request received for + block + on + + node.getName() + size + block.getNumBytes() + + and it belongs to a file under construction. ); + } else { +storedBlock = null; + } {code} Events are as follows. 1) DN1-DN2-DN3 are in pipeline with genstamp 1 2) Client completed writing and closed the file. 3) Now DN3 killed. 4) Now reopened the file in append. 5) Now Pipeline contains DN1-DN2 with genstamp 2 6) Client continues writing some more data. 7) Now DN3 started. Replica presents in current directory as this was already finalized before. 8) DN3 triggered blockReport. 9) Since this block with genstamp 1 is not there in BlocksMap, it is trying to get the block with WILDCARD. And able to get the block. Will contains newer genstamp(2). 10) Since the file is UnderConstruction, it will just accept the block and updates in BlocksMap. Problem is, if the client gets DN3 for read, they will fail because, NN may give the block ID with latest genstamp (2), and DN3 does not contain the block with genstamp 2. Of cource data also inconsistent. Thanks Uma In HDFS, sync() not yet guarantees data available to the new readers Key: HDFS-200 URL: https://issues.apache.org/jira/browse/HDFS-200 Project: Hadoop HDFS Issue Type: New Feature Affects Versions: 0.20-append Reporter: Tsz Wo (Nicholas), SZE Assignee: dhruba borthakur Priority: Blocker Fix For: 0.20-append, 0.20.205.0 Attachments: 4379_20081010TC3.java, HDFS-200.20-security.1.patch, Reader.java, Reader.java, ReopenProblem.java, Writer.java, Writer.java, checkLeases-fix-1.txt, checkLeases-fix-unit-test-1.txt, fsyncConcurrentReaders.txt, fsyncConcurrentReaders11_20.txt, fsyncConcurrentReaders12_20.txt, fsyncConcurrentReaders13_20.txt, fsyncConcurrentReaders14_20.txt, fsyncConcurrentReaders15_20.txt, fsyncConcurrentReaders16_20.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, fsyncConcurrentReaders5.txt, fsyncConcurrentReaders6.patch, fsyncConcurrentReaders9.patch, hadoop-stack-namenode-aa0-000-12.u.powerset.com.log.gz, hdfs-200-ryan-existing-file-fail.txt, hypertable-namenode.log.gz, namenode.log, namenode.log, reopen_test.sh In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file However, this feature is not yet implemented. Note that the operation 'flushed' is now called sync. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-200) In HDFS, sync() not yet guarantees data available to the new readers
[ https://issues.apache.org/jira/browse/HDFS-200?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13096247#comment-13096247 ] Suresh Srinivas commented on HDFS-200: -- +1 for the patch. In HDFS, sync() not yet guarantees data available to the new readers Key: HDFS-200 URL: https://issues.apache.org/jira/browse/HDFS-200 Project: Hadoop HDFS Issue Type: New Feature Affects Versions: 0.20-append Reporter: Tsz Wo (Nicholas), SZE Assignee: dhruba borthakur Priority: Blocker Fix For: 0.20-append Attachments: 4379_20081010TC3.java, HDFS-200.20-security.1.patch, Reader.java, Reader.java, ReopenProblem.java, Writer.java, Writer.java, checkLeases-fix-1.txt, checkLeases-fix-unit-test-1.txt, fsyncConcurrentReaders.txt, fsyncConcurrentReaders11_20.txt, fsyncConcurrentReaders12_20.txt, fsyncConcurrentReaders13_20.txt, fsyncConcurrentReaders14_20.txt, fsyncConcurrentReaders15_20.txt, fsyncConcurrentReaders16_20.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, fsyncConcurrentReaders5.txt, fsyncConcurrentReaders6.patch, fsyncConcurrentReaders9.patch, hadoop-stack-namenode-aa0-000-12.u.powerset.com.log.gz, hdfs-200-ryan-existing-file-fail.txt, hypertable-namenode.log.gz, namenode.log, namenode.log, reopen_test.sh In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file However, this feature is not yet implemented. Note that the operation 'flushed' is now called sync. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-200) In HDFS, sync() not yet guarantees data available to the new readers
[ https://issues.apache.org/jira/browse/HDFS-200?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13096250#comment-13096250 ] Suresh Srinivas commented on HDFS-200: -- I committed this change to 0.20-security. In HDFS, sync() not yet guarantees data available to the new readers Key: HDFS-200 URL: https://issues.apache.org/jira/browse/HDFS-200 Project: Hadoop HDFS Issue Type: New Feature Affects Versions: 0.20-append Reporter: Tsz Wo (Nicholas), SZE Assignee: dhruba borthakur Priority: Blocker Fix For: 0.20-append, 0.20.205.0 Attachments: 4379_20081010TC3.java, HDFS-200.20-security.1.patch, Reader.java, Reader.java, ReopenProblem.java, Writer.java, Writer.java, checkLeases-fix-1.txt, checkLeases-fix-unit-test-1.txt, fsyncConcurrentReaders.txt, fsyncConcurrentReaders11_20.txt, fsyncConcurrentReaders12_20.txt, fsyncConcurrentReaders13_20.txt, fsyncConcurrentReaders14_20.txt, fsyncConcurrentReaders15_20.txt, fsyncConcurrentReaders16_20.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, fsyncConcurrentReaders5.txt, fsyncConcurrentReaders6.patch, fsyncConcurrentReaders9.patch, hadoop-stack-namenode-aa0-000-12.u.powerset.com.log.gz, hdfs-200-ryan-existing-file-fail.txt, hypertable-namenode.log.gz, namenode.log, namenode.log, reopen_test.sh In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file However, this feature is not yet implemented. Note that the operation 'flushed' is now called sync. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (HDFS-200) In HDFS, sync() not yet guarantees data available to the new readers
[ https://issues.apache.org/jira/browse/HDFS-200?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12867748#action_12867748 ] Todd Lipcon commented on HDFS-200: -- Hey Sam, any unit test for that last fix? Or description of what was wrong with it? In HDFS, sync() not yet guarantees data available to the new readers Key: HDFS-200 URL: https://issues.apache.org/jira/browse/HDFS-200 Project: Hadoop HDFS Issue Type: New Feature Reporter: Tsz Wo (Nicholas), SZE Assignee: dhruba borthakur Priority: Blocker Attachments: 4379_20081010TC3.java, checkLeases-fix-1.txt, fsyncConcurrentReaders.txt, fsyncConcurrentReaders11_20.txt, fsyncConcurrentReaders12_20.txt, fsyncConcurrentReaders13_20.txt, fsyncConcurrentReaders14_20.txt, fsyncConcurrentReaders15_20.txt, fsyncConcurrentReaders16_20.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, fsyncConcurrentReaders5.txt, fsyncConcurrentReaders6.patch, fsyncConcurrentReaders9.patch, hadoop-stack-namenode-aa0-000-12.u.powerset.com.log.gz, hdfs-200-ryan-existing-file-fail.txt, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file However, this feature is not yet implemented. Note that the operation 'flushed' is now called sync. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-200) In HDFS, sync() not yet guarantees data available to the new readers
[ https://issues.apache.org/jira/browse/HDFS-200?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12867762#action_12867762 ] sam rash commented on HDFS-200: --- sorry, should be easy to make a unit test, but i didn't have time. the case is that any lease that expires and has more than one path associated with it will throw exceptions. the leaseManager calls internalRelease(lease, src) and expects only the lease for src to be removed by the call (hence it does its own loop on all the paths for a lease). However, with hdfs-200, this function removes *all* paths for the lease, not just the specified one. internalReleaseLeaseOne does just the single path. if I get a minute, I can make such a test case quickly and update the patch w/it In HDFS, sync() not yet guarantees data available to the new readers Key: HDFS-200 URL: https://issues.apache.org/jira/browse/HDFS-200 Project: Hadoop HDFS Issue Type: New Feature Reporter: Tsz Wo (Nicholas), SZE Assignee: dhruba borthakur Priority: Blocker Attachments: 4379_20081010TC3.java, checkLeases-fix-1.txt, fsyncConcurrentReaders.txt, fsyncConcurrentReaders11_20.txt, fsyncConcurrentReaders12_20.txt, fsyncConcurrentReaders13_20.txt, fsyncConcurrentReaders14_20.txt, fsyncConcurrentReaders15_20.txt, fsyncConcurrentReaders16_20.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, fsyncConcurrentReaders5.txt, fsyncConcurrentReaders6.patch, fsyncConcurrentReaders9.patch, hadoop-stack-namenode-aa0-000-12.u.powerset.com.log.gz, hdfs-200-ryan-existing-file-fail.txt, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file However, this feature is not yet implemented. Note that the operation 'flushed' is now called sync. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-200) In HDFS, sync() not yet guarantees data available to the new readers
[ https://issues.apache.org/jira/browse/HDFS-200?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12867766#action_12867766 ] Todd Lipcon commented on HDFS-200: -- cool, I agree with your assessment. Unit test should be good whenever you have a chance, thanks! In HDFS, sync() not yet guarantees data available to the new readers Key: HDFS-200 URL: https://issues.apache.org/jira/browse/HDFS-200 Project: Hadoop HDFS Issue Type: New Feature Reporter: Tsz Wo (Nicholas), SZE Assignee: dhruba borthakur Priority: Blocker Attachments: 4379_20081010TC3.java, checkLeases-fix-1.txt, fsyncConcurrentReaders.txt, fsyncConcurrentReaders11_20.txt, fsyncConcurrentReaders12_20.txt, fsyncConcurrentReaders13_20.txt, fsyncConcurrentReaders14_20.txt, fsyncConcurrentReaders15_20.txt, fsyncConcurrentReaders16_20.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, fsyncConcurrentReaders5.txt, fsyncConcurrentReaders6.patch, fsyncConcurrentReaders9.patch, hadoop-stack-namenode-aa0-000-12.u.powerset.com.log.gz, hdfs-200-ryan-existing-file-fail.txt, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file However, this feature is not yet implemented. Note that the operation 'flushed' is now called sync. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-200) In HDFS, sync() not yet guarantees data available to the new readers
[ https://issues.apache.org/jira/browse/HDFS-200?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12842921#action_12842921 ] stack commented on HDFS-200: Its not important but the last patch seems to have some dross on the end, patches for content under src/docs/cn* In HDFS, sync() not yet guarantees data available to the new readers Key: HDFS-200 URL: https://issues.apache.org/jira/browse/HDFS-200 Project: Hadoop HDFS Issue Type: New Feature Reporter: Tsz Wo (Nicholas), SZE Assignee: dhruba borthakur Priority: Blocker Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders11_20.txt, fsyncConcurrentReaders12_20.txt, fsyncConcurrentReaders13_20.txt, fsyncConcurrentReaders14_20.txt, fsyncConcurrentReaders15_20.txt, fsyncConcurrentReaders16_20.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, fsyncConcurrentReaders5.txt, fsyncConcurrentReaders6.patch, fsyncConcurrentReaders9.patch, hadoop-stack-namenode-aa0-000-12.u.powerset.com.log.gz, hdfs-200-ryan-existing-file-fail.txt, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file However, this feature is not yet implemented. Note that the operation 'flushed' is now called sync. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-200) In HDFS, sync() not yet guarantees data available to the new readers
[ https://issues.apache.org/jira/browse/HDFS-200?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12830863#action_12830863 ] Todd Lipcon commented on HDFS-200: -- bq. The change you proposed is an optimization, isn't it? The earlier patch should have done the job too. Yes, I think so :) I'll continue to look over the patch this week to see if I see anything else. Thanks again for posting this on the jira. In HDFS, sync() not yet guarantees data available to the new readers Key: HDFS-200 URL: https://issues.apache.org/jira/browse/HDFS-200 Project: Hadoop HDFS Issue Type: New Feature Reporter: Tsz Wo (Nicholas), SZE Assignee: dhruba borthakur Priority: Blocker Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders11_20.txt, fsyncConcurrentReaders12_20.txt, fsyncConcurrentReaders13_20.txt, fsyncConcurrentReaders14_20.txt, fsyncConcurrentReaders15_20.txt, fsyncConcurrentReaders16_20.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, fsyncConcurrentReaders5.txt, fsyncConcurrentReaders6.patch, fsyncConcurrentReaders9.patch, hadoop-stack-namenode-aa0-000-12.u.powerset.com.log.gz, hdfs-200-ryan-existing-file-fail.txt, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file However, this feature is not yet implemented. Note that the operation 'flushed' is now called sync. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-200) In HDFS, sync() not yet guarantees data available to the new readers
[ https://issues.apache.org/jira/browse/HDFS-200?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12830584#action_12830584 ] dhruba borthakur commented on HDFS-200: --- Hi Todd, I think your proposal might not work. The newInfo.getFileLength() that you are using is the length of the file as returned by the namenode. We are interested in a file that is currently being written into, thus its length is constantly changing at the datanode. In HDFS, sync() not yet guarantees data available to the new readers Key: HDFS-200 URL: https://issues.apache.org/jira/browse/HDFS-200 Project: Hadoop HDFS Issue Type: New Feature Reporter: Tsz Wo (Nicholas), SZE Assignee: dhruba borthakur Priority: Blocker Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders11_20.txt, fsyncConcurrentReaders12_20.txt, fsyncConcurrentReaders13_20.txt, fsyncConcurrentReaders14_20.txt, fsyncConcurrentReaders15_20.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, fsyncConcurrentReaders5.txt, fsyncConcurrentReaders6.patch, fsyncConcurrentReaders9.patch, hadoop-stack-namenode-aa0-000-12.u.powerset.com.log.gz, hdfs-200-ryan-existing-file-fail.txt, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file However, this feature is not yet implemented. Note that the operation 'flushed' is now called sync. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-200) In HDFS, sync() not yet guarantees data available to the new readers
[ https://issues.apache.org/jira/browse/HDFS-200?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12830586#action_12830586 ] Todd Lipcon commented on HDFS-200: -- Hey Dhruba. newInfo.getFileLength is indeed the NN's view of the length, which is why I think it should be consistent with lastLocated.getBlockSize - the NN's view of the last block. Would the NN ever return an 'inconsistent' LocatedBlocks such that the file size has changed but the last block size hasn't? In HDFS, sync() not yet guarantees data available to the new readers Key: HDFS-200 URL: https://issues.apache.org/jira/browse/HDFS-200 Project: Hadoop HDFS Issue Type: New Feature Reporter: Tsz Wo (Nicholas), SZE Assignee: dhruba borthakur Priority: Blocker Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders11_20.txt, fsyncConcurrentReaders12_20.txt, fsyncConcurrentReaders13_20.txt, fsyncConcurrentReaders14_20.txt, fsyncConcurrentReaders15_20.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, fsyncConcurrentReaders5.txt, fsyncConcurrentReaders6.patch, fsyncConcurrentReaders9.patch, hadoop-stack-namenode-aa0-000-12.u.powerset.com.log.gz, hdfs-200-ryan-existing-file-fail.txt, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file However, this feature is not yet implemented. Note that the operation 'flushed' is now called sync. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-200) In HDFS, sync() not yet guarantees data available to the new readers
[ https://issues.apache.org/jira/browse/HDFS-200?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12830327#action_12830327 ] Todd Lipcon commented on HDFS-200: -- Hey Dhruba, In the client-side length fetching code, you use check the LocatedBlock last = newInfo.get(newInfo.locatedBlockCount()-1) to fetch the size. However, this is only the last block in the file for cases where the file has fewer than prefetchSize blocks, right? Has anyone tried a test after setting block size to 1MB? The trunk append code solves this issue by adding a lastBlock field to LocatedBlocks. I imagine you avoided this to keep wire compatibility in this patch. -Todd In HDFS, sync() not yet guarantees data available to the new readers Key: HDFS-200 URL: https://issues.apache.org/jira/browse/HDFS-200 Project: Hadoop HDFS Issue Type: New Feature Reporter: Tsz Wo (Nicholas), SZE Assignee: dhruba borthakur Priority: Blocker Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders11_20.txt, fsyncConcurrentReaders12_20.txt, fsyncConcurrentReaders13_20.txt, fsyncConcurrentReaders14_20.txt, fsyncConcurrentReaders15_20.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, fsyncConcurrentReaders5.txt, fsyncConcurrentReaders6.patch, fsyncConcurrentReaders9.patch, hadoop-stack-namenode-aa0-000-12.u.powerset.com.log.gz, hdfs-200-ryan-existing-file-fail.txt, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file However, this feature is not yet implemented. Note that the operation 'flushed' is now called sync. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-200) In HDFS, sync() not yet guarantees data available to the new readers
[ https://issues.apache.org/jira/browse/HDFS-200?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12830334#action_12830334 ] Todd Lipcon commented on HDFS-200: -- Regarding above comment - I suppose it doesn't entirely matter, since you only care about getting the true file length if you're reading at the end of the file. So, when you get near the end (10 blocks away) you'll get the updated length. However, you'll still do useless getBlockInfo calls on earlier blocks in the file. Perhaps some check against the LocatedBlocks structure to see if the last block in the block list is actually the same as the last block of the file is in order? In HDFS, sync() not yet guarantees data available to the new readers Key: HDFS-200 URL: https://issues.apache.org/jira/browse/HDFS-200 Project: Hadoop HDFS Issue Type: New Feature Reporter: Tsz Wo (Nicholas), SZE Assignee: dhruba borthakur Priority: Blocker Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders11_20.txt, fsyncConcurrentReaders12_20.txt, fsyncConcurrentReaders13_20.txt, fsyncConcurrentReaders14_20.txt, fsyncConcurrentReaders15_20.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, fsyncConcurrentReaders5.txt, fsyncConcurrentReaders6.patch, fsyncConcurrentReaders9.patch, hadoop-stack-namenode-aa0-000-12.u.powerset.com.log.gz, hdfs-200-ryan-existing-file-fail.txt, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file However, this feature is not yet implemented. Note that the operation 'flushed' is now called sync. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-200) In HDFS, sync() not yet guarantees data available to the new readers
[ https://issues.apache.org/jira/browse/HDFS-200?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12830416#action_12830416 ] dhruba borthakur commented on HDFS-200: --- Hi Todd, I think your point of concern deems investigation. I will post a new patch pretty soon. Thanks for looking over this one. One option is to check if the size of the lastBlock in newInfo is lesser than the size of the previous block. if so, then it means that we have reached the last block. of the file; only then trigger the code to retrieve the last block size from the datanode. (For a HDFS file, all blocks except the last one can be partial.) In HDFS, sync() not yet guarantees data available to the new readers Key: HDFS-200 URL: https://issues.apache.org/jira/browse/HDFS-200 Project: Hadoop HDFS Issue Type: New Feature Reporter: Tsz Wo (Nicholas), SZE Assignee: dhruba borthakur Priority: Blocker Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders11_20.txt, fsyncConcurrentReaders12_20.txt, fsyncConcurrentReaders13_20.txt, fsyncConcurrentReaders14_20.txt, fsyncConcurrentReaders15_20.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, fsyncConcurrentReaders5.txt, fsyncConcurrentReaders6.patch, fsyncConcurrentReaders9.patch, hadoop-stack-namenode-aa0-000-12.u.powerset.com.log.gz, hdfs-200-ryan-existing-file-fail.txt, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file However, this feature is not yet implemented. Note that the operation 'flushed' is now called sync. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-200) In HDFS, sync() not yet guarantees data available to the new readers
[ https://issues.apache.org/jira/browse/HDFS-200?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12830417#action_12830417 ] dhruba borthakur commented on HDFS-200: --- I meant For a HDFS file, only the last block can be partial in size. In HDFS, sync() not yet guarantees data available to the new readers Key: HDFS-200 URL: https://issues.apache.org/jira/browse/HDFS-200 Project: Hadoop HDFS Issue Type: New Feature Reporter: Tsz Wo (Nicholas), SZE Assignee: dhruba borthakur Priority: Blocker Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders11_20.txt, fsyncConcurrentReaders12_20.txt, fsyncConcurrentReaders13_20.txt, fsyncConcurrentReaders14_20.txt, fsyncConcurrentReaders15_20.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, fsyncConcurrentReaders5.txt, fsyncConcurrentReaders6.patch, fsyncConcurrentReaders9.patch, hadoop-stack-namenode-aa0-000-12.u.powerset.com.log.gz, hdfs-200-ryan-existing-file-fail.txt, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file However, this feature is not yet implemented. Note that the operation 'flushed' is now called sync. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-200) In HDFS, sync() not yet guarantees data available to the new readers
[ https://issues.apache.org/jira/browse/HDFS-200?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12794224#action_12794224 ] Alex Newman commented on HDFS-200: -- Anything going on here? Can we expect this to be fixed by 0.21.0? In HDFS, sync() not yet guarantees data available to the new readers Key: HDFS-200 URL: https://issues.apache.org/jira/browse/HDFS-200 Project: Hadoop HDFS Issue Type: New Feature Reporter: Tsz Wo (Nicholas), SZE Assignee: dhruba borthakur Priority: Blocker Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders11_20.txt, fsyncConcurrentReaders12_20.txt, fsyncConcurrentReaders13_20.txt, fsyncConcurrentReaders14_20.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, fsyncConcurrentReaders5.txt, fsyncConcurrentReaders6.patch, fsyncConcurrentReaders9.patch, hadoop-stack-namenode-aa0-000-12.u.powerset.com.log.gz, hdfs-200-ryan-existing-file-fail.txt, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file However, this feature is not yet implemented. Note that the operation 'flushed' is now called sync. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-200) In HDFS, sync() not yet guarantees data available to the new readers
[ https://issues.apache.org/jira/browse/HDFS-200?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12758946#action_12758946 ] ryan rawson commented on HDFS-200: -- I'm not sure exactly what I'm seeing, here is some flow of events: namenode says when the regionserver closes a log: 2009-09-23 17:21:05,128 DEBUG org.apache.hadoop.hdfs.StateChange: *BLOCK* NameNode.blockReceived: from 10.10.21.38:50010 1 blocks. 2009-09-23 17:21:05,128 DEBUG org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.blockReceived: blk_8594965619504827451_4351 is received from 10.10.21.38:50010 2009-09-23 17:21:05,128 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.addStoredBlock: blockMap updated: 10.10.21.38:50010 is added to blk_859496561950482745 1_4351 size 573866 2009-09-23 17:21:05,130 DEBUG org.apache.hadoop.hdfs.StateChange: *BLOCK* NameNode.blockReceived: from 10.10.21.45:50010 1 blocks. 2009-09-23 17:21:05,130 DEBUG org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.blockReceived: blk_8594965619504827451_4351 is received from 10.10.21.45:50010 2009-09-23 17:21:05,130 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.addStoredBlock: blockMap updated: 10.10.21.45:50010 is added to blk_859496561950482745 1_4351 size 573866 2009-09-23 17:21:05,131 DEBUG org.apache.hadoop.hdfs.StateChange: *BLOCK* NameNode.blockReceived: from 10.10.21.32:50010 1 blocks. 2009-09-23 17:21:05,131 DEBUG org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.blockReceived: blk_8594965619504827451_4351 is received from 10.10.21.32:50010 2009-09-23 17:21:05,131 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.addStoredBlock: blockMap updated: 10.10.21.32:50010 is added to blk_859496561950482745 1_4351 size 573866 2009-09-23 17:21:05,131 DEBUG org.apache.hadoop.hdfs.StateChange: *DIR* NameNode.complete: /hbase/.logs/sv4borg32,60020,1253751520085/hlog.dat.1253751663228 for DFSClien t_-2129099062 2009-09-23 17:21:05,131 DEBUG org.apache.hadoop.hdfs.StateChange: DIR* NameSystem.completeFile: /hbase/.logs/sv4borg32,60020,1253751520085/hlog.dat.1253751663228 for DFS Client_-2129099062 2009-09-23 17:21:05,132 DEBUG org.apache.hadoop.hdfs.StateChange: DIR* FSDirectory.closeFile: /hbase/.logs/sv4borg32,60020,1253751520085/hlog.dat.1253751663228 with 1 bl ocks is persisted to the file system 2009-09-23 17:21:05,132 DEBUG org.apache.hadoop.hdfs.StateChange: DIR* NameSystem.completeFile: /hbase/.logs/sv4borg32,60020,1253751520085/hlog.dat.1253751663228 blockli st persisted So at this point we have 3 blocks, they have all checked in, right? then during logfile recovery we see: 2009-09-23 17:21:45,997 DEBUG org.apache.hadoop.hdfs.StateChange: *DIR* NameNode.append: file /hbase/.logs/sv4borg32,60020,1253751520085/hlog.dat.1253751663228 for DFSCl ient_-828773542 at 10.10.21.29 2009-09-23 17:21:45,997 DEBUG org.apache.hadoop.hdfs.StateChange: DIR* NameSystem.startFile: src=/hbase/.logs/sv4borg32,60020,1253751520085/hlog.dat.1253751663228, holde r=DFSClient_-828773542, clientMachine=10.10.21.29, replication=512, overwrite=false, append=true 2009-09-23 17:21:45,997 INFO org.apache.hadoop.hdfs.server.namenode.FSNamesystem: Number of transactions: 567 Total time for transactions(ms): 9Number of transactions ba tched in Syncs: 54 Number of syncs: 374 SyncTimes(ms): 12023 4148 3690 7663 2009-09-23 17:21:45,997 DEBUG org.apache.hadoop.hdfs.StateChange: UnderReplicationBlocks.update blk_8594965619504827451_4351 curReplicas 0 curExpectedReplicas 3 oldReplicas 0 oldExpectedReplicas 3 curPri 2 oldPri 2 2009-09-23 17:21:45,997 DEBUG org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.UnderReplicationBlock.update:blk_8594965619504827451_4351 has only 0 replicas and need 3 replicas so is added to neededReplications at priority level 2 2009-09-23 17:21:45,997 DEBUG org.apache.hadoop.hdfs.StateChange: DIR* NameSystem.appendFile: file /hbase/.logs/sv4borg32,60020,1253751520085/hlog.dat.1253751663228 for DFSClient_-828773542 at 10.10.21.29 block blk_8594965619504827451_4351 block size 573866 2009-09-23 17:21:45,997 INFO org.apache.hadoop.hdfs.server.namenode.FSNamesystem.audit: ugi=hadoop,hadoop ip=/10.10.21.29 cmd=append src=/hbase/.logs/sv4borg32,60020,1253751520085/hlog.dat.1253751663228 dst=nullperm=null 2009-09-23 17:21:47,265 DEBUG org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.UnderReplicationBlock.remove: Removing block blk_8594965619504827451_4351 from priority queue 2 2009-09-23 17:21:56,016 DEBUG org.apache.hadoop.hdfs.StateChange: *BLOCK* NameNode.addBlock: file /hbase/.logs/sv4borg32,60020,1253751520085/hlog.dat.1253751663228 for DFSClient_-828773542 2009-09-23 17:21:56,016 DEBUG org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.getAdditionalBlock: file /hbase/.logs/sv4borg32,60020,1253751520085/hlog.dat.1253751663228 for DFSClient_-828773542 2009-09-23 17:21:56,016
[jira] Commented: (HDFS-200) In HDFS, sync() not yet guarantees data available to the new readers
[ https://issues.apache.org/jira/browse/HDFS-200?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12758949#action_12758949 ] ryan rawson commented on HDFS-200: -- scratch the last, i was having some environment/library version problems. In HDFS, sync() not yet guarantees data available to the new readers Key: HDFS-200 URL: https://issues.apache.org/jira/browse/HDFS-200 Project: Hadoop HDFS Issue Type: New Feature Reporter: Tsz Wo (Nicholas), SZE Assignee: dhruba borthakur Priority: Blocker Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders11_20.txt, fsyncConcurrentReaders12_20.txt, fsyncConcurrentReaders13_20.txt, fsyncConcurrentReaders14_20.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, fsyncConcurrentReaders5.txt, fsyncConcurrentReaders6.patch, fsyncConcurrentReaders9.patch, hadoop-stack-namenode-aa0-000-12.u.powerset.com.log.gz, hdfs-200-ryan-existing-file-fail.txt, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file However, this feature is not yet implemented. Note that the operation 'flushed' is now called sync. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-200) In HDFS, sync() not yet guarantees data available to the new readers
[ https://issues.apache.org/jira/browse/HDFS-200?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12754043#action_12754043 ] dhruba borthakur commented on HDFS-200: --- @ryan, stack: thanks for the info. I will follow it up very soon. will keep you posted. In HDFS, sync() not yet guarantees data available to the new readers Key: HDFS-200 URL: https://issues.apache.org/jira/browse/HDFS-200 Project: Hadoop HDFS Issue Type: New Feature Reporter: Tsz Wo (Nicholas), SZE Assignee: dhruba borthakur Priority: Blocker Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders11_20.txt, fsyncConcurrentReaders12_20.txt, fsyncConcurrentReaders13_20.txt, fsyncConcurrentReaders14_20.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, fsyncConcurrentReaders5.txt, fsyncConcurrentReaders6.patch, fsyncConcurrentReaders9.patch, hadoop-stack-namenode-aa0-000-12.u.powerset.com.log.gz, hdfs-200-ryan-existing-file-fail.txt, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file However, this feature is not yet implemented. Note that the operation 'flushed' is now called sync. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-200) In HDFS, sync() not yet guarantees data available to the new readers
[ https://issues.apache.org/jira/browse/HDFS-200?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12754052#action_12754052 ] dhruba borthakur commented on HDFS-200: --- I think this is the scenario that you are facing: 1. The file is written for the first time and was not closed. The writer closed the file but at this time only one of the three replicas have checked in with the namenode. 2. The new writer invoked append() to write more data into the file. The new writer found the one remaining replica of the block, stamped a new generation stamp for this block, made it ready to receive new data for this file and lease recovery is successful. The stamping of the new generation stamp essentially invaidated the other two replicas of this blockthis block now has only one valid replica. The namenode won't start replicating this block till the block is full. If this sole datanode now goes down, then the file will be missing a block. This is what you folks encountered. One option is to set dfs.replication.min to 2. This will ensure that closing a file (step 1) will be successful only when at least two replcias of the block have checked in with the namenode. This should reduce the probability of this problem occuring. Another option is to set the replication factor of the hbase log file(s) to be greater than 3. In HDFS, sync() not yet guarantees data available to the new readers Key: HDFS-200 URL: https://issues.apache.org/jira/browse/HDFS-200 Project: Hadoop HDFS Issue Type: New Feature Reporter: Tsz Wo (Nicholas), SZE Assignee: dhruba borthakur Priority: Blocker Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders11_20.txt, fsyncConcurrentReaders12_20.txt, fsyncConcurrentReaders13_20.txt, fsyncConcurrentReaders14_20.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, fsyncConcurrentReaders5.txt, fsyncConcurrentReaders6.patch, fsyncConcurrentReaders9.patch, hadoop-stack-namenode-aa0-000-12.u.powerset.com.log.gz, hdfs-200-ryan-existing-file-fail.txt, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file However, this feature is not yet implemented. Note that the operation 'flushed' is now called sync. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-200) In HDFS, sync() not yet guarantees data available to the new readers
[ https://issues.apache.org/jira/browse/HDFS-200?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12748236#action_12748236 ] ryan rawson commented on HDFS-200: -- I've been testing this on my 20 node cluster here. Lease recovery can take a long time which is a bit of an issue. The sync seems to be pretty good overall, we are recovering most of the edits up until the last flush, and it's pretty responsive. However, I have discovered a new bug, the scenario is like so: - we roll the logs every 1MB (block size). - we now have 18 logs to recover. The first 17 were closed properly, only the last one was in mid-write. - during log recovery, the hbase master calls fs.append(f); out.close(); - But the master gets stuck at the out.close(); It can't seem to progress. Investigating the logs, it looks like the namenode 'forgets' about the other 2 replicas for the block (file is 1 block), and thus we are stuck until another replica comes back. I've attached logs, hadoop fsck, stack traces from hbase. In HDFS, sync() not yet guarantees data available to the new readers Key: HDFS-200 URL: https://issues.apache.org/jira/browse/HDFS-200 Project: Hadoop HDFS Issue Type: New Feature Reporter: Tsz Wo (Nicholas), SZE Assignee: dhruba borthakur Priority: Blocker Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders11_20.txt, fsyncConcurrentReaders12_20.txt, fsyncConcurrentReaders13_20.txt, fsyncConcurrentReaders14_20.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, fsyncConcurrentReaders5.txt, fsyncConcurrentReaders6.patch, fsyncConcurrentReaders9.patch, hadoop-stack-namenode-aa0-000-12.u.powerset.com.log.gz, hdfs-200-ryan-existing-file-fail.txt, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file However, this feature is not yet implemented. Note that the operation 'flushed' is now called sync. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-200) In HDFS, sync() not yet guarantees data available to the new readers
[ https://issues.apache.org/jira/browse/HDFS-200?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12734125#action_12734125 ] dhruba borthakur commented on HDFS-200: --- Hi Ruyue, your option of excluding specific datanodes (specified by the client) sounds reasonable. This might help in the case of network partitioning where a specific client loses access to a set of datanodes while the datanode is alive and well and is able to send heartbeats to the namenode. Can you pl create a separate JIRA for your prosposed fix and attach your patch there? Thanks. In HDFS, sync() not yet guarantees data available to the new readers Key: HDFS-200 URL: https://issues.apache.org/jira/browse/HDFS-200 Project: Hadoop HDFS Issue Type: New Feature Reporter: Tsz Wo (Nicholas), SZE Assignee: dhruba borthakur Priority: Blocker Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders11_20.txt, fsyncConcurrentReaders12_20.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, fsyncConcurrentReaders5.txt, fsyncConcurrentReaders6.patch, fsyncConcurrentReaders9.patch, hadoop-stack-namenode-aa0-000-12.u.powerset.com.log.gz, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file However, this feature is not yet implemented. Note that the operation 'flushed' is now called sync. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-200) In HDFS, sync() not yet guarantees data available to the new readers
[ https://issues.apache.org/jira/browse/HDFS-200?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12733502#action_12733502 ] Ruyue Ma commented on HDFS-200: --- to: dhruba borthakur This is not related to HDFS-4379. let me explain why. The problem is actually related to HDFS-xxx. The namenode waits for 10 minutes after losing heartbeats from a datanode to declare it dead. During this 10 minutes, the NN is free to choose the dead datanode as a possible replica for a newly allocated block. If during a write, the dfsclient sees that a block replica location for a newly allocated block is not-connectable, it re-requests the NN to get a fresh set of replica locations of the block. It tries this dfs.client.block.write.retries times (default 3), sleeping 6 seconds between each retry ( see DFSClient.nextBlockOutputStream). This setting works well when you have a reasonable size cluster; if u have only 4 datanodes in the cluster, every retry picks the dead-datanode and the above logic bails out. One solution is to change the value of dfs.client.block.write.retries to a much much larger value, say 200 or so. Better still, increase the number of nodes in ur cluster. Our modification: when getting block location from namenode, we give nn the excluded datanodes. The list of dead datanodes is only for one block allocation. +++ hadoop-new/src/hdfs/org/apache/hadoop/hdfs/DFSClient.java 2009-07-20 00:19:03.0 +0800 @@ -2734,6 +2734,7 @@ LocatedBlock lb = null; boolean retry = false; DatanodeInfo[] nodes; + DatanodeInfo[] exludedNodes = null; int count = conf.getInt(dfs.client.block.write.retries, 3); boolean success; do { @@ -2745,7 +2746,7 @@ success = false; long startTime = System.currentTimeMillis(); -lb = locateFollowingBlock(startTime); +lb = locateFollowingBlock(startTime, exludedNodes); block = lb.getBlock(); nodes = lb.getLocations(); @@ -2755,6 +2756,19 @@ success = createBlockOutputStream(nodes, clientName, false); if (!success) { + + LOG.info(Excluding node: + nodes[errorIndex]); + // Mark datanode as excluded + DatanodeInfo errorNode = nodes[errorIndex]; + if (exludedNodes != null) { +DatanodeInfo[] newExcludedNodes = new DatanodeInfo[exludedNodes.length + 1]; +System.arraycopy(exludedNodes, 0, newExcludedNodes, 0, exludedNodes.length); +newExcludedNodes[exludedNodes.length] = errorNode; +exludedNodes = newExcludedNodes; + } else { +exludedNodes = new DatanodeInfo[] { errorNode }; + } + LOG.info(Abandoning block + block); namenode.abandonBlock(block, src, clientName); In HDFS, sync() not yet guarantees data available to the new readers Key: HDFS-200 URL: https://issues.apache.org/jira/browse/HDFS-200 Project: Hadoop HDFS Issue Type: New Feature Reporter: Tsz Wo (Nicholas), SZE Assignee: dhruba borthakur Priority: Blocker Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders11_20.txt, fsyncConcurrentReaders12_20.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, fsyncConcurrentReaders5.txt, fsyncConcurrentReaders6.patch, fsyncConcurrentReaders9.patch, hadoop-stack-namenode-aa0-000-12.u.powerset.com.log.gz, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file However, this feature is not yet implemented. Note that the operation 'flushed' is now called sync. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HDFS-200) In HDFS, sync() not yet guarantees data available to the new readers
[ https://issues.apache.org/jira/browse/HDFS-200?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12730666#action_12730666 ] stack commented on HDFS-200: (Thanks for review Konstantin) In my last few test runs, NameNode has shut itself down with the below: {code} ... 009-07-14 00:17:46,586 DEBUG org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.removeStoredBlock: blk_-9156287469566772234_2527 from XX.XX.XX.142:51010 2009-07-14 00:17:46,586 DEBUG org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.removeStoredBlock: blk_-9181830129071396520_2355 from XX.XX.XX.142:51010 2009-07-14 00:17:46,586 DEBUG org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.removeStoredBlock: blk_-9205119721509648294_2410 from XX.XX.XX.142:51010 2009-07-14 00:17:46,586 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.processReport: block blk_-7011715647341740217_1 on XX.XX.XX.142:51010 size 47027149 does not belong to any file. 2009-07-14 00:17:46,586 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.addToInvalidates: blk_-7011715647341740217 is added to invalidSet of XX.XX.XX.142:51010 2009-07-14 00:17:46,586 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.processReport: block blk_-280166356715716926_1 on XX.XX.XX.142:51010 size 6487 does not belong to any file. 2009-07-14 00:17:46,586 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.addToInvalidates: blk_-280166356715716926 is added to invalidSet of XX.XX.XX.142:51010 2009-07-14 00:17:46,586 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.processReport: block blk_1532053033915429278_1 on XX.XX.XX.142:51010 size 3869 does not belong to any file. 2009-07-14 00:17:46,586 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.addToInvalidates: blk_1532053033915429278 is added to invalidSet of XX.XX.XX.142:51010 2009-07-14 00:17:47,303 WARN org.apache.hadoop.hdfs.server.namenode.FSNamesystem: ReplicationMonitor thread received Runtime exception. java.lang.IllegalStateException: generationStamp (=1) == GenerationStamp.WILDCARD_STAMP 2009-07-14 00:17:47,304 INFO org.apache.hadoop.hdfs.server.namenode.NameNode: SHUTDOWN_MSG: / SHUTDOWN_MSG: Shutting down NameNode at aa0-000-12.u.powerset.com/XX.XX.XX.139 / {code} My guess this is a bug only fellas with dfs.support.append=true set run in to? Here is code from ReplicationMonitor: {code} } catch (Throwable t) { LOG.warn(ReplicationMonitor thread received Runtime exception. + t); Runtime.getRuntime().exit(-1); } {code} Thats a rough call I'd say? There are no more detailed exceptions in NN log. Dig in more and stick what I find in another issue? In HDFS, sync() not yet guarantees data available to the new readers Key: HDFS-200 URL: https://issues.apache.org/jira/browse/HDFS-200 Project: Hadoop HDFS Issue Type: New Feature Reporter: Tsz Wo (Nicholas), SZE Assignee: dhruba borthakur Priority: Blocker Attachments: 4379_20081010TC3.java, fsyncConcurrentReaders.txt, fsyncConcurrentReaders11_20.txt, fsyncConcurrentReaders3.patch, fsyncConcurrentReaders4.patch, fsyncConcurrentReaders5.txt, fsyncConcurrentReaders6.patch, fsyncConcurrentReaders9.patch, hadoop-stack-namenode-aa0-000-12.u.powerset.com.log.gz, hypertable-namenode.log.gz, namenode.log, namenode.log, Reader.java, Reader.java, reopen_test.sh, ReopenProblem.java, Writer.java, Writer.java In the append design doc (https://issues.apache.org/jira/secure/attachment/12370562/Appends.doc), it says * A reader is guaranteed to be able to read data that was 'flushed' before the reader opened the file However, this feature is not yet implemented. Note that the operation 'flushed' is now called sync. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.