HBASE-3709 HFile compression not sharing configuration

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1086504 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Andrew Kyle Purtell 2011-03-29 07:46:54 +00:00
parent 1e005639e2
commit df3a53e764
2 changed files with 13 additions and 12 deletions

View File

@ -51,6 +51,7 @@ Release 0.91.0 - Unreleased
Andrew Purtell)
HBASE-3688 Setters of class HTableDescriptor do not work properly
HBASE-3702 Fix NPE in Exec method parameter serialization
HBASE-3709 HFile compression not sharing configuration
IMPROVEMENTS
HBASE-3290 Max Compaction Size (Nicolas Spiegelberg via Stack)

View File

@ -26,7 +26,6 @@ import java.io.OutputStream;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.io.compress.CodecPool;
import org.apache.hadoop.io.compress.CompressionCodec;
import org.apache.hadoop.io.compress.CompressionInputStream;
@ -83,12 +82,10 @@ public final class Compression {
@Override
CompressionCodec getCodec() {
if (lzoCodec == null) {
Configuration conf = new Configuration();
conf.setBoolean("hadoop.native.lib", true);
try {
Class<?> externalCodec =
ClassLoader.getSystemClassLoader().loadClass("com.hadoop.compression.lzo.LzoCodec");
lzoCodec = (CompressionCodec) ReflectionUtils.newInstance(externalCodec, conf);
lzoCodec = (CompressionCodec) ReflectionUtils.newInstance(externalCodec, getConf());
} catch (ClassNotFoundException e) {
throw new RuntimeException(e);
}
@ -102,10 +99,8 @@ public final class Compression {
@Override
DefaultCodec getCodec() {
if (codec == null) {
Configuration conf = new Configuration();
conf.setBoolean("hadoop.native.lib", true);
codec = new GzipCodec();
codec.setConf(conf);
codec.setConf(getConf());
}
return codec;
@ -145,6 +140,7 @@ public final class Compression {
}
};
private final Configuration conf;
private final String compressName;
// data input buffer size to absorb small reads from application.
private static final int DATA_IBUF_SIZE = 1 * 1024;
@ -152,19 +148,24 @@ public final class Compression {
private static final int DATA_OBUF_SIZE = 4 * 1024;
Algorithm(String name) {
this.conf = new Configuration();
this.conf.setBoolean("hadoop.native.lib", true);
this.compressName = name;
}
abstract CompressionCodec getCodec();
public Configuration getConf() {
return conf;
}
public InputStream createDecompressionStream(
InputStream downStream, Decompressor decompressor,
int downStreamBufferSize) throws IOException {
CompressionCodec codec = getCodec();
// Set the internal buffer size to read from down stream.
if (downStreamBufferSize > 0) {
Configurable c = (Configurable) codec;
c.getConf().setInt("io.file.buffer.size", downStreamBufferSize);
getConf().setInt("io.file.buffer.size", downStreamBufferSize);
}
CompressionInputStream cis =
codec.createInputStream(downStream, decompressor);
@ -184,8 +185,7 @@ public final class Compression {
else {
bos1 = downStream;
}
Configurable c = (Configurable) codec;
c.getConf().setInt("io.file.buffer.size", 32 * 1024);
getConf().setInt("io.file.buffer.size", 32 * 1024);
CompressionOutputStream cos =
codec.createOutputStream(bos1, compressor);
BufferedOutputStream bos2 =
@ -197,7 +197,7 @@ public final class Compression {
public Compressor getCompressor() {
CompressionCodec codec = getCodec();
if (codec != null) {
Compressor compressor = CodecPool.getCompressor(codec);
Compressor compressor = CodecPool.getCompressor(codec, getConf());
if (compressor != null) {
if (compressor.finished()) {
// Somebody returns the compressor to CodecPool but is still using