|
@@ -30,9 +30,9 @@ import javax.servlet.http.HttpServletResponse;
|
|
|
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
-import org.apache.hadoop.fs.FSInputStream;
|
|
|
+import org.apache.hadoop.fs.FSDataInputStream;
|
|
|
import org.apache.hadoop.hdfs.DFSClient;
|
|
|
-import org.apache.hadoop.hdfs.DFSInputStream;
|
|
|
+import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
|
|
|
import org.apache.hadoop.hdfs.server.common.JspHelper;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.DatanodeJspHelper;
|
|
@@ -87,13 +87,13 @@ public class StreamFile extends DfsServlet {
|
|
|
return;
|
|
|
}
|
|
|
|
|
|
- DFSInputStream in = null;
|
|
|
+ HdfsDataInputStream in = null;
|
|
|
OutputStream out = null;
|
|
|
|
|
|
try {
|
|
|
- in = dfs.open(filename);
|
|
|
+ in = dfs.createWrappedInputStream(dfs.open(filename));
|
|
|
out = response.getOutputStream();
|
|
|
- final long fileLen = in.getFileLength();
|
|
|
+ final long fileLen = in.getVisibleLength();
|
|
|
if (reqRanges != null) {
|
|
|
List<InclusiveByteRange> ranges =
|
|
|
InclusiveByteRange.satisfiableRanges(reqRanges, fileLen);
|
|
@@ -136,7 +136,7 @@ public class StreamFile extends DfsServlet {
|
|
|
* @param ranges to write to respond with
|
|
|
* @throws IOException on error sending the response
|
|
|
*/
|
|
|
- static void sendPartialData(FSInputStream in,
|
|
|
+ static void sendPartialData(FSDataInputStream in,
|
|
|
OutputStream out,
|
|
|
HttpServletResponse response,
|
|
|
long contentLength,
|
|
@@ -160,8 +160,8 @@ public class StreamFile extends DfsServlet {
|
|
|
}
|
|
|
|
|
|
/* 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 {
|
|
|
+ static void copyFromOffset(FSDataInputStream in, OutputStream out,
|
|
|
+ long offset, long count) throws IOException {
|
|
|
in.seek(offset);
|
|
|
IOUtils.copyBytes(in, out, count, false);
|
|
|
}
|