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;
020    
021    import java.io.IOException;
022    
023    import org.apache.hadoop.classification.InterfaceAudience;
024    import org.apache.hadoop.classification.InterfaceStability;
025    import org.apache.hadoop.fs.FileSystem;
026    
027    /** 
028     * <code>InputFormat</code> describes the input-specification for a 
029     * Map-Reduce job. 
030     * 
031     * <p>The Map-Reduce framework relies on the <code>InputFormat</code> of the
032     * job to:<p>
033     * <ol>
034     *   <li>
035     *   Validate the input-specification of the job. 
036     *   <li>
037     *   Split-up the input file(s) into logical {@link InputSplit}s, each of 
038     *   which is then assigned to an individual {@link Mapper}.
039     *   </li>
040     *   <li>
041     *   Provide the {@link RecordReader} implementation to be used to glean
042     *   input records from the logical <code>InputSplit</code> for processing by 
043     *   the {@link Mapper}.
044     *   </li>
045     * </ol>
046     * 
047     * <p>The default behavior of file-based {@link InputFormat}s, typically 
048     * sub-classes of {@link FileInputFormat}, is to split the 
049     * input into <i>logical</i> {@link InputSplit}s based on the total size, in 
050     * bytes, of the input files. However, the {@link FileSystem} blocksize of  
051     * the input files is treated as an upper bound for input splits. A lower bound 
052     * on the split size can be set via 
053     * <a href="{@docRoot}/../hadoop-mapreduce-client/hadoop-mapreduce-client-core/mapred-default.xml#mapreduce.input.fileinputformat.split.minsize">
054     * mapreduce.input.fileinputformat.split.minsize</a>.</p>
055     * 
056     * <p>Clearly, logical splits based on input-size is insufficient for many 
057     * applications since record boundaries are to be respected. In such cases, the
058     * application has to also implement a {@link RecordReader} on whom lies the
059     * responsibilty to respect record-boundaries and present a record-oriented
060     * view of the logical <code>InputSplit</code> to the individual task.
061     *
062     * @see InputSplit
063     * @see RecordReader
064     * @see JobClient
065     * @see FileInputFormat
066     */
067    @InterfaceAudience.Public
068    @InterfaceStability.Stable
069    public interface InputFormat<K, V> {
070    
071      /** 
072       * Logically split the set of input files for the job.  
073       * 
074       * <p>Each {@link InputSplit} is then assigned to an individual {@link Mapper}
075       * for processing.</p>
076       *
077       * <p><i>Note</i>: The split is a <i>logical</i> split of the inputs and the
078       * input files are not physically split into chunks. For e.g. a split could
079       * be <i>&lt;input-file-path, start, offset&gt;</i> tuple.
080       * 
081       * @param job job configuration.
082       * @param numSplits the desired number of splits, a hint.
083       * @return an array of {@link InputSplit}s for the job.
084       */
085      InputSplit[] getSplits(JobConf job, int numSplits) throws IOException;
086    
087      /** 
088       * Get the {@link RecordReader} for the given {@link InputSplit}.
089       *
090       * <p>It is the responsibility of the <code>RecordReader</code> to respect
091       * record boundaries while processing the logical split to present a 
092       * record-oriented view to the individual task.</p>
093       * 
094       * @param split the {@link InputSplit}
095       * @param job the job that this split belongs to
096       * @return a {@link RecordReader}
097       */
098      RecordReader<K, V> getRecordReader(InputSplit split,
099                                         JobConf job, 
100                                         Reporter reporter) throws IOException;
101    }
102