Przeglądaj źródła

MAPREDUCE-5049. CombineFileInputFormat counts all compressed files non-splitable. (sandyr via tucu)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-1@1454137 13f79535-47bb-0310-9956-ffa450edef68
Alejandro Abdelnur 12 lat temu
rodzic
commit
61195255d9

+ 3 - 0
CHANGES.txt

@@ -531,6 +531,9 @@ Release 1.2.0 - unreleased
     MAPREDUCE-5038. old API CombineFileInputFormat missing fixes that are in 
     MAPREDUCE-5038. old API CombineFileInputFormat missing fixes that are in 
     new API. (sandyr via tucu)
     new API. (sandyr via tucu)
 
 
+    MAPREDUCE-5049. CombineFileInputFormat counts all compressed files 
+    non-splitable. (sandyr via tucu)
+
 Release 1.1.2 - 2013.01.30
 Release 1.1.2 - 2013.01.30
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 5 - 1
src/mapred/org/apache/hadoop/mapred/lib/CombineFileInputFormat.java

@@ -37,6 +37,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.io.compress.CompressionCodecFactory;
 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.NodeBase;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.NetworkTopology;
 
 
@@ -135,7 +136,10 @@ public abstract class CombineFileInputFormat<K, V>
   protected boolean isSplitable(FileSystem fs, Path file) {
   protected boolean isSplitable(FileSystem fs, Path file) {
     final CompressionCodec codec =
     final CompressionCodec codec =
       new CompressionCodecFactory(fs.getConf()).getCodec(file);
       new CompressionCodecFactory(fs.getConf()).getCodec(file);
-    return codec == null;
+    if (null == codec) {
+      return true;
+    }
+    return codec instanceof SplittableCompressionCodec;
   }
   }
   
   
   /**
   /**

+ 5 - 1
src/mapred/org/apache/hadoop/mapreduce/lib/input/CombineFileInputFormat.java

@@ -40,6 +40,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.io.compress.CompressionCodecFactory;
 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.InputFormat;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.JobContext;
@@ -147,7 +148,10 @@ public abstract class CombineFileInputFormat<K, V>
   protected boolean isSplitable(JobContext context, Path file) {
   protected boolean isSplitable(JobContext context, Path file) {
     final CompressionCodec codec =
     final CompressionCodec codec =
       new CompressionCodecFactory(context.getConfiguration()).getCodec(file);
       new CompressionCodecFactory(context.getConfiguration()).getCodec(file);
-    return codec == null;
+    if (null == codec) {
+      return true;
+    }
+    return codec instanceof SplittableCompressionCodec;
   }
   }
 
 
   /**
   /**