Index: src/java/org/apache/hadoop/hbase/io/hfile/HFile.java =================================================================== --- src/java/org/apache/hadoop/hbase/io/hfile/HFile.java (revision 810243) +++ src/java/org/apache/hadoop/hbase/io/hfile/HFile.java (working copy) @@ -967,18 +967,26 @@ private ByteBuffer decompress(final long offset, final int compressedSize, final int decompressedSize) throws IOException { - Decompressor decompressor = this.compressAlgo.getDecompressor(); - // My guess is that the bounded range fis is needed to stop the - // decompressor reading into next block -- IIRC, it just grabs a - // bunch of data w/o regard to whether decompressor is coming to end of a - // decompression. - InputStream is = this.compressAlgo.createDecompressionStream( - new BoundedRangeFileInputStream(this.istream, offset, compressedSize), - decompressor, 0); - ByteBuffer buf = ByteBuffer.allocate(decompressedSize); - IOUtils.readFully(is, buf.array(), 0, buf.capacity()); - is.close(); - this.compressAlgo.returnDecompressor(decompressor); + + Decompressor decompressor = null; + + try { + decompressor = this.compressAlgo.getDecompressor(); + // My guess is that the bounded range fis is needed to stop the + // decompressor reading into next block -- IIRC, it just grabs a + // bunch of data w/o regard to whether decompressor is coming to end of a + // decompression. + InputStream is = this.compressAlgo.createDecompressionStream( + new BoundedRangeFileInputStream(this.istream, offset, compressedSize), + decompressor, 0); + ByteBuffer buf = ByteBuffer.allocate(decompressedSize); + IOUtils.readFully(is, buf.array(), 0, buf.capacity()); + is.close(); + } finally { + if (null != decompressor) { + this.compressAlgo.returnDecompressor(decompressor); + } + } return buf; }