You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2011/03/29 09:46:59 UTC

svn commit: r1086505 - in /hbase/branches/0.90: CHANGES.txt src/main/java/org/apache/hadoop/hbase/io/hfile/Compression.java

Author: apurtell
Date: Tue Mar 29 07:46:59 2011
New Revision: 1086505

URL: http://svn.apache.org/viewvc?rev=1086505&view=rev
Log:
HBASE-3709 HFile compression not sharing configuration

Modified:
    hbase/branches/0.90/CHANGES.txt
    hbase/branches/0.90/src/main/java/org/apache/hadoop/hbase/io/hfile/Compression.java

Modified: hbase/branches/0.90/CHANGES.txt
URL: http://svn.apache.org/viewvc/hbase/branches/0.90/CHANGES.txt?rev=1086505&r1=1086504&r2=1086505&view=diff
==============================================================================
--- hbase/branches/0.90/CHANGES.txt (original)
+++ hbase/branches/0.90/CHANGES.txt Tue Mar 29 07:46:59 2011
@@ -62,6 +62,7 @@ Release 0.90.2 - Unreleased
                method in TableOutputFormat
    HBASE-3686  ClientScanner skips too many rows on recovery if using scanner
                caching (Sean Sechrist via Stack)
+   HBASE-3709  HFile compression not sharing configuration
 
   IMPROVEMENTS
    HBASE-3542  MultiGet methods in Thrift

Modified: hbase/branches/0.90/src/main/java/org/apache/hadoop/hbase/io/hfile/Compression.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90/src/main/java/org/apache/hadoop/hbase/io/hfile/Compression.java?rev=1086505&r1=1086504&r2=1086505&view=diff
==============================================================================
--- hbase/branches/0.90/src/main/java/org/apache/hadoop/hbase/io/hfile/Compression.java (original)
+++ hbase/branches/0.90/src/main/java/org/apache/hadoop/hbase/io/hfile/Compression.java Tue Mar 29 07:46:59 2011
@@ -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