001    /**
002     * Licensed to the Apache Software Foundation (ASF) under one
003     * or more contributor license agreements.  See the NOTICE file
004     * distributed with this work for additional information
005     * regarding copyright ownership.  The ASF licenses this file
006     * to you under the Apache License, Version 2.0 (the
007     * "License"); you may not use this file except in compliance
008     * with the License.  You may obtain a copy of the License at
009     *
010     *     http://www.apache.org/licenses/LICENSE-2.0
011     *
012     * Unless required by applicable law or agreed to in writing, software
013     * distributed under the License is distributed on an "AS IS" BASIS,
014     * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015     * See the License for the specific language governing permissions and
016     * limitations under the License.
017     */
018    package org.apache.hadoop.mapred.lib;
019    
020    import java.util.Collections;
021    import java.util.HashMap;
022    import java.util.Map;
023    
024    import org.apache.hadoop.classification.InterfaceAudience;
025    import org.apache.hadoop.classification.InterfaceStability;
026    import org.apache.hadoop.fs.Path;
027    import org.apache.hadoop.mapred.InputFormat;
028    import org.apache.hadoop.mapred.JobConf;
029    import org.apache.hadoop.mapred.Mapper;
030    import org.apache.hadoop.util.ReflectionUtils;
031    
032    /**
033     * This class supports MapReduce jobs that have multiple input paths with
034     * a different {@link InputFormat} and {@link Mapper} for each path 
035     * @deprecated Use 
036     * {@link org.apache.hadoop.mapreduce.lib.input.MultipleInputs} instead
037     */
038    @Deprecated
039    @InterfaceAudience.Public
040    @InterfaceStability.Stable
041    public class MultipleInputs {
042      /**
043       * Add a {@link Path} with a custom {@link InputFormat} to the list of
044       * inputs for the map-reduce job.
045       * 
046       * @param conf The configuration of the job
047       * @param path {@link Path} to be added to the list of inputs for the job
048       * @param inputFormatClass {@link InputFormat} class to use for this path
049       */
050      public static void addInputPath(JobConf conf, Path path,
051          Class<? extends InputFormat> inputFormatClass) {
052    
053        String inputFormatMapping = path.toString() + ";"
054           + inputFormatClass.getName();
055        String inputFormats = conf.get("mapreduce.input.multipleinputs.dir.formats");
056        conf.set("mapreduce.input.multipleinputs.dir.formats",
057           inputFormats == null ? inputFormatMapping : inputFormats + ","
058               + inputFormatMapping);
059    
060        conf.setInputFormat(DelegatingInputFormat.class);
061      }
062    
063      /**
064       * Add a {@link Path} with a custom {@link InputFormat} and
065       * {@link Mapper} to the list of inputs for the map-reduce job.
066       * 
067       * @param conf The configuration of the job
068       * @param path {@link Path} to be added to the list of inputs for the job
069       * @param inputFormatClass {@link InputFormat} class to use for this path
070       * @param mapperClass {@link Mapper} class to use for this path
071       */
072      public static void addInputPath(JobConf conf, Path path,
073          Class<? extends InputFormat> inputFormatClass,
074          Class<? extends Mapper> mapperClass) {
075    
076        addInputPath(conf, path, inputFormatClass);
077    
078        String mapperMapping = path.toString() + ";" + mapperClass.getName();
079        String mappers = conf.get("mapreduce.input.multipleinputs.dir.mappers");
080        conf.set("mapreduce.input.multipleinputs.dir.mappers", mappers == null ? mapperMapping
081           : mappers + "," + mapperMapping);
082    
083        conf.setMapperClass(DelegatingMapper.class);
084      }
085    
086      /**
087       * Retrieves a map of {@link Path}s to the {@link InputFormat} class
088       * that should be used for them.
089       * 
090       * @param conf The confuration of the job
091       * @see #addInputPath(JobConf, Path, Class)
092       * @return A map of paths to inputformats for the job
093       */
094      static Map<Path, InputFormat> getInputFormatMap(JobConf conf) {
095        Map<Path, InputFormat> m = new HashMap<Path, InputFormat>();
096        String[] pathMappings = conf.get("mapreduce.input.multipleinputs.dir.formats").split(",");
097        for (String pathMapping : pathMappings) {
098          String[] split = pathMapping.split(";");
099          InputFormat inputFormat;
100          try {
101           inputFormat = (InputFormat) ReflectionUtils.newInstance(conf
102               .getClassByName(split[1]), conf);
103          } catch (ClassNotFoundException e) {
104           throw new RuntimeException(e);
105          }
106          m.put(new Path(split[0]), inputFormat);
107        }
108        return m;
109      }
110    
111      /**
112       * Retrieves a map of {@link Path}s to the {@link Mapper} class that
113       * should be used for them.
114       * 
115       * @param conf The confuration of the job
116       * @see #addInputPath(JobConf, Path, Class, Class)
117       * @return A map of paths to mappers for the job
118       */
119      @SuppressWarnings("unchecked")
120      static Map<Path, Class<? extends Mapper>> getMapperTypeMap(JobConf conf) {
121        if (conf.get("mapreduce.input.multipleinputs.dir.mappers") == null) {
122          return Collections.emptyMap();
123        }
124        Map<Path, Class<? extends Mapper>> m = new HashMap<Path, Class<? extends Mapper>>();
125        String[] pathMappings = conf.get("mapreduce.input.multipleinputs.dir.mappers").split(",");
126        for (String pathMapping : pathMappings) {
127          String[] split = pathMapping.split(";");
128          Class<? extends Mapper> mapClass;
129          try {
130           mapClass = (Class<? extends Mapper>) conf.getClassByName(split[1]);
131          } catch (ClassNotFoundException e) {
132           throw new RuntimeException(e);
133          }
134          m.put(new Path(split[0]), mapClass);
135        }
136        return m;
137      }
138    }