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.fs;
019
020 import java.io.*;
021 import java.nio.ByteBuffer;
022
023 import org.apache.hadoop.classification.InterfaceAudience;
024 import org.apache.hadoop.classification.InterfaceStability;
025 import org.apache.hadoop.classification.MapRModified;
026
027 /** Utility that wraps a {@link FSInputStream} in a {@link DataInputStream}
028 * and buffers input through a {@link BufferedInputStream}. */
029 @InterfaceAudience.Public
030 @InterfaceStability.Stable
031 public class FSDataInputStream extends DataInputStream
032 implements Seekable, PositionedReadable, Closeable,
033 ByteBufferReadable, HasFileDescriptor, CanSetDropBehind, CanSetReadahead {
034
035 /**
036 * Type of file advise to be passed on to the underlying file system. This
037 * information can be used to make optimizations such as reclaiming buffers
038 * for files that are no longer needed by the application, etc.
039 */
040 @MapRModified
041 public static enum FadviseType {
042 FILE_DONTNEED,
043 FILE_RANDOM,
044 FILE_SEQUENTIAL;
045 }
046
047 public FSDataInputStream(InputStream in)
048 throws IOException {
049 super(in);
050 if( !(in instanceof Seekable) || !(in instanceof PositionedReadable) ) {
051 throw new IllegalArgumentException(
052 "In is not an instance of Seekable or PositionedReadable");
053 }
054 }
055
056 /**
057 * Seek to the given offset.
058 *
059 * @param desired offset to seek to
060 */
061 @Override
062 public synchronized void seek(long desired) throws IOException {
063 ((Seekable)in).seek(desired);
064 }
065
066 /**
067 * Get the current position in the input stream.
068 *
069 * @return current position in the input stream
070 */
071 @Override
072 public long getPos() throws IOException {
073 return ((Seekable)in).getPos();
074 }
075
076 /**
077 * Read bytes from the given position in the stream to the given buffer.
078 *
079 * @param position position in the input stream to seek
080 * @param buffer buffer into which data is read
081 * @param offset offset into the buffer in which data is written
082 * @param length maximum number of bytes to read
083 * @return total number of bytes read into the buffer, or <code>-1</code>
084 * if there is no more data because the end of the stream has been
085 * reached
086 */
087 @Override
088 public int read(long position, byte[] buffer, int offset, int length)
089 throws IOException {
090 return ((PositionedReadable)in).read(position, buffer, offset, length);
091 }
092
093 /**
094 * Read bytes from the given position in the stream to the given buffer.
095 * Continues to read until <code>length</code> bytes have been read.
096 *
097 * @param position position in the input stream to seek
098 * @param buffer buffer into which data is read
099 * @param offset offset into the buffer in which data is written
100 * @param length the number of bytes to read
101 * @throws EOFException If the end of stream is reached while reading.
102 * If an exception is thrown an undetermined number
103 * of bytes in the buffer may have been written.
104 */
105 @Override
106 public void readFully(long position, byte[] buffer, int offset, int length)
107 throws IOException {
108 ((PositionedReadable)in).readFully(position, buffer, offset, length);
109 }
110
111 /**
112 * See {@link #readFully(long, byte[], int, int)}.
113 */
114 @Override
115 public void readFully(long position, byte[] buffer)
116 throws IOException {
117 ((PositionedReadable)in).readFully(position, buffer, 0, buffer.length);
118 }
119
120 /**
121 * Seek to the given position on an alternate copy of the data.
122 *
123 * @param targetPos position to seek to
124 * @return true if a new source is found, false otherwise
125 */
126 @Override
127 public boolean seekToNewSource(long targetPos) throws IOException {
128 return ((Seekable)in).seekToNewSource(targetPos);
129 }
130
131 /**
132 * Get a reference to the wrapped input stream. Used by unit tests.
133 *
134 * @return the underlying input stream
135 */
136 @InterfaceAudience.LimitedPrivate({"HDFS"})
137 public InputStream getWrappedStream() {
138 return in;
139 }
140
141 @Override
142 public int read(ByteBuffer buf) throws IOException {
143 if (in instanceof ByteBufferReadable) {
144 return ((ByteBufferReadable)in).read(buf);
145 }
146
147 throw new UnsupportedOperationException("Byte-buffer read unsupported by input stream");
148 }
149
150 @Override
151 public FileDescriptor getFileDescriptor() throws IOException {
152 if (in instanceof HasFileDescriptor) {
153 return ((HasFileDescriptor) in).getFileDescriptor();
154 } else if (in instanceof FileInputStream) {
155 return ((FileInputStream) in).getFD();
156 } else {
157 return null;
158 }
159 }
160
161 @Override
162 public void setReadahead(Long readahead)
163 throws IOException, UnsupportedOperationException {
164 try {
165 ((CanSetReadahead)in).setReadahead(readahead);
166 } catch (ClassCastException e) {
167 throw new UnsupportedOperationException(
168 "this stream does not support setting the readahead " +
169 "caching strategy.");
170 }
171 }
172
173 @Override
174 public void setDropBehind(Boolean dropBehind)
175 throws IOException, UnsupportedOperationException {
176 try {
177 ((CanSetDropBehind)in).setDropBehind(dropBehind);
178 } catch (ClassCastException e) {
179 throw new UnsupportedOperationException("this stream does not " +
180 "support setting the drop-behind caching setting.");
181 }
182 }
183
184 /**
185 * Specifies the kind of advise to provide for this stream and the file
186 * offsets to which they apply.
187 *
188 * The default implementation does nothing. Sub classes can override this
189 * behavior.
190 *
191 * @param type advise type
192 * @param offset starting file offset
193 * @param count number of bytes starting from the offset
194 */
195 @MapRModified
196 public void adviseFile(FadviseType type, long offset, long count)
197 throws IOException {
198 }
199
200 /**
201 * Returns the file length.
202 *
203 * @return file length
204 */
205 @MapRModified
206 public long getFileLength() throws IOException {
207 throw new UnsupportedOperationException();
208 }
209
210 /**
211 * Returns the file id as string.
212 *
213 * @return file id as string
214 */
215 @MapRModified
216 public String getFidStr() {
217 throw new UnsupportedOperationException();
218 }
219
220 /**
221 * Returns the server IPs in which the file is stored. Each IP is stored in a
222 * long. For e.g., the first 4 bytes can be used to store the IP in
223 * hexadecimal format and the last 4 bytes to store the port number.
224 *
225 * @return array of server IPs in which the file is stored
226 */
227 @MapRModified
228 public long[] getFidServers() {
229 throw new UnsupportedOperationException();
230 }
231
232 /**
233 * Returns the file chunk size.
234 *
235 * @return file chunk size
236 */
237 @MapRModified
238 public long getChunkSize() {
239 throw new UnsupportedOperationException();
240 }
241 }