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 024 import org.apache.hadoop.classification.InterfaceAudience; 025 import org.apache.hadoop.classification.InterfaceStability; 026 import org.apache.hadoop.conf.Configuration; 027 import org.apache.hadoop.fs.FileSystem; 028 import org.apache.hadoop.fs.Path; 029 import org.apache.hadoop.io.*; 030 import org.apache.hadoop.util.ReflectionUtils; 031 032 /** An {@link RecordReader} for {@link SequenceFile}s. */ 033 @InterfaceAudience.Public 034 @InterfaceStability.Stable 035 public class SequenceFileRecordReader<K, V> implements RecordReader<K, V> { 036 037 private SequenceFile.Reader in; 038 private long start; 039 private long end; 040 private boolean more = true; 041 protected Configuration conf; 042 043 public SequenceFileRecordReader(Configuration conf, FileSplit split) 044 throws IOException { 045 Path path = split.getPath(); 046 FileSystem fs = path.getFileSystem(conf); 047 this.in = new SequenceFile.Reader(fs, path, conf); 048 this.end = split.getStart() + split.getLength(); 049 this.conf = conf; 050 051 if (split.getStart() > in.getPosition()) 052 in.sync(split.getStart()); // sync to start 053 054 this.start = in.getPosition(); 055 more = start < end; 056 } 057 058 059 /** The class of key that must be passed to {@link 060 * #next(Object, Object)}.. */ 061 public Class getKeyClass() { return in.getKeyClass(); } 062 063 /** The class of value that must be passed to {@link 064 * #next(Object, Object)}.. */ 065 public Class getValueClass() { return in.getValueClass(); } 066 067 @SuppressWarnings("unchecked") 068 public K createKey() { 069 return (K) ReflectionUtils.newInstance(getKeyClass(), conf); 070 } 071 072 @SuppressWarnings("unchecked") 073 public V createValue() { 074 return (V) ReflectionUtils.newInstance(getValueClass(), conf); 075 } 076 077 public synchronized boolean next(K key, V value) throws IOException { 078 if (!more) return false; 079 long pos = in.getPosition(); 080 boolean remaining = (in.next(key) != null); 081 if (remaining) { 082 getCurrentValue(value); 083 } 084 if (pos >= end && in.syncSeen()) { 085 more = false; 086 } else { 087 more = remaining; 088 } 089 return more; 090 } 091 092 protected synchronized boolean next(K key) 093 throws IOException { 094 if (!more) return false; 095 long pos = in.getPosition(); 096 boolean remaining = (in.next(key) != null); 097 if (pos >= end && in.syncSeen()) { 098 more = false; 099 } else { 100 more = remaining; 101 } 102 return more; 103 } 104 105 protected synchronized void getCurrentValue(V value) 106 throws IOException { 107 in.getCurrentValue(value); 108 } 109 110 /** 111 * Return the progress within the input split 112 * @return 0.0 to 1.0 of the input byte range 113 */ 114 public float getProgress() throws IOException { 115 if (end == start) { 116 return 0.0f; 117 } else { 118 return Math.min(1.0f, (in.getPosition() - start) / (float)(end - start)); 119 } 120 } 121 122 public synchronized long getPos() throws IOException { 123 return in.getPosition(); 124 } 125 126 protected synchronized void seek(long pos) throws IOException { 127 in.seek(pos); 128 } 129 public synchronized void close() throws IOException { in.close(); } 130 131 } 132