|
@@ -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);
|
|
|
- bloomFilter.write(out);
|
|
|
- out.flush();
|
|
|
- out.close();
|
|
|
+ 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);
|
|
|
}
|
|
|
}
|
|
|
|