hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From apurt...@apache.org
Subject svn commit: r1086512 - /hbase/branches/0.90/src/main/java/org/apache/hadoop/hbase/io/hfile/Compression.java
Date Tue, 29 Mar 2011 08:13:24 GMT
Author: apurtell
Date: Tue Mar 29 08:13:24 2011
New Revision: 1086512

URL: http://svn.apache.org/viewvc?rev=1086512&view=rev
Log:
HBASE-3709 HFile compression not sharing configuration; work around accidental use of CDH-ism

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

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=1086512&r1=1086511&r2=1086512&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 08:13:24 2011
@@ -25,6 +25,7 @@ import java.io.OutputStream;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.compress.CodecPool;
 import org.apache.hadoop.io.compress.CompressionCodec;
@@ -80,12 +81,13 @@ public final class Compression {
       private transient CompressionCodec lzoCodec;
 
       @Override
-      CompressionCodec getCodec() {
+      CompressionCodec getCodec(Configuration conf) {
         if (lzoCodec == null) {
           try {
             Class<?> externalCodec =
                 ClassLoader.getSystemClassLoader().loadClass("com.hadoop.compression.lzo.LzoCodec");
-            lzoCodec = (CompressionCodec) ReflectionUtils.newInstance(externalCodec, getConf());
+            lzoCodec = (CompressionCodec) ReflectionUtils.newInstance(externalCodec, 
+                new Configuration(conf));
           } catch (ClassNotFoundException e) {
             throw new RuntimeException(e);
           }
@@ -97,10 +99,10 @@ public final class Compression {
       private transient GzipCodec codec;
 
       @Override
-      DefaultCodec getCodec() {
+      DefaultCodec getCodec(Configuration conf) {
         if (codec == null) {
           codec = new GzipCodec();
-          codec.setConf(getConf());
+          codec.setConf(new Configuration(conf));
         }
 
         return codec;
@@ -109,7 +111,7 @@ public final class Compression {
 
     NONE("none") {
       @Override
-      DefaultCodec getCodec() {
+      DefaultCodec getCodec(Configuration conf) {
         return null;
       }
 
@@ -153,19 +155,16 @@ public final class Compression {
       this.compressName = name;
     }
 
-    abstract CompressionCodec getCodec();
-
-    public Configuration getConf() {
-      return conf;
-    }
+    abstract CompressionCodec getCodec(Configuration conf);
 
     public InputStream createDecompressionStream(
         InputStream downStream, Decompressor decompressor,
         int downStreamBufferSize) throws IOException {
-      CompressionCodec codec = getCodec();
+      CompressionCodec codec = getCodec(conf);
       // Set the internal buffer size to read from down stream.
       if (downStreamBufferSize > 0) {
-        getConf().setInt("io.file.buffer.size", downStreamBufferSize);
+        ((Configurable)codec).getConf().setInt("io.file.buffer.size",
+            downStreamBufferSize);
       }
       CompressionInputStream cis =
           codec.createInputStream(downStream, decompressor);
@@ -177,7 +176,7 @@ public final class Compression {
     public OutputStream createCompressionStream(
         OutputStream downStream, Compressor compressor, int downStreamBufferSize)
         throws IOException {
-      CompressionCodec codec = getCodec();
+      CompressionCodec codec = getCodec(conf);
       OutputStream bos1 = null;
       if (downStreamBufferSize > 0) {
         bos1 = new BufferedOutputStream(downStream, downStreamBufferSize);
@@ -185,7 +184,7 @@ public final class Compression {
       else {
         bos1 = downStream;
       }
-      getConf().setInt("io.file.buffer.size", 32 * 1024);
+      ((Configurable)codec).getConf().setInt("io.file.buffer.size", 32 * 1024);
       CompressionOutputStream cos =
           codec.createOutputStream(bos1, compressor);
       BufferedOutputStream bos2 =
@@ -195,9 +194,9 @@ public final class Compression {
     }
 
     public Compressor getCompressor() {
-      CompressionCodec codec = getCodec();
+      CompressionCodec codec = getCodec(conf);
       if (codec != null) {
-        Compressor compressor = CodecPool.getCompressor(codec, getConf());
+        Compressor compressor = CodecPool.getCompressor(codec);
         if (compressor != null) {
           if (compressor.finished()) {
             // Somebody returns the compressor to CodecPool but is still using
@@ -221,7 +220,7 @@ public final class Compression {
     }
 
     public Decompressor getDecompressor() {
-      CompressionCodec codec = getCodec();
+      CompressionCodec codec = getCodec(conf);
       if (codec != null) {
         Decompressor decompressor = CodecPool.getDecompressor(codec);
         if (decompressor != null) {



Mime
View raw message