[ https://issues.apache.org/jira/browse/HADOOP-3998?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12680783#action_12680783 ]
dhruba borthakur commented on HADOOP-3998: ------------------------------------------ I think we do not need this fix for 0.18 branch. The DFSClient.close() method actually invokes the DFSOutputStream.close() on all existing open fle descriptors before setting DFSClient.clientRunning to false. So, this is not a problem. Similarly, the 0.18 version of processDatanodeError() always removes the datanode that caused the error form the pipeline (irrespective of whether the primary datanode datanode that did the leaseRecovery suceeded or not). > Got an exception from ClientFinalizer when the JT is terminated > --------------------------------------------------------------- > > Key: HADOOP-3998 > URL: https://issues.apache.org/jira/browse/HADOOP-3998 > Project: Hadoop Core > Issue Type: Bug > Components: dfs > Affects Versions: 0.19.0 > Reporter: Amar Kamat > Assignee: dhruba borthakur > Priority: Blocker > Fix For: 0.18.4, 0.19.2, 0.20.0 > > Attachments: closeAll.patch, closeAll.patch, closeAll.patch, > closeAll2.patch, closeAll3.patch > > > This happens when we terminate the JT using _control-C_. It throws the > following exception > {noformat} > Exception closing file my-file > java.io.IOException: Filesystem closed > at org.apache.hadoop.hdfs.DFSClient.checkOpen(DFSClient.java:193) > at org.apache.hadoop.hdfs.DFSClient.access$700(DFSClient.java:64) > at > org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.closeInternal(DFSClient.java:2868) > at > org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.close(DFSClient.java:2837) > at > org.apache.hadoop.hdfs.DFSClient$LeaseChecker.close(DFSClient.java:808) > at org.apache.hadoop.hdfs.DFSClient.close(DFSClient.java:205) > at > org.apache.hadoop.hdfs.DistributedFileSystem.close(DistributedFileSystem.java:253) > at > org.apache.hadoop.fs.FileSystem$Cache.closeAll(FileSystem.java:1367) > at org.apache.hadoop.fs.FileSystem.closeAll(FileSystem.java:234) > at > org.apache.hadoop.fs.FileSystem$ClientFinalizer.run(FileSystem.java:219) > {noformat} > Note that _my-file_ is some file used by the JT. > Also if there is some file renaming done, then the exception states that the > earlier file does not exist. I am not sure if this is a MR issue or a DFS > issue. Opening this issue for investigation. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.