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 {
+ }
+}