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
026 /** Utility that wraps a {@link FSInputStream} in a {@link DataInputStream}
027 * and buffers input through a {@link BufferedInputStream}. */
028 @InterfaceAudience.Public
029 @InterfaceStability.Stable
030 public class FSDataInputStream extends DataInputStream
031 implements Seekable, PositionedReadable, Closeable,
032 ByteBufferReadable, HasFileDescriptor, CanSetDropBehind, CanSetReadahead {
033
034 public FSDataInputStream(InputStream in)
035 throws IOException {
036 super(in);
037 if( !(in instanceof Seekable) || !(in instanceof PositionedReadable) ) {
038 throw new IllegalArgumentException(
039 "In is not an instance of Seekable or PositionedReadable");
040 }
041 }
042
043 /**
044 * Seek to the given offset.
045 *
046 * @param desired offset to seek to
047 */
048 @Override
049 public synchronized void seek(long desired) throws IOException {
050 ((Seekable)in).seek(desired);
051 }
052
053 /**
054 * Get the current position in the input stream.
055 *
056 * @return current position in the input stream
057 */
058 @Override
059 public long getPos() throws IOException {
060 return ((Seekable)in).getPos();
061 }
062
063 /**
064 * Read bytes from the given position in the stream to the given buffer.
065 *
066 * @param position position in the input stream to seek
067 * @param buffer buffer into which data is read
068 * @param offset offset into the buffer in which data is written
069 * @param length maximum number of bytes to read
070 * @return total number of bytes read into the buffer, or <code>-1</code>
071 * if there is no more data because the end of the stream has been
072 * reached
073 */
074 @Override
075 public int read(long position, byte[] buffer, int offset, int length)
076 throws IOException {
077 return ((PositionedReadable)in).read(position, buffer, offset, length);
078 }
079
080 /**
081 * Read bytes from the given position in the stream to the given buffer.
082 * Continues to read until <code>length</code> bytes have been read.
083 *
084 * @param position position in the input stream to seek
085 * @param buffer buffer into which data is read
086 * @param offset offset into the buffer in which data is written
087 * @param length the number of bytes to read
088 * @throws EOFException If the end of stream is reached while reading.
089 * If an exception is thrown an undetermined number
090 * of bytes in the buffer may have been written.
091 */
092 @Override
093 public void readFully(long position, byte[] buffer, int offset, int length)
094 throws IOException {
095 ((PositionedReadable)in).readFully(position, buffer, offset, length);
096 }
097
098 /**
099 * See {@link #readFully(long, byte[], int, int)}.
100 */
101 @Override
102 public void readFully(long position, byte[] buffer)
103 throws IOException {
104 ((PositionedReadable)in).readFully(position, buffer, 0, buffer.length);
105 }
106
107 /**
108 * Seek to the given position on an alternate copy of the data.
109 *
110 * @param targetPos position to seek to
111 * @return true if a new source is found, false otherwise
112 */
113 @Override
114 public boolean seekToNewSource(long targetPos) throws IOException {
115 return ((Seekable)in).seekToNewSource(targetPos);
116 }
117
118 /**
119 * Get a reference to the wrapped input stream. Used by unit tests.
120 *
121 * @return the underlying input stream
122 */
123 @InterfaceAudience.LimitedPrivate({"HDFS"})
124 public InputStream getWrappedStream() {
125 return in;
126 }
127
128 @Override
129 public int read(ByteBuffer buf) throws IOException {
130 if (in instanceof ByteBufferReadable) {
131 return ((ByteBufferReadable)in).read(buf);
132 }
133
134 throw new UnsupportedOperationException("Byte-buffer read unsupported by input stream");
135 }
136
137 @Override
138 public FileDescriptor getFileDescriptor() throws IOException {
139 if (in instanceof HasFileDescriptor) {
140 return ((HasFileDescriptor) in).getFileDescriptor();
141 } else if (in instanceof FileInputStream) {
142 return ((FileInputStream) in).getFD();
143 } else {
144 return null;
145 }
146 }
147
148 @Override
149 public void setReadahead(Long readahead)
150 throws IOException, UnsupportedOperationException {
151 try {
152 ((CanSetReadahead)in).setReadahead(readahead);
153 } catch (ClassCastException e) {
154 throw new UnsupportedOperationException(
155 "this stream does not support setting the readahead " +
156 "caching strategy.");
157 }
158 }
159
160 @Override
161 public void setDropBehind(Boolean dropBehind)
162 throws IOException, UnsupportedOperationException {
163 try {
164 ((CanSetDropBehind)in).setDropBehind(dropBehind);
165 } catch (ClassCastException e) {
166 throw new UnsupportedOperationException("this stream does not " +
167 "support setting the drop-behind caching setting.");
168 }
169 }
170 }