hadoop-common-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From t...@apache.org
Subject svn commit: r1454137 - in /hadoop/common/branches/branch-1: CHANGES.txt src/mapred/org/apache/hadoop/mapred/lib/CombineFileInputFormat.java src/mapred/org/apache/hadoop/mapreduce/lib/input/CombineFileInputFormat.java
Date Thu, 07 Mar 2013 22:33:50 GMT
Author: tucu
Date: Thu Mar  7 22:33:50 2013
New Revision: 1454137

URL: http://svn.apache.org/r1454137
Log:
MAPREDUCE-5049. CombineFileInputFormat counts all compressed files non-splitable. (sandyr
via tucu)

Modified:
    hadoop/common/branches/branch-1/CHANGES.txt
    hadoop/common/branches/branch-1/src/mapred/org/apache/hadoop/mapred/lib/CombineFileInputFormat.java
    hadoop/common/branches/branch-1/src/mapred/org/apache/hadoop/mapreduce/lib/input/CombineFileInputFormat.java

Modified: hadoop/common/branches/branch-1/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1/CHANGES.txt?rev=1454137&r1=1454136&r2=1454137&view=diff
==============================================================================
--- hadoop/common/branches/branch-1/CHANGES.txt (original)
+++ hadoop/common/branches/branch-1/CHANGES.txt Thu Mar  7 22:33:50 2013
@@ -531,6 +531,9 @@ Release 1.2.0 - unreleased
     MAPREDUCE-5038. old API CombineFileInputFormat missing fixes that are in 
     new API. (sandyr via tucu)
 
+    MAPREDUCE-5049. CombineFileInputFormat counts all compressed files 
+    non-splitable. (sandyr via tucu)
+
 Release 1.1.2 - 2013.01.30
 
   INCOMPATIBLE CHANGES

Modified: hadoop/common/branches/branch-1/src/mapred/org/apache/hadoop/mapred/lib/CombineFileInputFormat.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1/src/mapred/org/apache/hadoop/mapred/lib/CombineFileInputFormat.java?rev=1454137&r1=1454136&r2=1454137&view=diff
==============================================================================
--- hadoop/common/branches/branch-1/src/mapred/org/apache/hadoop/mapred/lib/CombineFileInputFormat.java
(original)
+++ hadoop/common/branches/branch-1/src/mapred/org/apache/hadoop/mapred/lib/CombineFileInputFormat.java
Thu Mar  7 22:33:50 2013
@@ -37,6 +37,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.io.compress.CompressionCodecFactory;
+import org.apache.hadoop.io.compress.SplittableCompressionCodec;
 import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.net.NetworkTopology;
 
@@ -135,7 +136,10 @@ public abstract class CombineFileInputFo
   protected boolean isSplitable(FileSystem fs, Path file) {
     final CompressionCodec codec =
       new CompressionCodecFactory(fs.getConf()).getCodec(file);
-    return codec == null;
+    if (null == codec) {
+      return true;
+    }
+    return codec instanceof SplittableCompressionCodec;
   }
   
   /**

Modified: hadoop/common/branches/branch-1/src/mapred/org/apache/hadoop/mapreduce/lib/input/CombineFileInputFormat.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1/src/mapred/org/apache/hadoop/mapreduce/lib/input/CombineFileInputFormat.java?rev=1454137&r1=1454136&r2=1454137&view=diff
==============================================================================
--- hadoop/common/branches/branch-1/src/mapred/org/apache/hadoop/mapreduce/lib/input/CombineFileInputFormat.java
(original)
+++ hadoop/common/branches/branch-1/src/mapred/org/apache/hadoop/mapreduce/lib/input/CombineFileInputFormat.java
Thu Mar  7 22:33:50 2013
@@ -40,6 +40,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.io.compress.CompressionCodecFactory;
+import org.apache.hadoop.io.compress.SplittableCompressionCodec;
 import org.apache.hadoop.mapreduce.InputFormat;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.JobContext;
@@ -147,7 +148,10 @@ public abstract class CombineFileInputFo
   protected boolean isSplitable(JobContext context, Path file) {
     final CompressionCodec codec =
       new CompressionCodecFactory(context.getConfiguration()).getCodec(file);
-    return codec == null;
+    if (null == codec) {
+      return true;
+    }
+    return codec instanceof SplittableCompressionCodec;
   }
 
   /**



Mime
View raw message