|
@@ -18,7 +18,9 @@
|
|
|
|
|
|
package org.apache.hadoop.hdfs.web;
|
|
package org.apache.hadoop.hdfs.web;
|
|
|
|
|
|
|
|
+import java.io.BufferedInputStream;
|
|
import java.io.BufferedOutputStream;
|
|
import java.io.BufferedOutputStream;
|
|
|
|
+import java.io.EOFException;
|
|
import java.io.FileNotFoundException;
|
|
import java.io.FileNotFoundException;
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
import java.io.InputStream;
|
|
import java.io.InputStream;
|
|
@@ -34,8 +36,11 @@ import java.util.List;
|
|
import java.util.Map;
|
|
import java.util.Map;
|
|
import java.util.StringTokenizer;
|
|
import java.util.StringTokenizer;
|
|
|
|
|
|
|
|
+import javax.ws.rs.core.HttpHeaders;
|
|
import javax.ws.rs.core.MediaType;
|
|
import javax.ws.rs.core.MediaType;
|
|
|
|
|
|
|
|
+import org.apache.commons.io.IOUtils;
|
|
|
|
+import org.apache.commons.io.input.BoundedInputStream;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.fs.BlockLocation;
|
|
import org.apache.hadoop.fs.BlockLocation;
|
|
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
|
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
|
@@ -44,6 +49,7 @@ import org.apache.hadoop.fs.CreateFlag;
|
|
import org.apache.hadoop.fs.DelegationTokenRenewer;
|
|
import org.apache.hadoop.fs.DelegationTokenRenewer;
|
|
import org.apache.hadoop.fs.FSDataInputStream;
|
|
import org.apache.hadoop.fs.FSDataInputStream;
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
|
+import org.apache.hadoop.fs.FSInputStream;
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
|
|
import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
|
|
@@ -545,7 +551,7 @@ public class WebHdfsFileSystem extends FileSystem
|
|
* Also implements two-step connects for other operations redirected to
|
|
* Also implements two-step connects for other operations redirected to
|
|
* a DN such as open and checksum
|
|
* a DN such as open and checksum
|
|
*/
|
|
*/
|
|
- private HttpURLConnection connect(URL url) throws IOException {
|
|
|
|
|
|
+ protected HttpURLConnection connect(URL url) throws IOException {
|
|
//redirect hostname and port
|
|
//redirect hostname and port
|
|
String redirectHost = null;
|
|
String redirectHost = null;
|
|
|
|
|
|
@@ -698,7 +704,7 @@ public class WebHdfsFileSystem extends FileSystem
|
|
*/
|
|
*/
|
|
abstract class AbstractFsPathRunner<T> extends AbstractRunner<T> {
|
|
abstract class AbstractFsPathRunner<T> extends AbstractRunner<T> {
|
|
private final Path fspath;
|
|
private final Path fspath;
|
|
- private final Param<?,?>[] parameters;
|
|
|
|
|
|
+ private Param<?,?>[] parameters;
|
|
|
|
|
|
AbstractFsPathRunner(final HttpOpParam.Op op, final Path fspath,
|
|
AbstractFsPathRunner(final HttpOpParam.Op op, final Path fspath,
|
|
Param<?,?>... parameters) {
|
|
Param<?,?>... parameters) {
|
|
@@ -714,6 +720,10 @@ public class WebHdfsFileSystem extends FileSystem
|
|
this.parameters = parameters;
|
|
this.parameters = parameters;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ protected void updateURLParameters(Param<?, ?>... p) {
|
|
|
|
+ this.parameters = p;
|
|
|
|
+ }
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
protected URL getUrl() throws IOException {
|
|
protected URL getUrl() throws IOException {
|
|
if (excludeDatanodes.getValue() != null) {
|
|
if (excludeDatanodes.getValue() != null) {
|
|
@@ -1235,15 +1245,10 @@ public class WebHdfsFileSystem extends FileSystem
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
- public FSDataInputStream open(final Path f, final int buffersize
|
|
|
|
|
|
+ public FSDataInputStream open(final Path f, final int bufferSize
|
|
) throws IOException {
|
|
) throws IOException {
|
|
statistics.incrementReadOps(1);
|
|
statistics.incrementReadOps(1);
|
|
- final HttpOpParam.Op op = GetOpParam.Op.OPEN;
|
|
|
|
- // use a runner so the open can recover from an invalid token
|
|
|
|
- FsPathConnectionRunner runner =
|
|
|
|
- new FsPathConnectionRunner(op, f, new BufferSizeParam(buffersize));
|
|
|
|
- return new FSDataInputStream(new OffsetUrlInputStream(
|
|
|
|
- new UnresolvedUrlOpener(runner), new OffsetUrlOpener(null)));
|
|
|
|
|
|
+ return new FSDataInputStream(new WebHdfsInputStream(f, bufferSize));
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
@@ -1524,4 +1529,346 @@ public class WebHdfsFileSystem extends FileSystem
|
|
InetSocketAddress[] getResolvedNNAddr() {
|
|
InetSocketAddress[] getResolvedNNAddr() {
|
|
return nnAddrs;
|
|
return nnAddrs;
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ @VisibleForTesting
|
|
|
|
+ public void setRetryPolicy(RetryPolicy rp) {
|
|
|
|
+ this.retryPolicy = rp;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * This class is used for opening, reading, and seeking files while using the
|
|
|
|
+ * WebHdfsFileSystem. This class will invoke the retry policy when performing
|
|
|
|
+ * any of these actions.
|
|
|
|
+ */
|
|
|
|
+ @VisibleForTesting
|
|
|
|
+ public class WebHdfsInputStream extends FSInputStream {
|
|
|
|
+ private ReadRunner readRunner = null;
|
|
|
|
+
|
|
|
|
+ WebHdfsInputStream(Path path, int buffersize) throws IOException {
|
|
|
|
+ // Only create the ReadRunner once. Each read's byte array and position
|
|
|
|
+ // will be updated within the ReadRunner object before every read.
|
|
|
|
+ readRunner = new ReadRunner(path, buffersize);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ public int read() throws IOException {
|
|
|
|
+ final byte[] b = new byte[1];
|
|
|
|
+ return (read(b, 0, 1) == -1) ? -1 : (b[0] & 0xff);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ public int read(byte b[], int off, int len) throws IOException {
|
|
|
|
+ return readRunner.read(b, off, len);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ public void seek(long newPos) throws IOException {
|
|
|
|
+ readRunner.seek(newPos);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ public long getPos() throws IOException {
|
|
|
|
+ return readRunner.getPos();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ protected int getBufferSize() throws IOException {
|
|
|
|
+ return readRunner.getBufferSize();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ protected Path getPath() throws IOException {
|
|
|
|
+ return readRunner.getPath();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ public boolean seekToNewSource(long targetPos) throws IOException {
|
|
|
|
+ return false;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ public void close() throws IOException {
|
|
|
|
+ readRunner.close();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public void setFileLength(long len) {
|
|
|
|
+ readRunner.setFileLength(len);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public long getFileLength() {
|
|
|
|
+ return readRunner.getFileLength();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @VisibleForTesting
|
|
|
|
+ ReadRunner getReadRunner() {
|
|
|
|
+ return readRunner;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @VisibleForTesting
|
|
|
|
+ void setReadRunner(ReadRunner rr) {
|
|
|
|
+ this.readRunner = rr;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ enum RunnerState {
|
|
|
|
+ DISCONNECTED, // Connection is closed programmatically by ReadRunner
|
|
|
|
+ OPEN, // Connection has been established by ReadRunner
|
|
|
|
+ SEEK, // Calling code has explicitly called seek()
|
|
|
|
+ CLOSED // Calling code has explicitly called close()
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * This class will allow retries to occur for both open and read operations.
|
|
|
|
+ * The first WebHdfsFileSystem#open creates a new WebHdfsInputStream object,
|
|
|
|
+ * which creates a new ReadRunner object that will be used to open a
|
|
|
|
+ * connection and read or seek into the input stream.
|
|
|
|
+ *
|
|
|
|
+ * ReadRunner is a subclass of the AbstractRunner class, which will run the
|
|
|
|
+ * ReadRunner#getUrl(), ReadRunner#connect(URL), and ReadRunner#getResponse
|
|
|
|
+ * methods within a retry loop, based on the configured retry policy.
|
|
|
|
+ * ReadRunner#connect will create a connection if one has not already been
|
|
|
|
+ * created. Otherwise, it will return the previously created connection
|
|
|
|
+ * object. This is necessary because a new connection should not be created
|
|
|
|
+ * for every read.
|
|
|
|
+ * Likewise, ReadRunner#getUrl will construct a new URL object only if the
|
|
|
|
+ * connection has not previously been established. Otherwise, it will return
|
|
|
|
+ * the previously created URL object.
|
|
|
|
+ * ReadRunner#getResponse will initialize the input stream if it has not
|
|
|
|
+ * already been initialized and read the requested data from the specified
|
|
|
|
+ * input stream.
|
|
|
|
+ */
|
|
|
|
+ @VisibleForTesting
|
|
|
|
+ protected class ReadRunner extends AbstractFsPathRunner<Integer> {
|
|
|
|
+ private InputStream in = null;
|
|
|
|
+ private HttpURLConnection cachedConnection = null;
|
|
|
|
+ private byte[] readBuffer;
|
|
|
|
+ private int readOffset;
|
|
|
|
+ private int readLength;
|
|
|
|
+ private RunnerState runnerState = RunnerState.DISCONNECTED;
|
|
|
|
+ private URL originalUrl = null;
|
|
|
|
+ private URL resolvedUrl = null;
|
|
|
|
+
|
|
|
|
+ private final Path path;
|
|
|
|
+ private final int bufferSize;
|
|
|
|
+ private long pos = 0;
|
|
|
|
+ private long fileLength = 0;
|
|
|
|
+
|
|
|
|
+ /* The following methods are WebHdfsInputStream helpers. */
|
|
|
|
+
|
|
|
|
+ ReadRunner(Path p, int bs) throws IOException {
|
|
|
|
+ super(GetOpParam.Op.OPEN, p, new BufferSizeParam(bs));
|
|
|
|
+ this.path = p;
|
|
|
|
+ this.bufferSize = bs;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ int read(byte[] b, int off, int len) throws IOException {
|
|
|
|
+ if (runnerState == RunnerState.CLOSED) {
|
|
|
|
+ throw new IOException("Stream closed");
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // Before the first read, pos and fileLength will be 0 and readBuffer
|
|
|
|
+ // will all be null. They will be initialized once the first connection
|
|
|
|
+ // is made. Only after that it makes sense to compare pos and fileLength.
|
|
|
|
+ if (pos >= fileLength && readBuffer != null) {
|
|
|
|
+ return -1;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // If a seek is occurring, the input stream will have been closed, so it
|
|
|
|
+ // needs to be reopened. Use the URLRunner to call AbstractRunner#connect
|
|
|
|
+ // with the previously-cached resolved URL and with the 'redirected' flag
|
|
|
|
+ // set to 'true'. The resolved URL contains the URL of the previously
|
|
|
|
+ // opened DN as opposed to the NN. It is preferable to use the resolved
|
|
|
|
+ // URL when creating a connection because it does not hit the NN or every
|
|
|
|
+ // seek, nor does it open a connection to a new DN after every seek.
|
|
|
|
+ // The redirect flag is needed so that AbstractRunner#connect knows the
|
|
|
|
+ // URL is already resolved.
|
|
|
|
+ // Note that when the redirected flag is set, retries are not attempted.
|
|
|
|
+ // So, if the connection fails using URLRunner, clear out the connection
|
|
|
|
+ // and fall through to establish the connection using ReadRunner.
|
|
|
|
+ if (runnerState == RunnerState.SEEK) {
|
|
|
|
+ try {
|
|
|
|
+ final URL rurl = new URL(resolvedUrl + "&" + new OffsetParam(pos));
|
|
|
|
+ cachedConnection = new URLRunner(GetOpParam.Op.OPEN, rurl, true).run();
|
|
|
|
+ } catch (IOException ioe) {
|
|
|
|
+ closeInputStream(RunnerState.DISCONNECTED);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ readBuffer = b;
|
|
|
|
+ readOffset = off;
|
|
|
|
+ readLength = len;
|
|
|
|
+
|
|
|
|
+ int count = -1;
|
|
|
|
+ count = this.run();
|
|
|
|
+ if (count >= 0) {
|
|
|
|
+ statistics.incrementBytesRead(count);
|
|
|
|
+ pos += count;
|
|
|
|
+ } else if (pos < fileLength) {
|
|
|
|
+ throw new EOFException(
|
|
|
|
+ "Premature EOF: pos=" + pos + " < filelength=" + fileLength);
|
|
|
|
+ }
|
|
|
|
+ return count;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ void seek(long newPos) throws IOException {
|
|
|
|
+ if (pos != newPos) {
|
|
|
|
+ pos = newPos;
|
|
|
|
+ closeInputStream(RunnerState.SEEK);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public void close() throws IOException {
|
|
|
|
+ closeInputStream(RunnerState.CLOSED);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /* The following methods are overriding AbstractRunner methods,
|
|
|
|
+ * to be called within the retry policy context by runWithRetry.
|
|
|
|
+ */
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ protected URL getUrl() throws IOException {
|
|
|
|
+ // This method is called every time either a read is executed.
|
|
|
|
+ // The check for connection == null is to ensure that a new URL is only
|
|
|
|
+ // created upon a new connection and not for every read.
|
|
|
|
+ if (cachedConnection == null) {
|
|
|
|
+ // Update URL with current offset. BufferSize doesn't change, but it
|
|
|
|
+ // still must be included when creating the new URL.
|
|
|
|
+ updateURLParameters(new BufferSizeParam(bufferSize),
|
|
|
|
+ new OffsetParam(pos));
|
|
|
|
+ originalUrl = super.getUrl();
|
|
|
|
+ }
|
|
|
|
+ return originalUrl;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /* Only make the connection if it is not already open. Don't cache the
|
|
|
|
+ * connection here. After this method is called, runWithRetry will call
|
|
|
|
+ * validateResponse, and then call the below ReadRunner#getResponse. If
|
|
|
|
+ * the code path makes it that far, then we can cache the connection.
|
|
|
|
+ */
|
|
|
|
+ @Override
|
|
|
|
+ protected HttpURLConnection connect(URL url) throws IOException {
|
|
|
|
+ HttpURLConnection conn = cachedConnection;
|
|
|
|
+ if (conn == null) {
|
|
|
|
+ try {
|
|
|
|
+ conn = super.connect(url);
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ closeInputStream(RunnerState.DISCONNECTED);
|
|
|
|
+ throw e;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ return conn;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /*
|
|
|
|
+ * This method is used to perform reads within the retry policy context.
|
|
|
|
+ * This code is relying on runWithRetry to always call the above connect
|
|
|
|
+ * method and the verifyResponse method prior to calling getResponse.
|
|
|
|
+ */
|
|
|
|
+ @Override
|
|
|
|
+ Integer getResponse(final HttpURLConnection conn)
|
|
|
|
+ throws IOException {
|
|
|
|
+ try {
|
|
|
|
+ // In the "open-then-read" use case, runWithRetry will have executed
|
|
|
|
+ // ReadRunner#connect to make the connection and then executed
|
|
|
|
+ // validateResponse to validate the response code. Only then do we want
|
|
|
|
+ // to cache the connection.
|
|
|
|
+ // In the "read-after-seek" use case, the connection is made and the
|
|
|
|
+ // response is validated by the URLRunner. ReadRunner#read then caches
|
|
|
|
+ // the connection and the ReadRunner#connect will pass on the cached
|
|
|
|
+ // connection
|
|
|
|
+ // In either case, stream initialization is done here if necessary.
|
|
|
|
+ cachedConnection = conn;
|
|
|
|
+ if (in == null) {
|
|
|
|
+ in = initializeInputStream(conn);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ int count = in.read(readBuffer, readOffset, readLength);
|
|
|
|
+ if (count < 0 && pos < fileLength) {
|
|
|
|
+ throw new EOFException(
|
|
|
|
+ "Premature EOF: pos=" + pos + " < filelength=" + fileLength);
|
|
|
|
+ }
|
|
|
|
+ return Integer.valueOf(count);
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ String redirectHost = resolvedUrl.getAuthority();
|
|
|
|
+ if (excludeDatanodes.getValue() != null) {
|
|
|
|
+ excludeDatanodes = new ExcludeDatanodesParam(redirectHost + ","
|
|
|
|
+ + excludeDatanodes.getValue());
|
|
|
|
+ } else {
|
|
|
|
+ excludeDatanodes = new ExcludeDatanodesParam(redirectHost);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // If an exception occurs, close the input stream and null it out so
|
|
|
|
+ // that if the abstract runner decides to retry, it will reconnect.
|
|
|
|
+ closeInputStream(RunnerState.DISCONNECTED);
|
|
|
|
+ throw e;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @VisibleForTesting
|
|
|
|
+ InputStream initializeInputStream(HttpURLConnection conn)
|
|
|
|
+ throws IOException {
|
|
|
|
+ // Cache the resolved URL so that it can be used in the event of
|
|
|
|
+ // a future seek operation.
|
|
|
|
+ resolvedUrl = removeOffsetParam(conn.getURL());
|
|
|
|
+ final String cl = conn.getHeaderField(HttpHeaders.CONTENT_LENGTH);
|
|
|
|
+ InputStream inStream = conn.getInputStream();
|
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
|
+ LOG.debug("open file: " + conn.getURL());
|
|
|
|
+ }
|
|
|
|
+ if (cl != null) {
|
|
|
|
+ long streamLength = Long.parseLong(cl);
|
|
|
|
+ fileLength = pos + streamLength;
|
|
|
|
+ // Java has a bug with >2GB request streams. It won't bounds check
|
|
|
|
+ // the reads so the transfer blocks until the server times out
|
|
|
|
+ inStream = new BoundedInputStream(inStream, streamLength);
|
|
|
|
+ } else {
|
|
|
|
+ fileLength = getHdfsFileStatus(path).getLen();
|
|
|
|
+ }
|
|
|
|
+ // Wrapping in BufferedInputStream because it is more performant than
|
|
|
|
+ // BoundedInputStream by itself.
|
|
|
|
+ runnerState = RunnerState.OPEN;
|
|
|
|
+ return new BufferedInputStream(inStream, bufferSize);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // Close both the InputStream and the connection.
|
|
|
|
+ @VisibleForTesting
|
|
|
|
+ void closeInputStream(RunnerState rs) throws IOException {
|
|
|
|
+ if (in != null) {
|
|
|
|
+ IOUtils.close(cachedConnection);
|
|
|
|
+ in = null;
|
|
|
|
+ }
|
|
|
|
+ cachedConnection = null;
|
|
|
|
+ runnerState = rs;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /* Getters and Setters */
|
|
|
|
+
|
|
|
|
+ @VisibleForTesting
|
|
|
|
+ protected InputStream getInputStream() {
|
|
|
|
+ return in;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @VisibleForTesting
|
|
|
|
+ protected void setInputStream(InputStream inStream) {
|
|
|
|
+ in = inStream;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ Path getPath() {
|
|
|
|
+ return path;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ int getBufferSize() {
|
|
|
|
+ return bufferSize;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ long getFileLength() {
|
|
|
|
+ return fileLength;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ void setFileLength(long len) {
|
|
|
|
+ fileLength = len;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ long getPos() {
|
|
|
|
+ return pos;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
}
|
|
}
|