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    
019    package org.apache.hadoop.mapred.lib;
020    
021    import java.io.IOException;
022    import java.util.ArrayList;
023    
024    import org.apache.hadoop.classification.InterfaceAudience;
025    import org.apache.hadoop.classification.InterfaceStability;
026    import org.apache.hadoop.fs.FileStatus;
027    import org.apache.hadoop.fs.Path;
028    import org.apache.hadoop.io.LongWritable;
029    import org.apache.hadoop.io.Text;
030    import org.apache.hadoop.mapred.FileInputFormat;
031    import org.apache.hadoop.mapred.FileSplit;
032    import org.apache.hadoop.mapred.InputSplit;
033    import org.apache.hadoop.mapred.JobConf;
034    import org.apache.hadoop.mapred.JobConfigurable;
035    import org.apache.hadoop.mapred.LineRecordReader;
036    import org.apache.hadoop.mapred.RecordReader;
037    import org.apache.hadoop.mapred.Reporter;
038    
039    /**
040     * NLineInputFormat which splits N lines of input as one split.
041     *
042     * In many "pleasantly" parallel applications, each process/mapper 
043     * processes the same input file (s), but with computations are 
044     * controlled by different parameters.(Referred to as "parameter sweeps").
045     * One way to achieve this, is to specify a set of parameters 
046     * (one set per line) as input in a control file 
047     * (which is the input path to the map-reduce application,
048     * where as the input dataset is specified 
049     * via a config variable in JobConf.).
050     * 
051     * The NLineInputFormat can be used in such applications, that splits 
052     * the input file such that by default, one line is fed as
053     * a value to one map task, and key is the offset.
054     * i.e. (k,v) is (LongWritable, Text).
055     * The location hints will span the whole mapred cluster.
056     */
057    @InterfaceAudience.Public
058    @InterfaceStability.Stable
059    public class NLineInputFormat extends FileInputFormat<LongWritable, Text> 
060                                  implements JobConfigurable { 
061      private int N = 1;
062    
063      public RecordReader<LongWritable, Text> getRecordReader(
064                                                InputSplit genericSplit,
065                                                JobConf job,
066                                                Reporter reporter) 
067      throws IOException {
068        reporter.setStatus(genericSplit.toString());
069        return new LineRecordReader(job, (FileSplit) genericSplit);
070      }
071    
072      /** 
073       * Logically splits the set of input files for the job, splits N lines
074       * of the input as one split.
075       * 
076       * @see org.apache.hadoop.mapred.FileInputFormat#getSplits(JobConf, int)
077       */
078      public InputSplit[] getSplits(JobConf job, int numSplits)
079      throws IOException {
080        ArrayList<FileSplit> splits = new ArrayList<FileSplit>();
081        for (FileStatus status : listStatus(job)) {
082          for (org.apache.hadoop.mapreduce.lib.input.FileSplit split : 
083              org.apache.hadoop.mapreduce.lib.input.
084              NLineInputFormat.getSplitsForFile(status, job, N)) {
085            splits.add(new FileSplit(split));
086          }
087        }
088        return splits.toArray(new FileSplit[splits.size()]);
089      }
090    
091      public void configure(JobConf conf) {
092        N = conf.getInt("mapreduce.input.lineinputformat.linespermap", 1);
093      }
094      
095      /**
096       * NLineInputFormat uses LineRecordReader, which always reads
097       * (and consumes) at least one character out of its upper split
098       * boundary. So to make sure that each mapper gets N lines, we
099       * move back the upper split limits of each split 
100       * by one character here.
101       * @param fileName  Path of file
102       * @param begin  the position of the first byte in the file to process
103       * @param length  number of bytes in InputSplit
104       * @return  FileSplit
105       */
106      protected static FileSplit createFileSplit(Path fileName, long begin, long length) {
107        return (begin == 0) 
108        ? new FileSplit(fileName, begin, length - 1, new String[] {})
109        : new FileSplit(fileName, begin - 1, length, new String[] {});
110      }
111    }