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.lib.input;
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.FileStatus;
027    import org.apache.hadoop.fs.FileSystem;
028    import org.apache.hadoop.fs.Path;
029    
030    import org.apache.hadoop.io.SequenceFile;
031    import org.apache.hadoop.io.MapFile;
032    import org.apache.hadoop.mapreduce.InputFormat;
033    import org.apache.hadoop.mapreduce.InputSplit;
034    import org.apache.hadoop.mapreduce.JobContext;
035    import org.apache.hadoop.mapreduce.RecordReader;
036    import org.apache.hadoop.mapreduce.TaskAttemptContext;
037    
038    /** An {@link InputFormat} for {@link SequenceFile}s. */
039    @InterfaceAudience.Public
040    @InterfaceStability.Stable
041    public class SequenceFileInputFormat<K, V> extends FileInputFormat<K, V> {
042    
043      @Override
044      public RecordReader<K, V> createRecordReader(InputSplit split,
045                                                   TaskAttemptContext context
046                                                   ) throws IOException {
047        return new SequenceFileRecordReader<K,V>();
048      }
049    
050      @Override
051      protected long getFormatMinSplitSize() {
052        return SequenceFile.SYNC_INTERVAL;
053      }
054    
055      @Override
056      protected List<FileStatus> listStatus(JobContext job
057                                            )throws IOException {
058    
059        List<FileStatus> files = super.listStatus(job);
060        int len = files.size();
061        for(int i=0; i < len; ++i) {
062          FileStatus file = files.get(i);
063          if (file.isDirectory()) {     // it's a MapFile
064            Path p = file.getPath();
065            FileSystem fs = p.getFileSystem(job.getConfiguration());
066            // use the data file
067            files.set(i, fs.getFileStatus(new Path(p, MapFile.DATA_FILE_NAME)));
068          }
069        }
070        return files;
071      }
072    }
073