You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by el...@apache.org on 2011/06/30 09:05:32 UTC

svn commit: r1141416 - in /hadoop/common/trunk/hdfs/src: java/org/apache/hadoop/hdfs/ java/org/apache/hadoop/hdfs/server/namenode/ test/hdfs/org/apache/hadoop/hdfs/ test/hdfs/org/apache/hadoop/hdfs/server/namenode/

Author: eli
Date: Thu Jun 30 07:05:32 2011
New Revision: 1141416

URL: http://svn.apache.org/viewvc?rev=1141416&view=rev
Log:
Minor update to HDFS-2110.

Modified:
    hadoop/common/trunk/hdfs/src/java/org/apache/hadoop/hdfs/HftpFileSystem.java
    hadoop/common/trunk/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java
    hadoop/common/trunk/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/StreamFile.java
    hadoop/common/trunk/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestByteRangeInputStream.java
    hadoop/common/trunk/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStreamFile.java

Modified: hadoop/common/trunk/hdfs/src/java/org/apache/hadoop/hdfs/HftpFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/src/java/org/apache/hadoop/hdfs/HftpFileSystem.java?rev=1141416&r1=1141415&r2=1141416&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/src/java/org/apache/hadoop/hdfs/HftpFileSystem.java (original)
+++ hadoop/common/trunk/hdfs/src/java/org/apache/hadoop/hdfs/HftpFileSystem.java Thu Jun 30 07:05:32 2011
@@ -70,9 +70,8 @@ import org.xml.sax.XMLReader;
 import org.xml.sax.helpers.DefaultHandler;
 import org.xml.sax.helpers.XMLReaderFactory;
 
-
-
-/** An implementation of a protocol for accessing filesystems over HTTP.
+/**
+ * An implementation of a protocol for accessing filesystems over HTTP.
  * The following implementation provides a limited, read-only interface
  * to a filesystem over HTTP.
  * @see org.apache.hadoop.hdfs.server.namenode.ListPathsServlet
@@ -314,7 +313,7 @@ public class HftpFileSystem extends File
     try {
       connection.setRequestMethod("GET");
       connection.connect();
-    } catch(IOException ioe) {
+    } catch (IOException ioe) {
       throwIOExceptionFromConnection(connection, ioe);
     }
     return connection;

Modified: hadoop/common/trunk/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java?rev=1141416&r1=1141415&r2=1141416&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java (original)
+++ hadoop/common/trunk/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java Thu Jun 30 07:05:32 2011
@@ -117,7 +117,7 @@ public class FileDataServlet extends Dfs
               .getParameter(JspHelper.DELEGATION_PARAMETER_NAME);
 
           HdfsFileStatus info = nn.getFileInfo(path);
-          if ((info != null) && !info.isDir()) {
+          if (info != null && !info.isDir()) {
             try {
               response.sendRedirect(createUri(path, info, ugi, nn, request,
                   delegationToken).toURL().toString());

Modified: hadoop/common/trunk/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/StreamFile.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/StreamFile.java?rev=1141416&r1=1141415&r2=1141416&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/StreamFile.java (original)
+++ hadoop/common/trunk/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/StreamFile.java Thu Jun 30 07:05:32 2011
@@ -48,7 +48,7 @@ public class StreamFile extends DfsServl
 
   public static final String CONTENT_LENGTH = "Content-Length";
 
-  /** getting a client for connecting to dfs */
+  /* Return a DFS client to use to make the given HTTP request */
   protected DFSClient getDFSClient(HttpServletRequest request)
       throws IOException, InterruptedException {
     final Configuration conf =
@@ -59,6 +59,7 @@ public class StreamFile extends DfsServl
     return DatanodeJspHelper.getDFSClient(request, datanode, conf, ugi);
   }
   
+  @SuppressWarnings("unchecked")
   public void doGet(HttpServletRequest request, HttpServletResponse response)
     throws ServletException, IOException {
     final String path = request.getPathInfo() != null ? 
@@ -71,9 +72,10 @@ public class StreamFile extends DfsServl
       return;
     }
     
-    Enumeration<?> reqRanges = request.getHeaders("Range");
-    if (reqRanges != null && !reqRanges.hasMoreElements())
+    Enumeration<String> reqRanges = request.getHeaders("Range");
+    if (reqRanges != null && !reqRanges.hasMoreElements()) {
       reqRanges = null;
+    }
 
     DFSClient dfs;
     try {
@@ -89,29 +91,24 @@ public class StreamFile extends DfsServl
 
     try {
       if (reqRanges != null) {
-        List<?> ranges = InclusiveByteRange.satisfiableRanges(reqRanges,
-                                                           fileLen);
-        StreamFile.sendPartialData(in, os, response, fileLen, ranges);
+        List<InclusiveByteRange> ranges = 
+          InclusiveByteRange.satisfiableRanges(reqRanges, fileLen);
+        StreamFile.sendPartialData(in, os, response, fileLen, ranges, true);
       } else {
         // No ranges, so send entire file
         response.setHeader("Content-Disposition", "attachment; filename=\"" + 
                            filename + "\"");
         response.setContentType("application/octet-stream");
         response.setHeader(CONTENT_LENGTH, "" + fileLen);
-        StreamFile.copyFromOffset(in, os, 0L, fileLen);
+        StreamFile.copyFromOffset(in, os, 0L, fileLen, true);
       }
-    } catch(IOException e) {
+    } catch (IOException e) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("response.isCommitted()=" + response.isCommitted(), e);
       }
       throw e;
     } finally {
-      try {
-        in.close();
-        os.close();
-      } finally {
-        dfs.close();
-      }
+      dfs.close();
     }      
   }
   
@@ -125,13 +122,15 @@ public class StreamFile extends DfsServl
    * @param response http response to use
    * @param contentLength for the response header
    * @param ranges to write to respond with
+   * @param close whether to close the streams
    * @throws IOException on error sending the response
    */
   static void sendPartialData(FSInputStream in,
                               OutputStream out,
                               HttpServletResponse response,
                               long contentLength,
-                              List<?> ranges)
+                              List<InclusiveByteRange> ranges,
+                              boolean close)
       throws IOException {
     if (ranges == null || ranges.size() != 1) {
       response.setContentLength(0);
@@ -139,22 +138,21 @@ public class StreamFile extends DfsServl
       response.setHeader("Content-Range",
                 InclusiveByteRange.to416HeaderRangeString(contentLength));
     } else {
-      InclusiveByteRange singleSatisfiableRange =
-        (InclusiveByteRange)ranges.get(0);
+      InclusiveByteRange singleSatisfiableRange = ranges.get(0);
       long singleLength = singleSatisfiableRange.getSize(contentLength);
       response.setStatus(HttpServletResponse.SC_PARTIAL_CONTENT);
       response.setHeader("Content-Range", 
         singleSatisfiableRange.toHeaderRangeString(contentLength));
       copyFromOffset(in, out,
                      singleSatisfiableRange.getFirst(contentLength),
-                     singleLength);
+                     singleLength, close);
     }
   }
 
   /* 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 {
+      long count, boolean close) throws IOException {
     in.seek(offset);
-    IOUtils.copyBytes(in, out, count);
+    IOUtils.copyBytes(in, out, count, close);
   }
 }

Modified: hadoop/common/trunk/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestByteRangeInputStream.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestByteRangeInputStream.java?rev=1141416&r1=1141415&r2=1141416&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestByteRangeInputStream.java (original)
+++ hadoop/common/trunk/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestByteRangeInputStream.java Thu Jun 30 07:05:32 2011
@@ -146,7 +146,7 @@ public class TestByteRangeInputStream {
     is.seek(101);
     is.read();
 
-    assertNull("Seek to 101 should not result in another request", null);
+    assertNull("Seek to 101 should not result in another request", r.getMsg());
 
     r.setMsg(null);
     is.seek(2500);

Modified: hadoop/common/trunk/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStreamFile.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStreamFile.java?rev=1141416&r1=1141415&r2=1141416&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStreamFile.java (original)
+++ hadoop/common/trunk/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStreamFile.java Thu Jun 30 07:05:32 2011
@@ -28,7 +28,8 @@ import java.util.Vector;
 
 import javax.servlet.http.HttpServletResponse;
 
-import junit.framework.TestCase;
+import org.junit.Test;
+import static org.junit.Assert.*;
 
 import org.apache.hadoop.fs.FSInputStream;
 import org.mortbay.jetty.InclusiveByteRange;
@@ -186,8 +187,7 @@ class MockHttpServletResponse implements
 }
 
 
-
-public class TestStreamFile extends TestCase {
+public class TestStreamFile {
   
   // return an array matching the output of mockfsinputstream
   private static byte[] getOutputArray(int start, int count) {
@@ -200,6 +200,7 @@ public class TestStreamFile extends Test
     return a;
   }
   
+  @Test
   public void testWriteTo() throws IOException, InterruptedException {
 
     FSInputStream fsin = new MockFSInputStream();
@@ -219,7 +220,7 @@ public class TestStreamFile extends Test
     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(fsin, os, pairs[i], pairs[i+1], false);
       assertArrayEquals("Reading " + pairs[i+1]
                         + " bytes from offset " + pairs[i],
                         getOutputArray(pairs[i], pairs[i+1]),
@@ -228,22 +229,24 @@ public class TestStreamFile extends Test
     }
     
   }
-  
-  private List<?> strToRanges(String s, int contentLength) {
+
+  @SuppressWarnings("unchecked")
+  private List<InclusiveByteRange> strToRanges(String s, int contentLength) {
     List<String> l = Arrays.asList(new String[]{"bytes="+s});
     Enumeration<?> e = (new Vector<String>(l)).elements();
     return InclusiveByteRange.satisfiableRanges(e, contentLength);
   }
   
+  @Test
   public void testSendPartialData() throws IOException, InterruptedException {
     FSInputStream in = new MockFSInputStream();
     ByteArrayOutputStream os = new ByteArrayOutputStream();
 
     // test if multiple ranges, then 416
     { 
-      List<?> ranges = strToRanges("0-,10-300", 500);
+      List<InclusiveByteRange> ranges = strToRanges("0-,10-300", 500);
       MockHttpServletResponse response = new MockHttpServletResponse();
-      StreamFile.sendPartialData(in, os, response, 500, ranges);
+      StreamFile.sendPartialData(in, os, response, 500, ranges, false);
       assertEquals("Multiple ranges should result in a 416 error",
                    416, response.getStatus());
     }
@@ -252,16 +255,16 @@ public class TestStreamFile extends Test
     { 
       os.reset();
       MockHttpServletResponse response = new MockHttpServletResponse();
-      StreamFile.sendPartialData(in, os, response, 500, null);
+      StreamFile.sendPartialData(in, os, response, 500, null, false);
       assertEquals("No ranges should result in a 416 error",
                    416, response.getStatus());
     }
 
     // test if invalid single range (out of bounds), then 416
     { 
-      List<?> ranges = strToRanges("600-800", 500);
+      List<InclusiveByteRange> ranges = strToRanges("600-800", 500);
       MockHttpServletResponse response = new MockHttpServletResponse();
-      StreamFile.sendPartialData(in, os, response, 500, ranges);
+      StreamFile.sendPartialData(in, os, response, 500, ranges, false);
       assertEquals("Single (but invalid) range should result in a 416",
                    416, response.getStatus());
     }
@@ -269,9 +272,9 @@ public class TestStreamFile extends Test
       
     // test if one (valid) range, then 206
     { 
-      List<?> ranges = strToRanges("100-300", 500);
+      List<InclusiveByteRange> ranges = strToRanges("100-300", 500);
       MockHttpServletResponse response = new MockHttpServletResponse();
-      StreamFile.sendPartialData(in, os, response, 500, ranges);
+      StreamFile.sendPartialData(in, os, response, 500, ranges, false);
       assertEquals("Single (valid) range should result in a 206",
                    206, response.getStatus());
       assertArrayEquals("Byte range from 100-300",