|
@@ -39,6 +39,7 @@ import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.fs.PathFilter;
|
|
|
import org.apache.hadoop.io.DataInputByteBuffer;
|
|
|
+import org.apache.hadoop.io.IOUtils;
|
|
|
import org.apache.hadoop.security.Credentials;
|
|
|
import org.apache.hadoop.security.token.delegation.DelegationKey;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
@@ -573,12 +574,16 @@ public class FileSystemRMStateStore extends RMStateStore {
|
|
|
}
|
|
|
|
|
|
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;
|
|
|
+ FSDataInputStream fsIn = null;
|
|
|
+ try {
|
|
|
+ fsIn = fs.open(inputPath);
|
|
|
+ // state data will not be that "long"
|
|
|
+ byte[] data = new byte[(int) len];
|
|
|
+ fsIn.readFully(data);
|
|
|
+ return data;
|
|
|
+ } finally {
|
|
|
+ IOUtils.cleanup(LOG, fsIn);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/*
|
|
@@ -592,10 +597,15 @@ public class FileSystemRMStateStore extends RMStateStore {
|
|
|
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);
|
|
|
+ try {
|
|
|
+ fsOut = fs.create(tempPath, true);
|
|
|
+ fsOut.write(data);
|
|
|
+ fsOut.close();
|
|
|
+ fsOut = null;
|
|
|
+ fs.rename(tempPath, outputPath);
|
|
|
+ } finally {
|
|
|
+ IOUtils.cleanup(LOG, fsOut);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/*
|