MAPREDUCE-5530. Fix compat with hadoop-1 in mapred.lib.CombinFileInputFormat by re-introducing isSplittable(FileSystem, Path) api and ensuring semantic compatibility. Contributed by Robert Kanter.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1528533 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Arun Murthy 2013-10-02 16:08:45 +00:00
parent d14584dec5
commit 07ee205038
2 changed files with 41 additions and 0 deletions

View File

@ -260,6 +260,11 @@ Release 2.1.2 - UNRELEASED
MAPREDUCE-5536. Fixed MR AM and JHS to respect MAPREDUCE-5536. Fixed MR AM and JHS to respect
mapreduce.jobhistory.webapp.https.address. (Omkar Vinit Joshi via vinodkv) mapreduce.jobhistory.webapp.https.address. (Omkar Vinit Joshi via vinodkv)
MAPREDUCE-5530. Fix compat with hadoop-1 in
mapred.lib.CombinFileInputFormat by re-introducing
isSplittable(FileSystem, Path) api and ensuring semantic compatibility.
(Robert Kanter via acmurthy)
Release 2.1.1-beta - 2013-09-23 Release 2.1.1-beta - 2013-09-23
INCOMPATIBLE CHANGES INCOMPATIBLE CHANGES

View File

@ -24,13 +24,19 @@ import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
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.CompressionCodecFactory;
import org.apache.hadoop.io.compress.SplittableCompressionCodec;
import org.apache.hadoop.mapred.InputFormat; import org.apache.hadoop.mapred.InputFormat;
import org.apache.hadoop.mapred.InputSplit; import org.apache.hadoop.mapred.InputSplit;
import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.RecordReader; import org.apache.hadoop.mapred.RecordReader;
import org.apache.hadoop.mapred.Reporter; import org.apache.hadoop.mapred.Reporter;
import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.TaskAttemptContext;
/** /**
@ -127,4 +133,34 @@ public abstract class CombineFileInputFormat<K, V>
return result.toArray(new FileStatus[result.size()]); return result.toArray(new FileStatus[result.size()]);
} }
/**
* Subclasses should avoid overriding this method and should instead only
* override {@link #isSplitable(FileSystem, Path)}. The implementation of
* this method simply calls the other method to preserve compatibility.
* @see <a href="https://issues.apache.org/jira/browse/MAPREDUCE-5530">
* MAPREDUCE-5530</a>
*
* @param context the job context
* @param file the file name to check
* @return is this file splitable?
*/
@InterfaceAudience.Private
@Override
protected boolean isSplitable(JobContext context, Path file) {
try {
return isSplitable(FileSystem.get(context.getConfiguration()), file);
}
catch (IOException ioe) {
throw new RuntimeException(ioe);
}
}
protected boolean isSplitable(FileSystem fs, Path file) {
final CompressionCodec codec =
new CompressionCodecFactory(fs.getConf()).getCodec(file);
if (null == codec) {
return true;
}
return codec instanceof SplittableCompressionCodec;
}
} }