HADOOP-11349. RawLocalFileSystem leaks file descriptor while creating a file if creat succeeds but chmod fails. (Varun Saxena via Colin P. McCabe)

This commit is contained in:
Colin Patrick Mccabe 2014-12-09 14:31:44 -08:00
parent a2e07a5456
commit 03867eb1bb
2 changed files with 25 additions and 4 deletions

View File

@ -552,6 +552,10 @@ Release 2.7.0 - UNRELEASED
HADOOP-11378. Fix new findbugs warnings in hadoop-kms. (Li Lu via wheat9)
HADOOP-11349. RawLocalFileSystem leaks file descriptor while creating a
file if creat succeeds but chmod fails. (Varun Saxena via Colin P. McCabe)
Release 2.6.0 - 2014-11-18
INCOMPATIBLE CHANGES

View File

@ -41,6 +41,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.nativeio.NativeIO;
import org.apache.hadoop.util.Progressable;
import org.apache.hadoop.util.Shell;
@ -295,8 +296,16 @@ public class RawLocalFileSystem extends FileSystem {
FSDataOutputStream out = create(f,
overwrite, bufferSize, replication, blockSize, progress);
setPermission(f, permission);
return out;
boolean success = false;
try {
setPermission(f, permission);
success = true;
return out;
} finally {
if (!success) {
IOUtils.cleanup(LOG, out);
}
}
}
@Override
@ -306,8 +315,16 @@ public class RawLocalFileSystem extends FileSystem {
Progressable progress) throws IOException {
FSDataOutputStream out = create(f,
overwrite, false, bufferSize, replication, blockSize, progress);
setPermission(f, permission);
return out;
boolean success = false;
try {
setPermission(f, permission);
success = true;
return out;
} finally {
if (!success) {
IOUtils.cleanup(LOG, out);
}
}
}
@Override