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.FileStatus; 026 import org.apache.hadoop.fs.FileSystem; 027 import org.apache.hadoop.fs.Path; 028 029 import org.apache.hadoop.io.SequenceFile; 030 import org.apache.hadoop.io.MapFile; 031 032 /** 033 * An {@link InputFormat} for {@link SequenceFile}s. 034 */ 035 @InterfaceAudience.Public 036 @InterfaceStability.Stable 037 public class SequenceFileInputFormat<K, V> extends FileInputFormat<K, V> { 038 039 public SequenceFileInputFormat() { 040 setMinSplitSize(SequenceFile.SYNC_INTERVAL); 041 } 042 043 @Override 044 protected FileStatus[] listStatus(JobConf job) throws IOException { 045 FileStatus[] files = super.listStatus(job); 046 for (int i = 0; i < files.length; i++) { 047 FileStatus file = files[i]; 048 if (file.isDirectory()) { // it's a MapFile 049 Path dataFile = new Path(file.getPath(), MapFile.DATA_FILE_NAME); 050 FileSystem fs = file.getPath().getFileSystem(job); 051 // use the data file 052 files[i] = fs.getFileStatus(dataFile); 053 } 054 } 055 return files; 056 } 057 058 public RecordReader<K, V> getRecordReader(InputSplit split, 059 JobConf job, Reporter reporter) 060 throws IOException { 061 062 reporter.setStatus(split.toString()); 063 064 return new SequenceFileRecordReader<K, V>(job, (FileSplit) split); 065 } 066 067 } 068