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    package org.apache.hadoop.mapreduce.lib.input;
019    
020    import java.io.IOException;
021    
022    import org.apache.hadoop.classification.InterfaceAudience;
023    import org.apache.hadoop.classification.InterfaceStability;
024    import org.apache.hadoop.conf.Configuration;
025    import org.apache.hadoop.fs.Path;
026    import org.apache.hadoop.io.BytesWritable;
027    import org.apache.hadoop.io.LongWritable;
028    import org.apache.hadoop.io.compress.CompressionCodec;
029    import org.apache.hadoop.io.compress.CompressionCodecFactory;
030    import org.apache.hadoop.mapreduce.InputSplit;
031    import org.apache.hadoop.mapreduce.JobContext;
032    import org.apache.hadoop.mapreduce.RecordReader;
033    import org.apache.hadoop.mapreduce.TaskAttemptContext;
034    
035    /**
036     * FixedLengthInputFormat is an input format used to read input files
037     * which contain fixed length records.  The content of a record need not be
038     * text.  It can be arbitrary binary data.  Users must configure the record
039     * length property by calling:
040     * FixedLengthInputFormat.setRecordLength(conf, recordLength);<br><br> or
041     * conf.setInt(FixedLengthInputFormat.FIXED_RECORD_LENGTH, recordLength);
042     * <br><br>
043     * @see FixedLengthRecordReader
044     */
045    @InterfaceAudience.Public
046    @InterfaceStability.Stable
047    public class FixedLengthInputFormat
048        extends FileInputFormat<LongWritable, BytesWritable> {
049    
050      public static final String FIXED_RECORD_LENGTH =
051          "fixedlengthinputformat.record.length"; 
052    
053      /**
054       * Set the length of each record
055       * @param conf configuration
056       * @param recordLength the length of a record
057       */
058      public static void setRecordLength(Configuration conf, int recordLength) {
059        conf.setInt(FIXED_RECORD_LENGTH, recordLength);
060      }
061    
062      /**
063       * Get record length value
064       * @param conf configuration
065       * @return the record length, zero means none was set
066       */
067      public static int getRecordLength(Configuration conf) {
068        return conf.getInt(FIXED_RECORD_LENGTH, 0);
069      }
070    
071      @Override
072      public RecordReader<LongWritable, BytesWritable>
073          createRecordReader(InputSplit split, TaskAttemptContext context)
074          throws IOException, InterruptedException {
075        int recordLength = getRecordLength(context.getConfiguration());
076        if (recordLength <= 0) {
077          throw new IOException("Fixed record length " + recordLength
078              + " is invalid.  It should be set to a value greater than zero");
079        }
080        return new FixedLengthRecordReader(recordLength);
081      }
082    
083      @Override
084      protected boolean isSplitable(JobContext context, Path file) {
085        final CompressionCodec codec = 
086            new CompressionCodecFactory(context.getConfiguration()).getCodec(file);
087        return (null == codec);
088      } 
089    
090    }