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
019package org.apache.hadoop.mapreduce.lib.input;
020
021import java.io.IOException;
022import java.util.ArrayList;
023import java.util.List;
024
025import org.apache.hadoop.classification.InterfaceAudience;
026import org.apache.hadoop.classification.InterfaceStability;
027import org.apache.hadoop.conf.Configuration;
028import org.apache.hadoop.fs.FSDataInputStream;
029import org.apache.hadoop.fs.FileStatus;
030import org.apache.hadoop.fs.FileSystem;
031import org.apache.hadoop.fs.Path;
032import org.apache.hadoop.io.LongWritable;
033import org.apache.hadoop.io.Text;
034import org.apache.hadoop.mapreduce.InputSplit;
035import org.apache.hadoop.mapreduce.Job;
036import org.apache.hadoop.mapreduce.JobContext;
037import org.apache.hadoop.mapreduce.RecordReader;
038import org.apache.hadoop.mapreduce.TaskAttemptContext;
039import org.apache.hadoop.util.LineReader;
040
041/**
042 * NLineInputFormat which splits N lines of input as one split.
043 *
044 * In many "pleasantly" parallel applications, each process/mapper 
045 * processes the same input file (s), but with computations are 
046 * controlled by different parameters.(Referred to as "parameter sweeps").
047 * One way to achieve this, is to specify a set of parameters 
048 * (one set per line) as input in a control file 
049 * (which is the input path to the map-reduce application,
050 * where as the input dataset is specified 
051 * via a config variable in JobConf.).
052 * 
053 * The NLineInputFormat can be used in such applications, that splits 
054 * the input file such that by default, one line is fed as
055 * a value to one map task, and key is the offset.
056 * i.e. (k,v) is (LongWritable, Text).
057 * The location hints will span the whole mapred cluster.
058 */
059@InterfaceAudience.Public
060@InterfaceStability.Stable
061public class NLineInputFormat extends FileInputFormat<LongWritable, Text> { 
062  public static final String LINES_PER_MAP = 
063    "mapreduce.input.lineinputformat.linespermap";
064
065  public RecordReader<LongWritable, Text> createRecordReader(
066      InputSplit genericSplit, TaskAttemptContext context) 
067      throws IOException {
068    context.setStatus(genericSplit.toString());
069    return new LineRecordReader();
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 FileInputFormat#getSplits(JobContext)
077   */
078  public List<InputSplit> getSplits(JobContext job)
079  throws IOException {
080    List<InputSplit> splits = new ArrayList<InputSplit>();
081    int numLinesPerSplit = getNumLinesPerSplit(job);
082    for (FileStatus status : listStatus(job)) {
083      splits.addAll(getSplitsForFile(status,
084        job.getConfiguration(), numLinesPerSplit));
085    }
086    return splits;
087  }
088  
089  public static List<FileSplit> getSplitsForFile(FileStatus status,
090      Configuration conf, int numLinesPerSplit) throws IOException {
091    List<FileSplit> splits = new ArrayList<FileSplit> ();
092    Path fileName = status.getPath();
093    if (status.isDirectory()) {
094      throw new IOException("Not a file: " + fileName);
095    }
096    FileSystem  fs = fileName.getFileSystem(conf);
097    LineReader lr = null;
098    try {
099      FSDataInputStream in  = fs.open(fileName);
100      lr = new LineReader(in, conf);
101      Text line = new Text();
102      int numLines = 0;
103      long begin = 0;
104      long length = 0;
105      int num = -1;
106      while ((num = lr.readLine(line)) > 0) {
107        numLines++;
108        length += num;
109        if (numLines == numLinesPerSplit) {
110          splits.add(createFileSplit(fileName, begin, length));
111          begin += length;
112          length = 0;
113          numLines = 0;
114        }
115      }
116      if (numLines != 0) {
117        splits.add(createFileSplit(fileName, begin, length));
118      }
119    } finally {
120      if (lr != null) {
121        lr.close();
122      }
123    }
124    return splits; 
125  }
126
127  /**
128   * NLineInputFormat uses LineRecordReader, which always reads
129   * (and consumes) at least one character out of its upper split
130   * boundary. So to make sure that each mapper gets N lines, we
131   * move back the upper split limits of each split 
132   * by one character here.
133   * @param fileName  Path of file
134   * @param begin  the position of the first byte in the file to process
135   * @param length  number of bytes in InputSplit
136   * @return  FileSplit
137   */
138  protected static FileSplit createFileSplit(Path fileName, long begin, long length) {
139    return (begin == 0) 
140    ? new FileSplit(fileName, begin, length - 1, new String[] {})
141    : new FileSplit(fileName, begin - 1, length, new String[] {});
142  }
143  
144  /**
145   * Set the number of lines per split
146   * @param job the job to modify
147   * @param numLines the number of lines per split
148   */
149  public static void setNumLinesPerSplit(Job job, int numLines) {
150    job.getConfiguration().setInt(LINES_PER_MAP, numLines);
151  }
152
153  /**
154   * Get the number of lines per split
155   * @param job the job
156   * @return the number of lines per split
157   */
158  public static int getNumLinesPerSplit(JobContext job) {
159    return job.getConfiguration().getInt(LINES_PER_MAP, 1);
160  }
161}