HADOOP-7090. Fix resource leaks in s3.INode, BloomMapFile, WritableUtils and CBZip2OutputStream. Contributed by Uma Maheswara Rao G

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1143149 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Tsz-wo Sze 2011-07-05 17:28:27 +00:00
parent 5c778cdee3
commit d7f712cd42
5 changed files with 46 additions and 16 deletions

View File

@ -343,6 +343,9 @@ Trunk (unreleased changes)
HADOOP-7437. IOUtils.copybytes will suppress the stream closure exceptions.
(Uma Maheswara Rao G via szetszwo)
HADOOP-7090. Fix resource leaks in s3.INode, BloomMapFile, WritableUtils
and CBZip2OutputStream. (Uma Maheswara Rao G via szetszwo)
Release 0.22.0 - Unreleased
INCOMPATIBLE CHANGES

View File

@ -27,6 +27,7 @@ import java.io.InputStream;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.io.IOUtils;
/**
* Holds file metadata including type (regular file, or directory),
@ -82,6 +83,7 @@ public class INode {
public InputStream serialize() throws IOException {
ByteArrayOutputStream bytes = new ByteArrayOutputStream();
DataOutputStream out = new DataOutputStream(bytes);
try {
out.writeByte(fileType.ordinal());
if (isFile()) {
out.writeInt(blocks.length);
@ -91,6 +93,10 @@ public class INode {
}
}
out.close();
out = null;
} finally {
IOUtils.closeStream(out);
}
return new ByteArrayInputStream(bytes.toByteArray());
}

View File

@ -31,7 +31,6 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.SequenceFile.CompressionType;
import org.apache.hadoop.io.compress.CompressionCodec;
import org.apache.hadoop.util.Options;
import org.apache.hadoop.util.Progressable;
import org.apache.hadoop.util.bloom.DynamicBloomFilter;
import org.apache.hadoop.util.bloom.Filter;
@ -187,9 +186,14 @@ public class BloomMapFile {
public synchronized void close() throws IOException {
super.close();
DataOutputStream out = fs.create(new Path(dir, BLOOM_FILE_NAME), true);
try {
bloomFilter.write(out);
out.flush();
out.close();
out = null;
} finally {
IOUtils.closeStream(out);
}
}
}
@ -225,15 +229,20 @@ public class BloomMapFile {
private void initBloomFilter(Path dirName,
Configuration conf) {
DataInputStream in = null;
try {
FileSystem fs = dirName.getFileSystem(conf);
DataInputStream in = fs.open(new Path(dirName, BLOOM_FILE_NAME));
in = fs.open(new Path(dirName, BLOOM_FILE_NAME));
bloomFilter = new DynamicBloomFilter();
bloomFilter.readFields(in);
in.close();
in = null;
} catch (IOException ioe) {
LOG.warn("Can't open BloomFilter: " + ioe + " - fallback to MapFile.");
bloomFilter = null;
} finally {
IOUtils.closeStream(in);
}
}

View File

@ -62,8 +62,13 @@ public final class WritableUtils {
if (bytes != null) {
ByteArrayOutputStream bos = new ByteArrayOutputStream();
GZIPOutputStream gzout = new GZIPOutputStream(bos);
try {
gzout.write(bytes, 0, bytes.length);
gzout.close();
gzout = null;
} finally {
IOUtils.closeStream(gzout);
}
byte[] buffer = bos.toByteArray();
int len = buffer.length;
out.writeInt(len);

View File

@ -27,6 +27,8 @@ package org.apache.hadoop.io.compress.bzip2;
import java.io.OutputStream;
import java.io.IOException;
import org.apache.hadoop.io.IOUtils;
/**
* An output stream that compresses into the BZip2 format (without the file
* header chars) into another stream.
@ -727,8 +729,13 @@ public class CBZip2OutputStream extends OutputStream implements BZip2Constants {
public void close() throws IOException {
if (out != null) {
OutputStream outShadow = this.out;
try {
finish();
outShadow.close();
outShadow = null;
} finally {
IOUtils.closeStream(outShadow);
}
}
}