Seb Mo created HADOOP-13264:
-------------------------------

             Summary: DFSOutputStream close method fails to clean up resources 
in case no hdfs datanodes are accessible 
                 Key: HADOOP-13264
                 URL: https://issues.apache.org/jira/browse/HADOOP-13264
             Project: Hadoop Common
          Issue Type: Bug
    Affects Versions: 2.7.2
            Reporter: Seb Mo


Using:
hadoop-hdfs\2.7.2\hadoop-hdfs-2.7.2-sources.jar!\org\apache\hadoop\hdfs\DFSOutputStream.java

Close method fails when the client can't connect to any data nodes. When 
re-using the same DistributedFileSystem in the same JVM, if all the datanodes 
can't be accessed, then this causes a memory leak as the 
DFSClient#filesBeingWritten map is never cleared after that.


Here is a test program:

        public static void main(String args[]) throws Exception
        {
                final Configuration conf = new Configuration();
                conf.addResource(new FileInputStream(new 
File("core-site.xml")));
                conf.addResource(new FileInputStream(new 
File("hdfs-site.xml")));

                final DistributedFileSystem newFileSystem = 
(DistributedFileSystem)FileSystem.get(conf);
                OutputStream outputStream = null;
                try
                {
                        outputStream = newFileSystem.create(new 
Path("/user/ssmogos", "test1"));
                        outputStream.write("test".getBytes());
                }
                catch (IOException e)
                {
                        e.printStackTrace();//don't care about this
                }
                finally
                {
                        try
                        {
                                if (outputStream != null)
                                        outputStream.close();//now this one 
will fail to close the stream
                        }
                        catch (IOException e)
                        {
                                e.printStackTrace();//this will list the thrown 
exception from DFSOutputStream->flushInternal->checkClosed
                                //TODO the 
DFSOutputStream#close->dfsClient.endFileLease(fileId) is never getting closed
                        }
                }

                Field field = 
DFSClient.class.getDeclaredField("filesBeingWritten");
                field.setAccessible(true);
                System.out.print("THIS SHOULD BE EMPTY: " + 
field.get(newFileSystem.getClient()));



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

---------------------------------------------------------------------
To unsubscribe, e-mail: common-dev-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-dev-h...@hadoop.apache.org

Reply via email to