hadoop-common-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From tomwh...@apache.org
Subject svn commit: r525581 - in /lucene/hadoop/trunk: CHANGES.txt src/java/org/apache/hadoop/mapred/JobConf.java
Date Wed, 04 Apr 2007 19:12:26 GMT
Author: tomwhite
Date: Wed Apr  4 12:12:26 2007
New Revision: 525581

URL: http://svn.apache.org/viewvc?view=rev&rev=525581
Log:
HADOOP-1194.  Make compression style record level for map output compression.  Contributed
by Arun C Murthy.

Modified:
    lucene/hadoop/trunk/CHANGES.txt
    lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobConf.java

Modified: lucene/hadoop/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/CHANGES.txt?view=diff&rev=525581&r1=525580&r2=525581
==============================================================================
--- lucene/hadoop/trunk/CHANGES.txt (original)
+++ lucene/hadoop/trunk/CHANGES.txt Wed Apr  4 12:12:26 2007
@@ -100,6 +100,9 @@
 31. HADOOP-1163.  Fix ganglia metrics to aggregate metrics from different
     hosts properly.  (Michael Bieniosek via tomwhite)
 
+32. HADOOP-1194.  Make compression style record level for map output
+    compression.  (Arun C Murthy via tomwhite)
+
 
 Release 0.12.3 (not yet released)
 

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobConf.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobConf.java?view=diff&rev=525581&r1=525580&r2=525581
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobConf.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobConf.java Wed Apr  4 12:12:26
2007
@@ -334,12 +334,8 @@
    * @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);
-    }
+    String val = get("map.output.compression.type", "RECORD");
+    return SequenceFile.CompressionType.valueOf(val);
   }
   
   /**



Mime
View raw message