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.mapred;
020
021import java.io.IOException;
022
023import org.apache.hadoop.classification.InterfaceAudience;
024import org.apache.hadoop.classification.InterfaceStability;
025import 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
069public 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