[
https://issues.apache.org/jira/browse/YARN-2484?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14130560#comment-14130560
]
Jason Lowe commented on YARN-2484:
----------------------------------
Thanks for the report and patch, Tsuyoshi!
I'm a bit concerned that calling close() after encountering an error could
itself throw another exception and obscure the original error. So we may want
to do something more like this instead:
{code}
try {
...
f.close();
f = null;
} finally {
IOUtils.cleanup(LOG, f);
}
{code}
Also in the read case I'm not sure we care about the result of close(), since
we have all of the data we need. I don't want to take down the RM because
there was an error closing a file that we completely read.
> FileSystemRMStateStore#readFile/writeFile should close
> FSData(In|Out)putStream in final block
> ---------------------------------------------------------------------------------------------
>
> Key: YARN-2484
> URL: https://issues.apache.org/jira/browse/YARN-2484
> Project: Hadoop YARN
> Issue Type: Bug
> Reporter: Tsuyoshi OZAWA
> Assignee: Tsuyoshi OZAWA
> Priority: Trivial
> Attachments: YARN-2484.1.patch
>
>
> File descriptors can leak if exceptions are thrown in these methods.
> {code}
> private byte[] readFile(Path inputPath, long len) throws Exception {
> FSDataInputStream fsIn = fs.open(inputPath);
> // state data will not be that "long"
> byte[] data = new byte[(int)len];
> fsIn.readFully(data);
> fsIn.close();
> return data;
> }
> {code}
> {code}
> private void writeFile(Path outputPath, byte[] data) throws Exception {
> Path tempPath =
> new Path(outputPath.getParent(), outputPath.getName() + ".tmp");
> FSDataOutputStream fsOut = null;
> // This file will be overwritten when app/attempt finishes for saving the
> // final status.
> fsOut = fs.create(tempPath, true);
> fsOut.write(data);
> fsOut.close();
> fs.rename(tempPath, outputPath);
> }
> {code}
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)