|
@@ -22,7 +22,11 @@ import java.io.IOException;
|
|
|
import java.io.InputStream;
|
|
|
import java.net.HttpURLConnection;
|
|
|
import java.net.URL;
|
|
|
+import java.util.List;
|
|
|
+import java.util.Map;
|
|
|
+import java.util.StringTokenizer;
|
|
|
|
|
|
+import org.apache.commons.io.input.BoundedInputStream;
|
|
|
import org.apache.hadoop.fs.FSInputStream;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.StreamFile;
|
|
|
|
|
@@ -60,14 +64,14 @@ public abstract class ByteRangeInputStream extends FSInputStream {
|
|
|
}
|
|
|
|
|
|
enum StreamStatus {
|
|
|
- NORMAL, SEEK
|
|
|
+ NORMAL, SEEK, CLOSED
|
|
|
}
|
|
|
protected InputStream in;
|
|
|
protected URLOpener originalURL;
|
|
|
protected URLOpener resolvedURL;
|
|
|
protected long startPos = 0;
|
|
|
protected long currentPos = 0;
|
|
|
- protected long filelength;
|
|
|
+ protected Long fileLength = null;
|
|
|
|
|
|
StreamStatus status = StreamStatus.SEEK;
|
|
|
|
|
@@ -85,63 +89,120 @@ public abstract class ByteRangeInputStream extends FSInputStream {
|
|
|
protected abstract URL getResolvedUrl(final HttpURLConnection connection
|
|
|
) throws IOException;
|
|
|
|
|
|
- private InputStream getInputStream() throws IOException {
|
|
|
- if (status != StreamStatus.NORMAL) {
|
|
|
-
|
|
|
- if (in != null) {
|
|
|
- in.close();
|
|
|
- in = null;
|
|
|
- }
|
|
|
-
|
|
|
- // Use the original url if no resolved url exists, eg. if
|
|
|
- // it's the first time a request is made.
|
|
|
- final boolean resolved = resolvedURL.getURL() != null;
|
|
|
- final URLOpener opener = resolved? resolvedURL: originalURL;
|
|
|
+ protected InputStream getInputStream() throws IOException {
|
|
|
+ switch (status) {
|
|
|
+ case NORMAL:
|
|
|
+ break;
|
|
|
+ case SEEK:
|
|
|
+ if (in != null) {
|
|
|
+ in.close();
|
|
|
+ }
|
|
|
+ in = openInputStream();
|
|
|
+ status = StreamStatus.NORMAL;
|
|
|
+ break;
|
|
|
+ case CLOSED:
|
|
|
+ throw new IOException("Stream closed");
|
|
|
+ }
|
|
|
+ return in;
|
|
|
+ }
|
|
|
+
|
|
|
+ protected InputStream openInputStream() throws IOException {
|
|
|
+ // Use the original url if no resolved url exists, eg. if
|
|
|
+ // it's the first time a request is made.
|
|
|
+ final boolean resolved = resolvedURL.getURL() != null;
|
|
|
+ final URLOpener opener = resolved? resolvedURL: originalURL;
|
|
|
+
|
|
|
+ final HttpURLConnection connection = opener.connect(startPos, resolved);
|
|
|
+ resolvedURL.setURL(getResolvedUrl(connection));
|
|
|
|
|
|
- final HttpURLConnection connection = opener.connect(startPos, resolved);
|
|
|
+ InputStream in = connection.getInputStream();
|
|
|
+ final Map<String, List<String>> headers = connection.getHeaderFields();
|
|
|
+ if (isChunkedTransferEncoding(headers)) {
|
|
|
+ // file length is not known
|
|
|
+ fileLength = null;
|
|
|
+ } else {
|
|
|
+ // for non-chunked transfer-encoding, get content-length
|
|
|
final String cl = connection.getHeaderField(StreamFile.CONTENT_LENGTH);
|
|
|
- filelength = (cl == null) ? -1 : Long.parseLong(cl);
|
|
|
- in = connection.getInputStream();
|
|
|
+ if (cl == null) {
|
|
|
+ throw new IOException(StreamFile.CONTENT_LENGTH + " is missing: "
|
|
|
+ + headers);
|
|
|
+ }
|
|
|
+ final long streamlength = Long.parseLong(cl);
|
|
|
+ fileLength = startPos + streamlength;
|
|
|
|
|
|
- resolvedURL.setURL(getResolvedUrl(connection));
|
|
|
- status = StreamStatus.NORMAL;
|
|
|
+ // Java has a bug with >2GB request streams. It won't bounds check
|
|
|
+ // the reads so the transfer blocks until the server times out
|
|
|
+ in = new BoundedInputStream(in, streamlength);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
return in;
|
|
|
}
|
|
|
|
|
|
- private void update(final boolean isEOF, final int n)
|
|
|
- throws IOException {
|
|
|
- if (!isEOF) {
|
|
|
+ private static boolean isChunkedTransferEncoding(
|
|
|
+ final Map<String, List<String>> headers) {
|
|
|
+ return contains(headers, "Transfer-Encoding", "chunked")
|
|
|
+ || contains(headers, "TE", "chunked");
|
|
|
+ }
|
|
|
+
|
|
|
+ /** Does the HTTP header map contain the given key, value pair? */
|
|
|
+ private static boolean contains(final Map<String, List<String>> headers,
|
|
|
+ final String key, final String value) {
|
|
|
+ final List<String> values = headers.get(key);
|
|
|
+ if (values != null) {
|
|
|
+ for(String v : values) {
|
|
|
+ for(final StringTokenizer t = new StringTokenizer(v, ",");
|
|
|
+ t.hasMoreTokens(); ) {
|
|
|
+ if (value.equalsIgnoreCase(t.nextToken())) {
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+
|
|
|
+ private int update(final int n) throws IOException {
|
|
|
+ if (n != -1) {
|
|
|
currentPos += n;
|
|
|
- } else if (currentPos < filelength) {
|
|
|
+ } else if (fileLength != null && currentPos < fileLength) {
|
|
|
throw new IOException("Got EOF but currentPos = " + currentPos
|
|
|
- + " < filelength = " + filelength);
|
|
|
+ + " < filelength = " + fileLength);
|
|
|
}
|
|
|
+ return n;
|
|
|
}
|
|
|
|
|
|
+ @Override
|
|
|
public int read() throws IOException {
|
|
|
final int b = getInputStream().read();
|
|
|
- update(b == -1, 1);
|
|
|
+ update((b == -1)? -1 : 1);
|
|
|
return b;
|
|
|
}
|
|
|
|
|
|
+ @Override
|
|
|
+ public int read(byte[] b, int off, int len) throws IOException{
|
|
|
+ return update(getInputStream().read(b, off, len));
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Seek to the given offset from the start of the file.
|
|
|
* The next read() will be from that location. Can't
|
|
|
* seek past the end of the file.
|
|
|
*/
|
|
|
+ @Override
|
|
|
public void seek(long pos) throws IOException {
|
|
|
if (pos != currentPos) {
|
|
|
startPos = pos;
|
|
|
currentPos = pos;
|
|
|
- status = StreamStatus.SEEK;
|
|
|
+ if (status != StreamStatus.CLOSED) {
|
|
|
+ status = StreamStatus.SEEK;
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Return the current offset from the start of the file
|
|
|
*/
|
|
|
+ @Override
|
|
|
public long getPos() throws IOException {
|
|
|
return currentPos;
|
|
|
}
|
|
@@ -150,7 +211,17 @@ public abstract class ByteRangeInputStream extends FSInputStream {
|
|
|
* Seeks a different copy of the data. Returns true if
|
|
|
* found a new source, false otherwise.
|
|
|
*/
|
|
|
+ @Override
|
|
|
public boolean seekToNewSource(long targetPos) throws IOException {
|
|
|
return false;
|
|
|
}
|
|
|
-}
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void close() throws IOException {
|
|
|
+ if (in != null) {
|
|
|
+ in.close();
|
|
|
+ in = null;
|
|
|
+ }
|
|
|
+ status = StreamStatus.CLOSED;
|
|
|
+ }
|
|
|
+}
|