|
@@ -18,6 +18,7 @@
|
|
|
package org.apache.hadoop.hdfs.server.namenode;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
+import java.io.InputStream;
|
|
|
import java.io.OutputStream;
|
|
|
import java.io.PrintWriter;
|
|
|
import java.util.Enumeration;
|
|
@@ -36,6 +37,7 @@ import org.apache.hadoop.hdfs.DFSInputStream;
|
|
|
import org.apache.hadoop.hdfs.server.common.JspHelper;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.DatanodeJspHelper;
|
|
|
+import org.apache.hadoop.io.IOUtils;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
import org.mortbay.jetty.InclusiveByteRange;
|
|
|
|
|
@@ -96,7 +98,7 @@ public class StreamFile extends DfsServlet {
|
|
|
filename + "\"");
|
|
|
response.setContentType("application/octet-stream");
|
|
|
response.setHeader(CONTENT_LENGTH, "" + fileLen);
|
|
|
- StreamFile.writeTo(in, os, 0L, fileLen);
|
|
|
+ StreamFile.copyFromOffset(in, os, 0L, fileLen);
|
|
|
}
|
|
|
} catch(IOException e) {
|
|
|
if (LOG.isDebugEnabled()) {
|
|
@@ -113,75 +115,46 @@ public class StreamFile extends DfsServlet {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Send a partial content response with the given range. If there are
|
|
|
+ * no satisfiable ranges, or if multiple ranges are requested, which
|
|
|
+ * is unsupported, respond with range not satisfiable.
|
|
|
+ *
|
|
|
+ * @param in stream to read from
|
|
|
+ * @param out stream to write to
|
|
|
+ * @param response http response to use
|
|
|
+ * @param contentLength for the response header
|
|
|
+ * @param ranges to write to respond with
|
|
|
+ * @throws IOException on error sending the response
|
|
|
+ */
|
|
|
static void sendPartialData(FSInputStream in,
|
|
|
- OutputStream os,
|
|
|
+ OutputStream out,
|
|
|
HttpServletResponse response,
|
|
|
long contentLength,
|
|
|
List<?> ranges)
|
|
|
- throws IOException {
|
|
|
-
|
|
|
+ throws IOException {
|
|
|
if (ranges == null || ranges.size() != 1) {
|
|
|
- // if there are no satisfiable ranges, or if multiple ranges are
|
|
|
- // requested (we don't support multiple range requests), send 416 response
|
|
|
response.setContentLength(0);
|
|
|
- int status = HttpServletResponse.SC_REQUESTED_RANGE_NOT_SATISFIABLE;
|
|
|
- response.setStatus(status);
|
|
|
- response.setHeader("Content-Range",
|
|
|
+ response.setStatus(HttpServletResponse.SC_REQUESTED_RANGE_NOT_SATISFIABLE);
|
|
|
+ response.setHeader("Content-Range",
|
|
|
InclusiveByteRange.to416HeaderRangeString(contentLength));
|
|
|
} else {
|
|
|
- // if there is only a single valid range (must be satisfiable
|
|
|
- // since were here now), send that range with a 206 response
|
|
|
InclusiveByteRange singleSatisfiableRange =
|
|
|
(InclusiveByteRange)ranges.get(0);
|
|
|
long singleLength = singleSatisfiableRange.getSize(contentLength);
|
|
|
response.setStatus(HttpServletResponse.SC_PARTIAL_CONTENT);
|
|
|
response.setHeader("Content-Range",
|
|
|
singleSatisfiableRange.toHeaderRangeString(contentLength));
|
|
|
- System.out.println("first: "+singleSatisfiableRange.getFirst(contentLength));
|
|
|
- System.out.println("singleLength: "+singleLength);
|
|
|
-
|
|
|
- StreamFile.writeTo(in,
|
|
|
- os,
|
|
|
- singleSatisfiableRange.getFirst(contentLength),
|
|
|
- singleLength);
|
|
|
+ copyFromOffset(in, out,
|
|
|
+ singleSatisfiableRange.getFirst(contentLength),
|
|
|
+ singleLength);
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- static void writeTo(FSInputStream in,
|
|
|
- OutputStream os,
|
|
|
- long start,
|
|
|
- long count)
|
|
|
- throws IOException {
|
|
|
- byte buf[] = new byte[4096];
|
|
|
- long bytesRemaining = count;
|
|
|
- int bytesRead;
|
|
|
- int bytesToRead;
|
|
|
-
|
|
|
- in.seek(start);
|
|
|
-
|
|
|
- while (true) {
|
|
|
- // number of bytes to read this iteration
|
|
|
- bytesToRead = (int)(bytesRemaining<buf.length ?
|
|
|
- bytesRemaining:
|
|
|
- buf.length);
|
|
|
-
|
|
|
- // number of bytes actually read this iteration
|
|
|
- bytesRead = in.read(buf, 0, bytesToRead);
|
|
|
-
|
|
|
- // if we can't read anymore, break
|
|
|
- if (bytesRead == -1) {
|
|
|
- break;
|
|
|
- }
|
|
|
-
|
|
|
- os.write(buf, 0, bytesRead);
|
|
|
-
|
|
|
- bytesRemaining -= bytesRead;
|
|
|
-
|
|
|
- // if we don't need to read anymore, break
|
|
|
- if (bytesRemaining <= 0) {
|
|
|
- break;
|
|
|
- }
|
|
|
|
|
|
- }
|
|
|
+ /* Copy count bytes at the given offset from one stream to another */
|
|
|
+ static void copyFromOffset(FSInputStream in, OutputStream out, long offset,
|
|
|
+ long count) throws IOException {
|
|
|
+ in.seek(offset);
|
|
|
+ IOUtils.copyBytes(in, out, count);
|
|
|
}
|
|
|
}
|