浏览代码

MAPREDUCE-5794. Merging r1577281 from trunk to branch-2.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1577283 13f79535-47bb-0310-9956-ffa450edef68
Arpit Agarwal 11 年之前
父节点
当前提交
fd4d1d1d05

+ 3 - 0
hadoop-mapreduce-project/CHANGES.txt

@@ -86,6 +86,9 @@ Release 2.4.0 - UNRELEASED
     MAPREDUCE-5789. Average Reduce time is incorrect on Job Overview page
     (Rushabh S Shah via jlowe)
 
+    MAPREDUCE-5794. SliveMapper always uses default FileSystem. (szetszwo via
+    Arpit Agarwal)
+
 Release 2.3.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 0 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/DummyInputFormat.java

@@ -31,7 +31,6 @@ import org.apache.hadoop.mapred.Reporter;
 /**
  * A input format which returns one dummy key and value
  */
-@SuppressWarnings("deprecation")
 class DummyInputFormat implements InputFormat<Object, Object> {
 
   static class EmptySplit implements InputSplit {

+ 1 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/SliveMapper.java

@@ -42,7 +42,6 @@ import org.apache.hadoop.util.StringUtils;
  * that have been selected and upon operation completion output the collected
  * output from that operation (and repeat until finished).
  */
-@SuppressWarnings("deprecation")
 public class SliveMapper extends MapReduceBase implements
     Mapper<Object, Object, Text, Text> {
 
@@ -63,15 +62,10 @@ public class SliveMapper extends MapReduceBase implements
    */
   @Override // MapReduceBase
   public void configure(JobConf conf) {
-    try {
-      filesystem = FileSystem.get(conf);
-    } catch (Exception e) {
-      throw new RuntimeException(
-          "Unable to get the filesystem from provided configuration", e);
-    }
     try {
       config = new ConfigExtractor(conf);
       ConfigExtractor.dumpOptions(config);
+      filesystem = config.getBaseDirectory().getFileSystem(conf);
     } catch (Exception e) {
       LOG.error("Unable to setup slive " + StringUtils.stringifyException(e));
       throw new RuntimeException("Unable to setup slive configuration", e);

+ 0 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/SlivePartitioner.java

@@ -26,7 +26,6 @@ import org.apache.hadoop.mapred.Partitioner;
  * The partition number is the hash of the operation type modular the total
  * number of the reducers.
  */
-@SuppressWarnings("deprecation")
 public class SlivePartitioner implements Partitioner<Text, Text> {
   @Override // JobConfigurable
   public void configure(JobConf conf) {}

+ 0 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/SliveReducer.java

@@ -35,7 +35,6 @@ import org.apache.hadoop.util.StringUtils;
  * The slive reducer which iterates over the given input values and merges them
  * together into a final output value.
  */
-@SuppressWarnings("deprecation")
 public class SliveReducer extends MapReduceBase implements
     Reducer<Text, Text, Text, Text> {