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.io.LongWritable;
028    import org.apache.hadoop.io.Text;
029    import org.apache.hadoop.mapred.FileInputFormat;
030    import org.apache.hadoop.mapred.FileSplit;
031    import org.apache.hadoop.mapred.InputSplit;
032    import org.apache.hadoop.mapred.JobConf;
033    import org.apache.hadoop.mapred.JobConfigurable;
034    import org.apache.hadoop.mapred.LineRecordReader;
035    import org.apache.hadoop.mapred.RecordReader;
036    import org.apache.hadoop.mapred.Reporter;
037    
038    /**
039     * NLineInputFormat which splits N lines of input as one split.
040     *
041     * In many "pleasantly" parallel applications, each process/mapper 
042     * processes the same input file (s), but with computations are 
043     * controlled by different parameters.(Referred to as "parameter sweeps").
044     * One way to achieve this, is to specify a set of parameters 
045     * (one set per line) as input in a control file 
046     * (which is the input path to the map-reduce application,
047     * where as the input dataset is specified 
048     * via a config variable in JobConf.).
049     * 
050     * The NLineInputFormat can be used in such applications, that splits 
051     * the input file such that by default, one line is fed as
052     * a value to one map task, and key is the offset.
053     * i.e. (k,v) is (LongWritable, Text).
054     * The location hints will span the whole mapred cluster.
055     */
056    @InterfaceAudience.Public
057    @InterfaceStability.Stable
058    public class NLineInputFormat extends FileInputFormat<LongWritable, Text> 
059                                  implements JobConfigurable { 
060      private int N = 1;
061    
062      public RecordReader<LongWritable, Text> getRecordReader(
063                                                InputSplit genericSplit,
064                                                JobConf job,
065                                                Reporter reporter) 
066      throws IOException {
067        reporter.setStatus(genericSplit.toString());
068        return new LineRecordReader(job, (FileSplit) genericSplit);
069      }
070    
071      /** 
072       * Logically splits the set of input files for the job, splits N lines
073       * of the input as one split.
074       * 
075       * @see org.apache.hadoop.mapred.FileInputFormat#getSplits(JobConf, int)
076       */
077      public InputSplit[] getSplits(JobConf job, int numSplits)
078      throws IOException {
079        ArrayList<FileSplit> splits = new ArrayList<FileSplit>();
080        for (FileStatus status : listStatus(job)) {
081          for (org.apache.hadoop.mapreduce.lib.input.FileSplit split : 
082              org.apache.hadoop.mapreduce.lib.input.
083              NLineInputFormat.getSplitsForFile(status, job, N)) {
084            splits.add(new FileSplit(split));
085          }
086        }
087        return splits.toArray(new FileSplit[splits.size()]);
088      }
089    
090      public void configure(JobConf conf) {
091        N = conf.getInt("mapreduce.input.lineinputformat.linespermap", 1);
092      }
093    }