瀏覽代碼

HADOOP-647. Permit map outputs to use a different compression type than job output. Contributed by Owen.

git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk@470209 13f79535-47bb-0310-9956-ffa450edef68
Doug Cutting 18 年之前
父節點
當前提交
bbae187bad
共有 3 個文件被更改,包括 27 次插入6 次删除
  1. 3 0
      CHANGES.txt
  2. 23 5
      src/java/org/apache/hadoop/mapred/JobConf.java
  3. 1 1
      src/java/org/apache/hadoop/mapred/MapTask.java

+ 3 - 0
CHANGES.txt

@@ -134,6 +134,9 @@ Trunk (unreleased changes)
     types, replace Vector with ArrayList, etc.
     (Konstantin Shvachko via cutting)
 
+38. HADOOP-647.  Permit map outputs to use a different compression
+    type than the job output.  (omalley via cutting)
+
 
 Release 0.7.2 - 2006-10-18
 

+ 23 - 5
src/java/org/apache/hadoop/mapred/JobConf.java

@@ -34,11 +34,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.conf.Configuration;
 
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.io.WritableComparator;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.*;
 import org.apache.hadoop.io.compress.CompressionCodec;
 
 import org.apache.hadoop.mapred.lib.IdentityMapper;
@@ -338,6 +334,28 @@ public class JobConf extends Configuration {
     return getBoolean("mapred.compress.map.output", false);
   }
 
+  /**
+   * Set the compression type for the map outputs.
+   * @param style NONE, RECORD, or BLOCK to control how the map outputs are 
+   *        compressed
+   */
+  public void setMapOutputCompressionType(SequenceFile.CompressionType style) {
+    set("map.output.compression.type", style.toString());
+  }
+  
+  /**
+   * Get the compression type for the map outputs.
+   * @return the compression type, defaulting to job output compression type
+   */
+  public SequenceFile.CompressionType getMapOutputCompressionType() {
+    String val = get("map.output.compression.type");
+    if (val == null) {
+      return SequenceFile.getCompressionType(this);
+    } else {
+      return SequenceFile.CompressionType.valueOf(val);
+    }
+  }
+  
   /**
    * Set the given class as the  compression codec for the map outputs.
    * @param codecClass the CompressionCodec class that will compress the 

+ 1 - 1
src/java/org/apache/hadoop/mapred/MapTask.java

@@ -137,7 +137,7 @@ class MapTask extends Task {
       CompressionType compressionType = CompressionType.NONE;
       if (job.getCompressMapOutput()) {
         // find the kind of compression to do, defaulting to record
-        compressionType = SequenceFile.getCompressionType(job);
+        compressionType = job.getMapOutputCompressionType();
 
         // find the right codec
         Class codecClass =