[ 
https://issues.apache.org/jira/browse/HDFS-10549?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15413397#comment-15413397
 ] 

Yiqun Lin commented on HDFS-10549:
----------------------------------

I debug the test for {{TestCrcCorruption.testCorruptionDuringWrt}} in 
HDFS-6532, I found some IOExceptions after {{InterruptedException}} that threw 
in {{#DataStreamer#waitForAckedSeqno}}:
{code}
2016-08-09 18:08:41,466 [Async disk worker #0 for volume 
/Users/lyq/Documents/work-project/hadoop-trunk/hadoop/hadoop-hdfs-project/hadoop-hdfs/target/test/data/dfs/data/data11/current]
 INFO  impl.FsDatasetAsyncDiskService (FsDatasetAsyncDiskService.java:run(296)) 
- Deleted BP-1814578309-127.0.0.1-1470737305923 blk_1073741825_1001 file 
/Users/lyq/Documents/work-project/hadoop-trunk/hadoop/hadoop-hdfs-project/hadoop-hdfs/target/test/data/dfs/data/data11/current/BP-1814578309-127.0.0.1-1470737305923/current/rbw/blk_1073741825
2016-08-09 18:08:43,540 [Thread-0] ERROR hdfs.DFSClient 
(DFSClient.java:closeAllFilesBeingWritten(579)) - Failed to close file: 
/test_corruption_file with inode: 16387
java.nio.channels.ClosedByInterruptException
        at 
java.nio.channels.spi.AbstractInterruptibleChannel.end(AbstractInterruptibleChannel.java:202)
        at sun.nio.ch.SocketChannelImpl.write(SocketChannelImpl.java:478)
        at 
org.apache.hadoop.net.SocketOutputStream$Writer.performIO(SocketOutputStream.java:63)
        at 
org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:142)
        at 
org.apache.hadoop.net.SocketOutputStream.write(SocketOutputStream.java:159)
        at 
org.apache.hadoop.net.SocketOutputStream.write(SocketOutputStream.java:117)
        at java.io.BufferedOutputStream.write(BufferedOutputStream.java:122)
        at java.io.DataOutputStream.write(DataOutputStream.java:107)
        at org.apache.hadoop.hdfs.DFSPacket.writeTo(DFSPacket.java:194)
        at org.apache.hadoop.hdfs.DataStreamer.run(DataStreamer.java:660)
2016-08-09 18:08:52,210 [Thread-0] ERROR hdfs.DFSClient 
(DFSClient.java:closeAllFilesBeingWritten(588)) - 
{code}
It seems that this case was related to this jira.

As the error logs shows that the files are not completely closed, then it will 
cause memory leak. We should close files again when the exception happens. Post 
a new patch for this to simplified v001 patch. This is a high priority issue, 
can take a look for this, [~xiaochen] and [~andrew.wang]. 

Thanks.

> Memory leak if exception happens when closing files being written
> -----------------------------------------------------------------
>
>                 Key: HDFS-10549
>                 URL: https://issues.apache.org/jira/browse/HDFS-10549
>             Project: Hadoop HDFS
>          Issue Type: Bug
>          Components: hdfs-client
>    Affects Versions: 2.7.1
>            Reporter: Yiqun Lin
>            Assignee: Yiqun Lin
>         Attachments: HDFS-10549.001.patch
>
>
> As HADOOP-13264 memtioned, the code dfsClient.endFileLease(fileId) in 
> {{DFSOutputStream}} will not be executed when the IOException happened in 
> {{closeImpl()}}.
> {code}
>   public void close() throws IOException {
>     synchronized (this) {
>       try (TraceScope ignored =
>           dfsClient.newPathTraceScope("DFSOutputStream#close", src)) {
>         closeImpl();
>       }
>     }
>     dfsClient.endFileLease(fileId);
>     }
>   }
> {code}
> This will cause that the files not be closed in {{DFSClient}} and finally 
> lead to the memory leak. In {{DFSStripedOutputStream}}, it existed the same 
> problem.



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

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

Reply via email to