Pārlūkot izejas kodu

HDFS-7138. Fix hftp to work with encryption. (clamb via wang)

Andrew Wang 10 gadi atpakaļ
vecāks
revīzija
835ade7064

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -235,6 +235,8 @@ Release 2.6.0 - UNRELEASED
 
     HDFS-7139. Unit test for creating encryption zone on root path. (Zhe Zhang via wang)
 
+    HDFS-7138. Fix hftp to work with encryption. (clamb via wang)
+
   OPTIMIZATIONS
 
     HDFS-6690. Deduplicate xattr names in memory. (wang)

+ 8 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/StreamFile.java

@@ -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);
   }

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStreamFile.java

@@ -35,6 +35,7 @@ import javax.servlet.ServletOutputStream;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
+import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -118,7 +119,7 @@ public class TestStreamFile {
   @Test
   public void testWriteTo() throws IOException {
 
-    FSInputStream fsin = new MockFSInputStream();
+    FSDataInputStream fsdin = new FSDataInputStream(new MockFSInputStream());
     ByteArrayOutputStream os = new ByteArrayOutputStream();
 
     // new int[]{s_1, c_1, s_2, c_2, ..., s_n, c_n} means to test
@@ -135,7 +136,7 @@ public class TestStreamFile {
     assertTrue("Pairs array must be even", pairs.length % 2 == 0);
     
     for (int i = 0; i < pairs.length; i+=2) {
-      StreamFile.copyFromOffset(fsin, os, pairs[i], pairs[i+1]);
+      StreamFile.copyFromOffset(fsdin, os, pairs[i], pairs[i+1]);
       assertArrayEquals("Reading " + pairs[i+1]
                         + " bytes from offset " + pairs[i],
                         getOutputArray(pairs[i], pairs[i+1]),
@@ -154,7 +155,7 @@ public class TestStreamFile {
   
   @Test
   public void testSendPartialData() throws IOException {
-    FSInputStream in = new MockFSInputStream();
+    FSDataInputStream in = new FSDataInputStream(new MockFSInputStream());
     ByteArrayOutputStream os = new ByteArrayOutputStream();
 
     // test if multiple ranges, then 416