You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-issues@hadoop.apache.org by "Seb Mo (JIRA)" <ji...@apache.org> on 2016/06/13 22:23:03 UTC
[jira] [Commented] (HADOOP-13264) Hadoop HDFS - DFSOutputStream
close method fails to clean up resources in case no hdfs datanodes are
accessible
[ https://issues.apache.org/jira/browse/HADOOP-13264?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15328425#comment-15328425 ]
Seb Mo commented on HADOOP-13264:
---------------------------------
Linked to the related ticket on which a sudden failure causes the JVM to die with OutOfMemory. In that test I was trying to migrate a couple of million files while the datanode ran out of diskspace. This caused by client to die, although I was calling close on the stream, but since that failed, the mentioned map caused by JVM to die.
Created a simple test that attempts a write while the datanode is down.
> Hadoop HDFS - 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-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org