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