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:
parent
5c778cdee3
commit
d7f712cd42
@ -343,6 +343,9 @@ Trunk (unreleased changes)
|
|||||||
HADOOP-7437. IOUtils.copybytes will suppress the stream closure exceptions.
|
HADOOP-7437. IOUtils.copybytes will suppress the stream closure exceptions.
|
||||||
(Uma Maheswara Rao G via szetszwo)
|
(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
|
Release 0.22.0 - Unreleased
|
||||||
|
|
||||||
INCOMPATIBLE CHANGES
|
INCOMPATIBLE CHANGES
|
||||||
|
@ -27,6 +27,7 @@
|
|||||||
|
|
||||||
import org.apache.hadoop.classification.InterfaceAudience;
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.classification.InterfaceStability;
|
import org.apache.hadoop.classification.InterfaceStability;
|
||||||
|
import org.apache.hadoop.io.IOUtils;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Holds file metadata including type (regular file, or directory),
|
* Holds file metadata including type (regular file, or directory),
|
||||||
@ -82,15 +83,20 @@ public long getSerializedLength() {
|
|||||||
public InputStream serialize() throws IOException {
|
public InputStream serialize() throws IOException {
|
||||||
ByteArrayOutputStream bytes = new ByteArrayOutputStream();
|
ByteArrayOutputStream bytes = new ByteArrayOutputStream();
|
||||||
DataOutputStream out = new DataOutputStream(bytes);
|
DataOutputStream out = new DataOutputStream(bytes);
|
||||||
out.writeByte(fileType.ordinal());
|
try {
|
||||||
if (isFile()) {
|
out.writeByte(fileType.ordinal());
|
||||||
out.writeInt(blocks.length);
|
if (isFile()) {
|
||||||
for (int i = 0; i < blocks.length; i++) {
|
out.writeInt(blocks.length);
|
||||||
out.writeLong(blocks[i].getId());
|
for (int i = 0; i < blocks.length; i++) {
|
||||||
out.writeLong(blocks[i].getLength());
|
out.writeLong(blocks[i].getId());
|
||||||
|
out.writeLong(blocks[i].getLength());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
out.close();
|
||||||
|
out = null;
|
||||||
|
} finally {
|
||||||
|
IOUtils.closeStream(out);
|
||||||
}
|
}
|
||||||
out.close();
|
|
||||||
return new ByteArrayInputStream(bytes.toByteArray());
|
return new ByteArrayInputStream(bytes.toByteArray());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -31,7 +31,6 @@
|
|||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.io.SequenceFile.CompressionType;
|
import org.apache.hadoop.io.SequenceFile.CompressionType;
|
||||||
import org.apache.hadoop.io.compress.CompressionCodec;
|
import org.apache.hadoop.io.compress.CompressionCodec;
|
||||||
import org.apache.hadoop.util.Options;
|
|
||||||
import org.apache.hadoop.util.Progressable;
|
import org.apache.hadoop.util.Progressable;
|
||||||
import org.apache.hadoop.util.bloom.DynamicBloomFilter;
|
import org.apache.hadoop.util.bloom.DynamicBloomFilter;
|
||||||
import org.apache.hadoop.util.bloom.Filter;
|
import org.apache.hadoop.util.bloom.Filter;
|
||||||
@ -187,9 +186,14 @@ public synchronized void append(WritableComparable key, Writable val)
|
|||||||
public synchronized void close() throws IOException {
|
public synchronized void close() throws IOException {
|
||||||
super.close();
|
super.close();
|
||||||
DataOutputStream out = fs.create(new Path(dir, BLOOM_FILE_NAME), true);
|
DataOutputStream out = fs.create(new Path(dir, BLOOM_FILE_NAME), true);
|
||||||
bloomFilter.write(out);
|
try {
|
||||||
out.flush();
|
bloomFilter.write(out);
|
||||||
out.close();
|
out.flush();
|
||||||
|
out.close();
|
||||||
|
out = null;
|
||||||
|
} finally {
|
||||||
|
IOUtils.closeStream(out);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
@ -225,15 +229,20 @@ public Reader(FileSystem fs, String dirName, WritableComparator comparator,
|
|||||||
|
|
||||||
private void initBloomFilter(Path dirName,
|
private void initBloomFilter(Path dirName,
|
||||||
Configuration conf) {
|
Configuration conf) {
|
||||||
|
|
||||||
|
DataInputStream in = null;
|
||||||
try {
|
try {
|
||||||
FileSystem fs = dirName.getFileSystem(conf);
|
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 = new DynamicBloomFilter();
|
||||||
bloomFilter.readFields(in);
|
bloomFilter.readFields(in);
|
||||||
in.close();
|
in.close();
|
||||||
|
in = null;
|
||||||
} catch (IOException ioe) {
|
} catch (IOException ioe) {
|
||||||
LOG.warn("Can't open BloomFilter: " + ioe + " - fallback to MapFile.");
|
LOG.warn("Can't open BloomFilter: " + ioe + " - fallback to MapFile.");
|
||||||
bloomFilter = null;
|
bloomFilter = null;
|
||||||
|
} finally {
|
||||||
|
IOUtils.closeStream(in);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -62,8 +62,13 @@ public static int writeCompressedByteArray(DataOutput out,
|
|||||||
if (bytes != null) {
|
if (bytes != null) {
|
||||||
ByteArrayOutputStream bos = new ByteArrayOutputStream();
|
ByteArrayOutputStream bos = new ByteArrayOutputStream();
|
||||||
GZIPOutputStream gzout = new GZIPOutputStream(bos);
|
GZIPOutputStream gzout = new GZIPOutputStream(bos);
|
||||||
gzout.write(bytes, 0, bytes.length);
|
try {
|
||||||
gzout.close();
|
gzout.write(bytes, 0, bytes.length);
|
||||||
|
gzout.close();
|
||||||
|
gzout = null;
|
||||||
|
} finally {
|
||||||
|
IOUtils.closeStream(gzout);
|
||||||
|
}
|
||||||
byte[] buffer = bos.toByteArray();
|
byte[] buffer = bos.toByteArray();
|
||||||
int len = buffer.length;
|
int len = buffer.length;
|
||||||
out.writeInt(len);
|
out.writeInt(len);
|
||||||
|
@ -27,6 +27,8 @@
|
|||||||
import java.io.OutputStream;
|
import java.io.OutputStream;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.hadoop.io.IOUtils;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* An output stream that compresses into the BZip2 format (without the file
|
* An output stream that compresses into the BZip2 format (without the file
|
||||||
* header chars) into another stream.
|
* header chars) into another stream.
|
||||||
@ -727,8 +729,13 @@ public void finish() throws IOException {
|
|||||||
public void close() throws IOException {
|
public void close() throws IOException {
|
||||||
if (out != null) {
|
if (out != null) {
|
||||||
OutputStream outShadow = this.out;
|
OutputStream outShadow = this.out;
|
||||||
finish();
|
try {
|
||||||
outShadow.close();
|
finish();
|
||||||
|
outShadow.close();
|
||||||
|
outShadow = null;
|
||||||
|
} finally {
|
||||||
|
IOUtils.closeStream(outShadow);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
Loading…
Reference in New Issue
Block a user