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
019package org.apache.hadoop.hdfs.web;
020
021import java.io.EOFException;
022import java.io.IOException;
023import java.io.InputStream;
024import java.net.HttpURLConnection;
025import java.net.URL;
026import java.util.List;
027import java.util.Map;
028import java.util.StringTokenizer;
029
030import org.apache.commons.io.input.BoundedInputStream;
031import org.apache.hadoop.fs.FSInputStream;
032import org.apache.http.HttpStatus;
033
034import com.google.common.annotations.VisibleForTesting;
035import com.google.common.net.HttpHeaders;
036
037/**
038 * To support HTTP byte streams, a new connection to an HTTP server needs to be
039 * created each time. This class hides the complexity of those multiple
040 * connections from the client. Whenever seek() is called, a new connection
041 * is made on the successive read(). The normal input stream functions are
042 * connected to the currently active input stream.
043 */
044public abstract class ByteRangeInputStream extends FSInputStream {
045
046  /**
047   * This class wraps a URL and provides method to open connection.
048   * It can be overridden to change how a connection is opened.
049   */
050  public static abstract class URLOpener {
051    protected URL url;
052
053    public URLOpener(URL u) {
054      url = u;
055    }
056
057    public void setURL(URL u) {
058      url = u;
059    }
060
061    public URL getURL() {
062      return url;
063    }
064
065    /** Connect to server with a data offset. */
066    protected abstract HttpURLConnection connect(final long offset,
067        final boolean resolved) throws IOException;
068  }
069
070  static class InputStreamAndFileLength {
071    final Long length;
072    final InputStream in;
073
074    InputStreamAndFileLength(Long length, InputStream in) {
075      this.length = length;
076      this.in = in;
077    }
078  }
079
080  enum StreamStatus {
081    NORMAL, SEEK, CLOSED
082  }
083  protected InputStream in;
084  protected final URLOpener originalURL;
085  protected final URLOpener resolvedURL;
086  protected long startPos = 0;
087  protected long currentPos = 0;
088  protected Long fileLength = null;
089
090  StreamStatus status = StreamStatus.SEEK;
091
092  /**
093   * Create with the specified URLOpeners. Original url is used to open the
094   * stream for the first time. Resolved url is used in subsequent requests.
095   * @param o Original url
096   * @param r Resolved url
097   */
098  public ByteRangeInputStream(URLOpener o, URLOpener r) throws IOException {
099    this.originalURL = o;
100    this.resolvedURL = r;
101    getInputStream();
102  }
103
104  protected abstract URL getResolvedUrl(final HttpURLConnection connection
105      ) throws IOException;
106
107  @VisibleForTesting
108  protected InputStream getInputStream() throws IOException {
109    switch (status) {
110      case NORMAL:
111        break;
112      case SEEK:
113        if (in != null) {
114          in.close();
115        }
116        InputStreamAndFileLength fin = openInputStream(startPos);
117        in = fin.in;
118        fileLength = fin.length;
119        status = StreamStatus.NORMAL;
120        break;
121      case CLOSED:
122        throw new IOException("Stream closed");
123    }
124    return in;
125  }
126
127  @VisibleForTesting
128  protected InputStreamAndFileLength openInputStream(long startOffset)
129      throws IOException {
130    // Use the original url if no resolved url exists, eg. if
131    // it's the first time a request is made.
132    final boolean resolved = resolvedURL.getURL() != null;
133    final URLOpener opener = resolved? resolvedURL: originalURL;
134
135    final HttpURLConnection connection = opener.connect(startOffset, resolved);
136    resolvedURL.setURL(getResolvedUrl(connection));
137
138    InputStream in = connection.getInputStream();
139    final Long length;
140    final Map<String, List<String>> headers = connection.getHeaderFields();
141    if (isChunkedTransferEncoding(headers)) {
142      // file length is not known
143      length = null;
144    } else {
145      // for non-chunked transfer-encoding, get content-length
146      long streamlength = getStreamLength(connection, headers);
147      length = startOffset + streamlength;
148
149      // Java has a bug with >2GB request streams.  It won't bounds check
150      // the reads so the transfer blocks until the server times out
151      in = new BoundedInputStream(in, streamlength);
152    }
153
154    return new InputStreamAndFileLength(length, in);
155  }
156
157  private static long getStreamLength(HttpURLConnection connection,
158      Map<String, List<String>> headers) throws IOException {
159    String cl = connection.getHeaderField(HttpHeaders.CONTENT_LENGTH);
160    if (cl == null) {
161      // Try to get the content length by parsing the content range
162      // because HftpFileSystem does not return the content length
163      // if the content is partial.
164      if (connection.getResponseCode() == HttpStatus.SC_PARTIAL_CONTENT) {
165        cl = connection.getHeaderField(HttpHeaders.CONTENT_RANGE);
166        return getLengthFromRange(cl);
167      } else {
168        throw new IOException(HttpHeaders.CONTENT_LENGTH + " is missing: "
169            + headers);
170      }
171    }
172    return Long.parseLong(cl);
173  }
174
175  private static long getLengthFromRange(String cl) throws IOException {
176    try {
177
178      String[] str = cl.substring(6).split("[-/]");
179      return Long.parseLong(str[1]) - Long.parseLong(str[0]) + 1;
180    } catch (Exception e) {
181      throw new IOException(
182          "failed to get content length by parsing the content range: " + cl
183              + " " + e.getMessage());
184    }
185  }
186
187  private static boolean isChunkedTransferEncoding(
188      final Map<String, List<String>> headers) {
189    return contains(headers, HttpHeaders.TRANSFER_ENCODING, "chunked")
190        || contains(headers, HttpHeaders.TE, "chunked");
191  }
192
193  /** Does the HTTP header map contain the given key, value pair? */
194  private static boolean contains(final Map<String, List<String>> headers,
195      final String key, final String value) {
196    final List<String> values = headers.get(key);
197    if (values != null) {
198      for(String v : values) {
199        for(final StringTokenizer t = new StringTokenizer(v, ",");
200            t.hasMoreTokens(); ) {
201          if (value.equalsIgnoreCase(t.nextToken())) {
202            return true;
203          }
204        }
205      }
206    }
207    return false;
208  }
209
210  private int update(final int n) throws IOException {
211    if (n != -1) {
212      currentPos += n;
213    } else if (fileLength != null && currentPos < fileLength) {
214      throw new IOException("Got EOF but currentPos = " + currentPos
215          + " < filelength = " + fileLength);
216    }
217    return n;
218  }
219
220  @Override
221  public int read() throws IOException {
222    final int b = getInputStream().read();
223    update((b == -1) ? -1 : 1);
224    return b;
225  }
226
227  @Override
228  public int read(byte b[], int off, int len) throws IOException {
229    return update(getInputStream().read(b, off, len));
230  }
231
232  /**
233   * Seek to the given offset from the start of the file.
234   * The next read() will be from that location.  Can't
235   * seek past the end of the file.
236   */
237  @Override
238  public void seek(long pos) throws IOException {
239    if (pos != currentPos) {
240      startPos = pos;
241      currentPos = pos;
242      if (status != StreamStatus.CLOSED) {
243        status = StreamStatus.SEEK;
244      }
245    }
246  }
247
248  @Override
249  public int read(long position, byte[] buffer, int offset, int length)
250      throws IOException {
251    try (InputStream in = openInputStream(position).in) {
252      return in.read(buffer, offset, length);
253    }
254  }
255
256  @Override
257  public void readFully(long position, byte[] buffer, int offset, int length)
258      throws IOException {
259    final InputStreamAndFileLength fin = openInputStream(position);
260    if (fin.length != null && length + position > fin.length) {
261      throw new EOFException("The length to read " + length
262          + " exceeds the file length " + fin.length);
263    }
264    try {
265      int nread = 0;
266      while (nread < length) {
267        int nbytes = fin.in.read(buffer, offset + nread, length - nread);
268        if (nbytes < 0) {
269          throw new EOFException("End of file reached before reading fully.");
270        }
271        nread += nbytes;
272      }
273    } finally {
274      fin.in.close();
275    }
276  }
277
278  /**
279   * Return the current offset from the start of the file
280   */
281  @Override
282  public long getPos() throws IOException {
283    return currentPos;
284  }
285
286  /**
287   * Seeks a different copy of the data.  Returns true if
288   * found a new source, false otherwise.
289   */
290  @Override
291  public boolean seekToNewSource(long targetPos) throws IOException {
292    return false;
293  }
294
295  @Override
296  public void close() throws IOException {
297    if (in != null) {
298      in.close();
299      in = null;
300    }
301    status = StreamStatus.CLOSED;
302  }
303}