You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by jo...@apache.org on 2008/08/12 19:02:45 UTC

svn commit: r685227 - in /hadoop/core/trunk: ./ src/mapred/org/apache/hadoop/mapred/ src/mapred/org/apache/hadoop/mapred/lib/ src/test/org/apache/hadoop/mapred/ src/test/org/apache/hadoop/mapred/lib/

Author: johan
Date: Tue Aug 12 10:02:44 2008
New Revision: 685227

URL: http://svn.apache.org/viewvc?rev=685227&view=rev
Log:
HADOOP-3853. Move multiple input format (HADOOP-372) extension to library package. (tomwhite via johan)

Added:
    hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/lib/DelegatingInputFormat.java   (contents, props changed)
      - copied, changed from r685120, hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/DelegatingInputFormat.java
    hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/lib/DelegatingMapper.java   (contents, props changed)
      - copied, changed from r685120, hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/DelegatingMapper.java
    hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/lib/MultipleInputs.java
    hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/lib/TaggedInputSplit.java   (contents, props changed)
      - copied, changed from r685120, hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/TaggedInputSplit.java
    hadoop/core/trunk/src/test/org/apache/hadoop/mapred/lib/TestDelegatingInputFormat.java   (contents, props changed)
      - copied, changed from r685120, hadoop/core/trunk/src/test/org/apache/hadoop/mapred/TestDelegatingInputFormat.java
    hadoop/core/trunk/src/test/org/apache/hadoop/mapred/lib/TestMultipleInputs.java
Removed:
    hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/DelegatingInputFormat.java
    hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/DelegatingMapper.java
    hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/TaggedInputSplit.java
    hadoop/core/trunk/src/test/org/apache/hadoop/mapred/TestDelegatingInputFormat.java
Modified:
    hadoop/core/trunk/CHANGES.txt
    hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/FileInputFormat.java
    hadoop/core/trunk/src/test/org/apache/hadoop/mapred/TestFileInputFormat.java

Modified: hadoop/core/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/CHANGES.txt?rev=685227&r1=685226&r2=685227&view=diff
==============================================================================
--- hadoop/core/trunk/CHANGES.txt (original)
+++ hadoop/core/trunk/CHANGES.txt Tue Aug 12 10:02:44 2008
@@ -154,6 +154,9 @@
     the end of the heartbeat rpc, rather than the start. This causes better
     behavior if the JobTracker is overloaded. (acmurthy via omalley)
 
+		HADOOP-3853. Move multiple input format (HADOOP-372) extension to 
+		library package. (tomwhite via johan)
+
   OPTIMIZATIONS
 
     HADOOP-3556. Removed lock contention in MD5Hash by changing the 

Modified: hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/FileInputFormat.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/FileInputFormat.java?rev=685227&r1=685226&r2=685227&view=diff
==============================================================================
--- hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/FileInputFormat.java (original)
+++ hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/FileInputFormat.java Tue Aug 12 10:02:44 2008
@@ -21,19 +21,16 @@
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
-import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 
@@ -401,103 +398,6 @@
     conf.set("mapred.input.dir", dirs == null ? dirStr :
       dirs + StringUtils.COMMA_STR + dirStr);
   }
-  
-  /**
-   * Add a {@link Path} with a custom {@link InputFormat} to the list of
-   * inputs for the map-reduce job.
-   * 
-   * @param conf The configuration of the job
-   * @param path {@link Path} to be added to the list of inputs for the job
-   * @param inputFormatClass {@link InputFormat} class to use for this path
-   */
-  public static void addInputPath(JobConf conf, Path path,
-      Class<? extends InputFormat> inputFormatClass) {
-
-    String inputFormatMapping = path.toString() + ";"
-       + inputFormatClass.getName();
-    String inputFormats = conf.get("mapred.input.dir.formats");
-    conf.set("mapred.input.dir.formats",
-       inputFormats == null ? inputFormatMapping : inputFormats + ","
-           + inputFormatMapping);
-
-    conf.setInputFormat(DelegatingInputFormat.class);
-  }
-
-  /**
-   * Add a {@link Path} with a custom {@link InputFormat} and
-   * {@link Mapper} to the list of inputs for the map-reduce job.
-   * 
-   * @param conf The configuration of the job
-   * @param path {@link Path} to be added to the list of inputs for the job
-   * @param inputFormatClass {@link InputFormat} class to use for this path
-   * @param mapperClass {@link Mapper} class to use for this path
-   */
-  public static void addInputPath(JobConf conf, Path path,
-      Class<? extends InputFormat> inputFormatClass,
-      Class<? extends Mapper> mapperClass) {
-
-    addInputPath(conf, path, inputFormatClass);
-
-    String mapperMapping = path.toString() + ";" + mapperClass.getName();
-    String mappers = conf.get("mapred.input.dir.mappers");
-    conf.set("mapred.input.dir.mappers", mappers == null ? mapperMapping
-       : mappers + "," + mapperMapping);
-
-    conf.setMapperClass(DelegatingMapper.class);
-  }
-
-  /**
-   * Retrieves a map of {@link Path}s to the {@link InputFormat} class
-   * that should be used for them.
-   * 
-   * @param conf The confuration of the job
-   * @see #addInputPath(JobConf, Path, Class)
-   * @return A map of paths to inputformats for the job
-   */
-  static Map<Path, InputFormat> getInputFormatMap(JobConf conf) {
-    Map<Path, InputFormat> m = new HashMap<Path, InputFormat>();
-    String[] pathMappings = conf.get("mapred.input.dir.formats").split(",");
-    for (String pathMapping : pathMappings) {
-      String[] split = pathMapping.split(";");
-      InputFormat inputFormat;
-      try {
-       inputFormat = (InputFormat) ReflectionUtils.newInstance(conf
-           .getClassByName(split[1]), conf);
-      } catch (ClassNotFoundException e) {
-       throw new RuntimeException(e);
-      }
-      m.put(new Path(split[0]), inputFormat);
-    }
-    return m;
-  }
-
-  /**
-   * Retrieves a map of {@link Path}s to the {@link Mapper} class that
-   * should be used for them.
-   * 
-   * @param conf The confuration of the job
-   * @see #addInputPath(JobConf, Path, Class, Class)
-   * @return A map of paths to mappers for the job
-   */
-  @SuppressWarnings("unchecked")
-  static Map<Path, Class<? extends Mapper>> getMapperTypeMap(JobConf conf) {
-    if (conf.get("mapred.input.dir.mappers") == null) {
-      return Collections.emptyMap();
-    }
-    Map<Path, Class<? extends Mapper>> m = new HashMap<Path, Class<? extends Mapper>>();
-    String[] pathMappings = conf.get("mapred.input.dir.mappers").split(",");
-    for (String pathMapping : pathMappings) {
-      String[] split = pathMapping.split(";");
-      Class<? extends Mapper> mapClass;
-      try {
-       mapClass = (Class<? extends Mapper>) conf.getClassByName(split[1]);
-      } catch (ClassNotFoundException e) {
-       throw new RuntimeException(e);
-      }
-      m.put(new Path(split[0]), mapClass);
-    }
-    return m;
-  }
          
   // This method escapes commas in the glob pattern of the given paths.
   private static String[] getPathStrings(String commaSeparatedPaths) {

Copied: hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/lib/DelegatingInputFormat.java (from r685120, hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/DelegatingInputFormat.java)
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/lib/DelegatingInputFormat.java?p2=hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/lib/DelegatingInputFormat.java&p1=hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/DelegatingInputFormat.java&r1=685120&r2=685227&rev=685227&view=diff
==============================================================================
--- hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/DelegatingInputFormat.java (original)
+++ hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/lib/DelegatingInputFormat.java Tue Aug 12 10:02:44 2008
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.mapred;
+package org.apache.hadoop.mapred.lib;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -27,20 +27,27 @@
 import java.util.Map.Entry;
 
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.util.ReflectionUtils;
 
 /**
  * An {@link InputFormat} that delegates behaviour of paths to multiple other
  * InputFormats.
  * 
- * @see FileInputFormat#addInputPath(JobConf, Path, Class, Class)
+ * @see MultipleInputs#addInputPath(JobConf, Path, Class, Class)
  */
 public class DelegatingInputFormat<K, V> implements InputFormat<K, V> {
 
   @Deprecated
   public void validateInput(JobConf conf) throws IOException {
     JobConf confCopy = new JobConf(conf);
-    Map<Path, InputFormat> formatMap = FileInputFormat.getInputFormatMap(conf);
+    Map<Path, InputFormat> formatMap = MultipleInputs.getInputFormatMap(conf);
     for (Entry<Path, InputFormat> entry : formatMap.entrySet()) {
       Path path = entry.getKey();
       InputFormat format = entry.getValue();
@@ -53,8 +60,8 @@
 
     JobConf confCopy = new JobConf(conf);
     List<InputSplit> splits = new ArrayList<InputSplit>();
-    Map<Path, InputFormat> formatMap = FileInputFormat.getInputFormatMap(conf);
-    Map<Path, Class<? extends Mapper>> mapperMap = FileInputFormat
+    Map<Path, InputFormat> formatMap = MultipleInputs.getInputFormatMap(conf);
+    Map<Path, Class<? extends Mapper>> mapperMap = MultipleInputs
        .getMapperTypeMap(conf);
     Map<Class<? extends InputFormat>, List<Path>> formatPaths
         = new HashMap<Class<? extends InputFormat>, List<Path>>();

Propchange: hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/lib/DelegatingInputFormat.java
------------------------------------------------------------------------------
    svn:mergeinfo = 

Copied: hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/lib/DelegatingMapper.java (from r685120, hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/DelegatingMapper.java)
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/lib/DelegatingMapper.java?p2=hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/lib/DelegatingMapper.java&p1=hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/DelegatingMapper.java&r1=685120&r2=685227&rev=685227&view=diff
==============================================================================
--- hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/DelegatingMapper.java (original)
+++ hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/lib/DelegatingMapper.java Tue Aug 12 10:02:44 2008
@@ -16,17 +16,22 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.mapred;
+package org.apache.hadoop.mapred.lib;
 
 import java.io.IOException;
 
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.util.ReflectionUtils;
 
 /**
  * An {@link Mapper} that delegates behaviour of paths to multiple other
  * mappers.
  * 
- * @see FileInputFormat#addInputPath(JobConf, Path, Class, Class)
+ * @see MultipleInputs#addInputPath(JobConf, Path, Class, Class)
  */
 public class DelegatingMapper<K1, V1, K2, V2> implements Mapper<K1, V1, K2, V2> {
 

Propchange: hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/lib/DelegatingMapper.java
------------------------------------------------------------------------------
    svn:mergeinfo = 

Added: hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/lib/MultipleInputs.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/lib/MultipleInputs.java?rev=685227&view=auto
==============================================================================
--- hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/lib/MultipleInputs.java (added)
+++ hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/lib/MultipleInputs.java Tue Aug 12 10:02:44 2008
@@ -0,0 +1,131 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.mapred.lib;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.util.ReflectionUtils;
+
+/**
+ * This class supports MapReduce jobs that have multiple input paths with
+ * a different {@link InputFormat} and {@link Mapper} for each path 
+ */
+public class MultipleInputs {
+  /**
+   * Add a {@link Path} with a custom {@link InputFormat} to the list of
+   * inputs for the map-reduce job.
+   * 
+   * @param conf The configuration of the job
+   * @param path {@link Path} to be added to the list of inputs for the job
+   * @param inputFormatClass {@link InputFormat} class to use for this path
+   */
+  public static void addInputPath(JobConf conf, Path path,
+      Class<? extends InputFormat> inputFormatClass) {
+
+    String inputFormatMapping = path.toString() + ";"
+       + inputFormatClass.getName();
+    String inputFormats = conf.get("mapred.input.dir.formats");
+    conf.set("mapred.input.dir.formats",
+       inputFormats == null ? inputFormatMapping : inputFormats + ","
+           + inputFormatMapping);
+
+    conf.setInputFormat(DelegatingInputFormat.class);
+  }
+
+  /**
+   * Add a {@link Path} with a custom {@link InputFormat} and
+   * {@link Mapper} to the list of inputs for the map-reduce job.
+   * 
+   * @param conf The configuration of the job
+   * @param path {@link Path} to be added to the list of inputs for the job
+   * @param inputFormatClass {@link InputFormat} class to use for this path
+   * @param mapperClass {@link Mapper} class to use for this path
+   */
+  public static void addInputPath(JobConf conf, Path path,
+      Class<? extends InputFormat> inputFormatClass,
+      Class<? extends Mapper> mapperClass) {
+
+    addInputPath(conf, path, inputFormatClass);
+
+    String mapperMapping = path.toString() + ";" + mapperClass.getName();
+    String mappers = conf.get("mapred.input.dir.mappers");
+    conf.set("mapred.input.dir.mappers", mappers == null ? mapperMapping
+       : mappers + "," + mapperMapping);
+
+    conf.setMapperClass(DelegatingMapper.class);
+  }
+
+  /**
+   * Retrieves a map of {@link Path}s to the {@link InputFormat} class
+   * that should be used for them.
+   * 
+   * @param conf The confuration of the job
+   * @see #addInputPath(JobConf, Path, Class)
+   * @return A map of paths to inputformats for the job
+   */
+  static Map<Path, InputFormat> getInputFormatMap(JobConf conf) {
+    Map<Path, InputFormat> m = new HashMap<Path, InputFormat>();
+    String[] pathMappings = conf.get("mapred.input.dir.formats").split(",");
+    for (String pathMapping : pathMappings) {
+      String[] split = pathMapping.split(";");
+      InputFormat inputFormat;
+      try {
+       inputFormat = (InputFormat) ReflectionUtils.newInstance(conf
+           .getClassByName(split[1]), conf);
+      } catch (ClassNotFoundException e) {
+       throw new RuntimeException(e);
+      }
+      m.put(new Path(split[0]), inputFormat);
+    }
+    return m;
+  }
+
+  /**
+   * Retrieves a map of {@link Path}s to the {@link Mapper} class that
+   * should be used for them.
+   * 
+   * @param conf The confuration of the job
+   * @see #addInputPath(JobConf, Path, Class, Class)
+   * @return A map of paths to mappers for the job
+   */
+  @SuppressWarnings("unchecked")
+  static Map<Path, Class<? extends Mapper>> getMapperTypeMap(JobConf conf) {
+    if (conf.get("mapred.input.dir.mappers") == null) {
+      return Collections.emptyMap();
+    }
+    Map<Path, Class<? extends Mapper>> m = new HashMap<Path, Class<? extends Mapper>>();
+    String[] pathMappings = conf.get("mapred.input.dir.mappers").split(",");
+    for (String pathMapping : pathMappings) {
+      String[] split = pathMapping.split(";");
+      Class<? extends Mapper> mapClass;
+      try {
+       mapClass = (Class<? extends Mapper>) conf.getClassByName(split[1]);
+      } catch (ClassNotFoundException e) {
+       throw new RuntimeException(e);
+      }
+      m.put(new Path(split[0]), mapClass);
+    }
+    return m;
+  }
+}

Copied: hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/lib/TaggedInputSplit.java (from r685120, hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/TaggedInputSplit.java)
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/lib/TaggedInputSplit.java?p2=hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/lib/TaggedInputSplit.java&p1=hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/TaggedInputSplit.java&r1=685120&r2=685227&rev=685227&view=diff
==============================================================================
--- hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/TaggedInputSplit.java (original)
+++ hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/lib/TaggedInputSplit.java Tue Aug 12 10:02:44 2008
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.mapred;
+package org.apache.hadoop.mapred.lib;
 
 import java.io.DataInput;
 import java.io.DataOutput;
@@ -25,13 +25,16 @@
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.Mapper;
 import org.apache.hadoop.util.ReflectionUtils;
 
 /**
  * An {@link InputSplit} that tags another InputSplit with extra data for use by
  * {@link DelegatingInputFormat}s and {@link DelegatingMapper}s.
  */
-public class TaggedInputSplit implements Configurable, InputSplit {
+class TaggedInputSplit implements Configurable, InputSplit {
 
   private Class<? extends InputSplit> inputSplitClass;
 

Propchange: hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/lib/TaggedInputSplit.java
------------------------------------------------------------------------------
    svn:mergeinfo = 

Modified: hadoop/core/trunk/src/test/org/apache/hadoop/mapred/TestFileInputFormat.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/test/org/apache/hadoop/mapred/TestFileInputFormat.java?rev=685227&r1=685226&r2=685227&view=diff
==============================================================================
--- hadoop/core/trunk/src/test/org/apache/hadoop/mapred/TestFileInputFormat.java (original)
+++ hadoop/core/trunk/src/test/org/apache/hadoop/mapred/TestFileInputFormat.java Tue Aug 12 10:02:44 2008
@@ -18,8 +18,6 @@
 package org.apache.hadoop.mapred;
 
 import java.io.DataOutputStream;
-import java.io.IOException;
-import java.util.Map;
 
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.fs.BlockLocation;
@@ -85,52 +83,5 @@
       }
     }
   }
-  
-  public void testAddInputPathWithFormat() {
-    final JobConf conf = new JobConf();
-    FileInputFormat.addInputPath(conf, new Path("/foo"), TextInputFormat.class);
-    FileInputFormat.addInputPath(conf, new Path("/bar"),
-        KeyValueTextInputFormat.class);
-    final Map<Path, InputFormat> inputs = FileInputFormat
-       .getInputFormatMap(conf);
-    assertEquals(TextInputFormat.class, inputs.get(new Path("/foo")).getClass());
-    assertEquals(KeyValueTextInputFormat.class, inputs.get(new Path("/bar"))
-       .getClass());
-  }
-
-  public void testAddInputPathWithMapper() {
-    final JobConf conf = new JobConf();
-    FileInputFormat.addInputPath(conf, new Path("/foo"), TextInputFormat.class,
-       MapClass.class);
-    FileInputFormat.addInputPath(conf, new Path("/bar"),
-       KeyValueTextInputFormat.class, MapClass2.class);
-    final Map<Path, InputFormat> inputs = FileInputFormat
-       .getInputFormatMap(conf);
-    final Map<Path, Class<? extends Mapper>> maps = FileInputFormat
-       .getMapperTypeMap(conf);
-
-    assertEquals(TextInputFormat.class, inputs.get(new Path("/foo")).getClass());
-    assertEquals(KeyValueTextInputFormat.class, inputs.get(new Path("/bar"))
-       .getClass());
-    assertEquals(MapClass.class, maps.get(new Path("/foo")));
-    assertEquals(MapClass2.class, maps.get(new Path("/bar")));
-  }
-
-  static class MapClass implements Mapper<String, String, String, String> {
-
-    public void map(String key, String value,
-       OutputCollector<String, String> output, Reporter reporter)
-       throws IOException {
-    }
-
-    public void configure(JobConf job) {
-    }
-
-    public void close() throws IOException {
-    }
-  }
-
-  static class MapClass2 extends MapClass {
-  }
 
 }

Copied: hadoop/core/trunk/src/test/org/apache/hadoop/mapred/lib/TestDelegatingInputFormat.java (from r685120, hadoop/core/trunk/src/test/org/apache/hadoop/mapred/TestDelegatingInputFormat.java)
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/test/org/apache/hadoop/mapred/lib/TestDelegatingInputFormat.java?p2=hadoop/core/trunk/src/test/org/apache/hadoop/mapred/lib/TestDelegatingInputFormat.java&p1=hadoop/core/trunk/src/test/org/apache/hadoop/mapred/TestDelegatingInputFormat.java&r1=685120&r2=685227&rev=685227&view=diff
==============================================================================
--- hadoop/core/trunk/src/test/org/apache/hadoop/mapred/TestDelegatingInputFormat.java (original)
+++ hadoop/core/trunk/src/test/org/apache/hadoop/mapred/lib/TestDelegatingInputFormat.java Tue Aug 12 10:02:44 2008
@@ -15,16 +15,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.mapred;
+package org.apache.hadoop.mapred.lib;
 
 import java.io.DataOutputStream;
 import java.io.IOException;
 
-import org.apache.hadoop.hdfs.MiniDFSCluster;
+import junit.framework.TestCase;
+
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-
-import junit.framework.TestCase;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.KeyValueTextInputFormat;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.TextInputFormat;
 
 public class TestDelegatingInputFormat extends TestCase {
 
@@ -46,13 +53,13 @@
 
       final int numSplits = 100;
 
-      FileInputFormat.addInputPath(conf, path, TextInputFormat.class,
+      MultipleInputs.addInputPath(conf, path, TextInputFormat.class,
          MapClass.class);
-      FileInputFormat.addInputPath(conf, path2, TextInputFormat.class,
+      MultipleInputs.addInputPath(conf, path2, TextInputFormat.class,
          MapClass2.class);
-      FileInputFormat.addInputPath(conf, path3, KeyValueTextInputFormat.class,
+      MultipleInputs.addInputPath(conf, path3, KeyValueTextInputFormat.class,
          MapClass.class);
-      FileInputFormat.addInputPath(conf, path4, TextInputFormat.class,
+      MultipleInputs.addInputPath(conf, path4, TextInputFormat.class,
          MapClass2.class);
       DelegatingInputFormat inFormat = new DelegatingInputFormat();
       InputSplit[] splits = inFormat.getSplits(conf, numSplits);

Propchange: hadoop/core/trunk/src/test/org/apache/hadoop/mapred/lib/TestDelegatingInputFormat.java
------------------------------------------------------------------------------
    svn:mergeinfo = 

Added: hadoop/core/trunk/src/test/org/apache/hadoop/mapred/lib/TestMultipleInputs.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/test/org/apache/hadoop/mapred/lib/TestMultipleInputs.java?rev=685227&view=auto
==============================================================================
--- hadoop/core/trunk/src/test/org/apache/hadoop/mapred/lib/TestMultipleInputs.java (added)
+++ hadoop/core/trunk/src/test/org/apache/hadoop/mapred/lib/TestMultipleInputs.java Tue Aug 12 10:02:44 2008
@@ -0,0 +1,85 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.mapred.lib;
+
+import java.io.IOException;
+import java.util.Map;
+
+import junit.framework.TestCase;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.KeyValueTextInputFormat;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.TextInputFormat;
+
+/**
+ * @see TestDelegatingInputFormat
+ */
+public class TestMultipleInputs extends TestCase {
+  
+  public void testAddInputPathWithFormat() {
+    final JobConf conf = new JobConf();
+    MultipleInputs.addInputPath(conf, new Path("/foo"), TextInputFormat.class);
+    MultipleInputs.addInputPath(conf, new Path("/bar"),
+        KeyValueTextInputFormat.class);
+    final Map<Path, InputFormat> inputs = MultipleInputs
+       .getInputFormatMap(conf);
+    assertEquals(TextInputFormat.class, inputs.get(new Path("/foo")).getClass());
+    assertEquals(KeyValueTextInputFormat.class, inputs.get(new Path("/bar"))
+       .getClass());
+  }
+
+  public void testAddInputPathWithMapper() {
+    final JobConf conf = new JobConf();
+    MultipleInputs.addInputPath(conf, new Path("/foo"), TextInputFormat.class,
+       MapClass.class);
+    MultipleInputs.addInputPath(conf, new Path("/bar"),
+       KeyValueTextInputFormat.class, MapClass2.class);
+    final Map<Path, InputFormat> inputs = MultipleInputs
+       .getInputFormatMap(conf);
+    final Map<Path, Class<? extends Mapper>> maps = MultipleInputs
+       .getMapperTypeMap(conf);
+
+    assertEquals(TextInputFormat.class, inputs.get(new Path("/foo")).getClass());
+    assertEquals(KeyValueTextInputFormat.class, inputs.get(new Path("/bar"))
+       .getClass());
+    assertEquals(MapClass.class, maps.get(new Path("/foo")));
+    assertEquals(MapClass2.class, maps.get(new Path("/bar")));
+  }
+
+  static class MapClass implements Mapper<String, String, String, String> {
+
+    public void map(String key, String value,
+       OutputCollector<String, String> output, Reporter reporter)
+       throws IOException {
+    }
+
+    public void configure(JobConf job) {
+    }
+
+    public void close() throws IOException {
+    }
+  }
+
+  static class MapClass2 extends MapClass {
+  }
+}