You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by sz...@apache.org on 2011/09/13 10:37:39 UTC

svn commit: r1170087 - in /hadoop/common/branches/branch-0.20-security: ./ src/core/org/apache/hadoop/io/ src/hdfs/org/apache/hadoop/hdfs/ src/hdfs/org/apache/hadoop/hdfs/server/datanode/web/resources/ src/hdfs/org/apache/hadoop/hdfs/server/namenode/we...

Author: szetszwo
Date: Tue Sep 13 08:37:38 2011
New Revision: 1170087

URL: http://svn.apache.org/viewvc?rev=1170087&view=rev
Log:
HDFS-2317. Support read access to HDFS in webhdfs.

Added:
    hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/ByteRangeInputStream.java
    hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/web/resources/LengthParam.java
    hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/web/resources/OffsetParam.java
Modified:
    hadoop/common/branches/branch-0.20-security/CHANGES.txt
    hadoop/common/branches/branch-0.20-security/src/core/org/apache/hadoop/io/IOUtils.java
    hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/DFSClient.java
    hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/HftpFileSystem.java
    hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/server/datanode/web/resources/DatanodeWebHdfsMethods.java
    hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
    hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
    hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
    hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/web/resources/UriFsPathParam.java
    hadoop/common/branches/branch-0.20-security/src/test/org/apache/hadoop/hdfs/MiniDFSCluster.java
    hadoop/common/branches/branch-0.20-security/src/test/org/apache/hadoop/hdfs/web/TestFSMainOperationsWebHdfs.java
    hadoop/common/branches/branch-0.20-security/src/test/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java

Modified: hadoop/common/branches/branch-0.20-security/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security/CHANGES.txt?rev=1170087&r1=1170086&r2=1170087&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security/CHANGES.txt (original)
+++ hadoop/common/branches/branch-0.20-security/CHANGES.txt Tue Sep 13 08:37:38 2011
@@ -42,6 +42,8 @@ Release 0.20.205.0 - unreleased
     HDFS-2284. Add a new FileSystem, webhdfs://, for supporting write Http
     access to HDFS.  (szetszwo)
 
+    HDFS-2317. Support read access to HDFS in webhdfs.  (szetszwo)
+
   BUG FIXES
 
     MAPREDUCE-2324. Removed usage of broken

Modified: hadoop/common/branches/branch-0.20-security/src/core/org/apache/hadoop/io/IOUtils.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security/src/core/org/apache/hadoop/io/IOUtils.java?rev=1170087&r1=1170086&r2=1170087&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security/src/core/org/apache/hadoop/io/IOUtils.java (original)
+++ hadoop/common/branches/branch-0.20-security/src/core/org/apache/hadoop/io/IOUtils.java Tue Sep 13 08:37:38 2011
@@ -100,6 +100,44 @@ public class IOUtils {
     copyBytes(in, out, conf.getInt("io.file.buffer.size", 4096),  close);
   }
   
+  /**
+   * Copies the specified length of bytes from in to out.
+   *
+   * @param in InputStream to read from
+   * @param out OutputStream to write to
+   * @param length number of bytes to copy
+   * @param bufferSize the size of the buffer 
+   * @param close whether to close the streams
+   * @throws IOException if bytes can not be read or written
+   */
+  public static void copyBytes(InputStream in, OutputStream out,
+      final long length, final int bufferSize, final boolean close
+      ) throws IOException {
+    final byte buf[] = new byte[bufferSize];
+    try {
+      int n = 0;
+      for(long remaining = length; remaining > 0 && n != -1; remaining -= n) {
+        final int toRead = remaining < buf.length? (int)remaining : buf.length;
+        n = in.read(buf, 0, toRead);
+        if (n > 0) {
+          out.write(buf, 0, n);
+        }
+      }
+
+      if (close) {
+        out.close();
+        out = null;
+        in.close();
+        in = null;
+      }
+    } finally {
+      if (close) {
+        closeStream(out);
+        closeStream(in);
+      }
+    }
+  }
+
   /** Reads len bytes in a loop.
    * @param in The InputStream to read from
    * @param buf The buffer to fill

Added: hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/ByteRangeInputStream.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/ByteRangeInputStream.java?rev=1170087&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/ByteRangeInputStream.java (added)
+++ hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/ByteRangeInputStream.java Tue Sep 13 08:37:38 2011
@@ -0,0 +1,175 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.HttpURLConnection;
+import java.net.URL;
+
+import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.hdfs.server.namenode.StreamFile;
+
+/**
+ * To support HTTP byte streams, a new connection to an HTTP server needs to be
+ * created each time. This class hides the complexity of those multiple 
+ * connections from the client. Whenever seek() is called, a new connection
+ * is made on the successive read(). The normal input stream functions are 
+ * connected to the currently active input stream. 
+ */
+public class ByteRangeInputStream extends FSInputStream {
+  
+  /**
+   * This class wraps a URL to allow easy mocking when testing. The URL class
+   * cannot be easily mocked because it is public.
+   */
+  static class URLOpener {
+    protected URL url;
+  
+    public URLOpener(URL u) {
+      url = u;
+    }
+  
+    public void setURL(URL u) {
+      url = u;
+    }
+  
+    public URL getURL() {
+      return url;
+    }
+  
+    public HttpURLConnection openConnection() throws IOException {
+      return (HttpURLConnection)url.openConnection();
+    }  
+  }
+  
+  enum StreamStatus {
+    NORMAL, SEEK
+  }
+  protected InputStream in;
+  protected URLOpener originalURL;
+  protected URLOpener resolvedURL;
+  protected long startPos = 0;
+  protected long currentPos = 0;
+  protected long filelength;
+
+  StreamStatus status = StreamStatus.SEEK;
+
+  /** Create an input stream with the URL. */
+  public ByteRangeInputStream(final URL url) {
+    this(new URLOpener(url), new URLOpener(null));
+  }
+  
+  ByteRangeInputStream(URLOpener o, URLOpener r) {
+    this.originalURL = o;
+    this.resolvedURL = r;
+  }
+  
+  private InputStream getInputStream() throws IOException {
+    if (status != StreamStatus.NORMAL) {
+      
+      if (in != null) {
+        in.close();
+        in = null;
+      }
+      
+      // Use the original url if no resolved url exists, eg. if
+      // it's the first time a request is made.
+      final URLOpener opener =
+        (resolvedURL.getURL() == null) ? originalURL : resolvedURL;
+
+      final HttpURLConnection connection = opener.openConnection();
+      try {
+        connection.setRequestMethod("GET");
+        if (startPos != 0) {
+          connection.setRequestProperty("Range", "bytes="+startPos+"-");
+        }
+        connection.connect();
+        final String cl = connection.getHeaderField(StreamFile.CONTENT_LENGTH);
+        filelength = (cl == null) ? -1 : Long.parseLong(cl);
+        if (HftpFileSystem.LOG.isDebugEnabled()) {
+          HftpFileSystem.LOG.debug("filelength = " + filelength);
+        }
+        in = connection.getInputStream();
+      } catch (IOException ioe) {
+        HftpFileSystem.throwIOExceptionFromConnection(connection, ioe);
+      }
+      
+      int respCode = connection.getResponseCode();
+      if (startPos != 0 && respCode != HttpURLConnection.HTTP_PARTIAL) {
+        // We asked for a byte range but did not receive a partial content
+        // response...
+        throw new IOException("HTTP_PARTIAL expected, received " + respCode);
+      } else if (startPos == 0 && respCode != HttpURLConnection.HTTP_OK) {
+        // We asked for all bytes from the beginning but didn't receive a 200
+        // response (none of the other 2xx codes are valid here)
+        throw new IOException("HTTP_OK expected, received " + respCode);
+      }
+
+      resolvedURL.setURL(connection.getURL());
+      status = StreamStatus.NORMAL;
+    }
+    
+    return in;
+  }
+  
+  private void update(final boolean isEOF, final int n)
+      throws IOException {
+    if (!isEOF) {
+      currentPos += n;
+    } else if (currentPos < filelength) {
+      throw new IOException("Got EOF but currentPos = " + currentPos
+          + " < filelength = " + filelength);
+    }
+  }
+
+  public int read() throws IOException {
+    final int b = getInputStream().read();
+    update(b == -1, 1);
+    return b;
+  }
+  
+  /**
+   * Seek to the given offset from the start of the file.
+   * The next read() will be from that location.  Can't
+   * seek past the end of the file.
+   */
+  public void seek(long pos) throws IOException {
+    if (pos != currentPos) {
+      startPos = pos;
+      currentPos = pos;
+      status = StreamStatus.SEEK;
+    }
+  }
+
+  /**
+   * Return the current offset from the start of the file
+   */
+  public long getPos() throws IOException {
+    return currentPos;
+  }
+
+  /**
+   * Seeks a different copy of the data.  Returns true if
+   * found a new source, false otherwise.
+   */
+  public boolean seekToNewSource(long targetPos) throws IOException {
+    return false;
+  }
+}
\ No newline at end of file

Modified: hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/DFSClient.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/DFSClient.java?rev=1170087&r1=1170086&r2=1170087&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/DFSClient.java (original)
+++ hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/DFSClient.java Tue Sep 13 08:37:38 2011
@@ -491,7 +491,7 @@ public class DFSClient implements FSCons
    * inner subclass of InputStream that does the right out-of-band
    * work.
    */
-  DFSInputStream open(String src, int buffersize, boolean verifyChecksum,
+  public DFSInputStream open(String src, int buffersize, boolean verifyChecksum,
                       FileSystem.Statistics stats
       ) throws IOException {
     checkOpen();
@@ -2359,7 +2359,7 @@ public class DFSClient implements FSCons
    * The Hdfs implementation of {@link FSDataInputStream}
    */
   public static class DFSDataInputStream extends FSDataInputStream {
-    DFSDataInputStream(DFSInputStream in)
+    public DFSDataInputStream(DFSInputStream in)
       throws IOException {
       super(in);
     }

Modified: hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/HftpFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/HftpFileSystem.java?rev=1170087&r1=1170086&r2=1170087&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/HftpFileSystem.java (original)
+++ hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/HftpFileSystem.java Tue Sep 13 08:37:38 2011
@@ -265,6 +265,15 @@ public class HftpFileSystem extends File
     return ugiParamenter.toString();
   }
 
+  static Void throwIOExceptionFromConnection(
+      final HttpURLConnection connection, final IOException ioe
+      ) throws IOException {
+    final int code = connection.getResponseCode();
+    final String s = connection.getResponseMessage();
+    throw s == null? ioe:
+        new IOException(s + " (error code=" + code + ")", ioe);
+  }
+
   /**
    * Open an HTTP connection to the namenode to read file data and metadata.
    * @param path The path component of the URL

Modified: hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/server/datanode/web/resources/DatanodeWebHdfsMethods.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/server/datanode/web/resources/DatanodeWebHdfsMethods.java?rev=1170087&r1=1170086&r2=1170087&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/server/datanode/web/resources/DatanodeWebHdfsMethods.java (original)
+++ hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/server/datanode/web/resources/DatanodeWebHdfsMethods.java Tue Sep 13 08:37:38 2011
@@ -19,12 +19,15 @@ package org.apache.hadoop.hdfs.server.da
 
 import java.io.IOException;
 import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URISyntaxException;
 
 import javax.servlet.ServletContext;
 import javax.ws.rs.Consumes;
 import javax.ws.rs.DefaultValue;
+import javax.ws.rs.GET;
 import javax.ws.rs.POST;
 import javax.ws.rs.PUT;
 import javax.ws.rs.Path;
@@ -34,18 +37,23 @@ import javax.ws.rs.QueryParam;
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
+import javax.ws.rs.core.StreamingOutput;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.DFSClient.DFSDataInputStream;
 import org.apache.hadoop.hdfs.DFSClient.DFSOutputStream;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.hdfs.web.resources.BlockSizeParam;
 import org.apache.hadoop.hdfs.web.resources.BufferSizeParam;
+import org.apache.hadoop.hdfs.web.resources.GetOpParam;
+import org.apache.hadoop.hdfs.web.resources.LengthParam;
+import org.apache.hadoop.hdfs.web.resources.OffsetParam;
 import org.apache.hadoop.hdfs.web.resources.OverwriteParam;
 import org.apache.hadoop.hdfs.web.resources.Param;
 import org.apache.hadoop.hdfs.web.resources.PermissionParam;
@@ -58,7 +66,7 @@ import org.apache.hadoop.io.IOUtils;
 /** Web-hdfs DataNode implementation. */
 @Path("")
 public class DatanodeWebHdfsMethods {
-  private static final Log LOG = LogFactory.getLog(DatanodeWebHdfsMethods.class);
+  public static final Log LOG = LogFactory.getLog(DatanodeWebHdfsMethods.class);
 
   private @Context ServletContext context;
 
@@ -156,4 +164,56 @@ public class DatanodeWebHdfsMethods {
       throw new UnsupportedOperationException(op + " is not supported");
     }
   }
+
+  /** Handle HTTP GET request. */
+  @GET
+  @Path("{" + UriFsPathParam.NAME + ":.*}")
+  @Produces({MediaType.APPLICATION_OCTET_STREAM, MediaType.APPLICATION_JSON})
+  public Response get(
+      @PathParam(UriFsPathParam.NAME) final UriFsPathParam path,
+      @QueryParam(GetOpParam.NAME) @DefaultValue(GetOpParam.DEFAULT)
+          final GetOpParam op,
+      @QueryParam(OffsetParam.NAME) @DefaultValue(OffsetParam.DEFAULT)
+          final OffsetParam offset,
+      @QueryParam(LengthParam.NAME) @DefaultValue(LengthParam.DEFAULT)
+          final LengthParam length,
+      @QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT)
+          final BufferSizeParam bufferSize
+      ) throws IOException, URISyntaxException {
+
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(op + ": " + path
+          + Param.toSortedString(", ", offset, length, bufferSize));
+    }
+
+    final String fullpath = path.getAbsolutePath();
+    final DataNode datanode = (DataNode)context.getAttribute("datanode");
+
+    switch(op.getValue()) {
+    case OPEN:
+    {
+      final Configuration conf = new Configuration(datanode.getConf());
+      final InetSocketAddress nnRpcAddr = NameNode.getAddress(conf);
+      final DFSClient dfsclient = new DFSClient(nnRpcAddr, conf);
+      final DFSDataInputStream in = new DFSClient.DFSDataInputStream(
+          dfsclient.open(fullpath, bufferSize.getValue(), true, null));
+      in.seek(offset.getValue());
+
+      final StreamingOutput streaming = new StreamingOutput() {
+        @Override
+        public void write(final OutputStream out) throws IOException {
+          final Long n = length.getValue();
+          if (n == null) {
+            IOUtils.copyBytes(in, out, bufferSize.getValue());
+          } else {
+            IOUtils.copyBytes(in, out, n, bufferSize.getValue(), false);
+          }
+        }
+      };
+      return Response.ok(streaming).type(MediaType.APPLICATION_OCTET_STREAM).build();
+    }
+    default:
+      throw new UnsupportedOperationException(op + " is not supported");
+    }
+  }
 }

Modified: hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java?rev=1170087&r1=1170086&r2=1170087&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java (original)
+++ hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java Tue Sep 13 08:37:38 2011
@@ -17,8 +17,11 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.web.resources;
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.PrintStream;
 import java.net.URI;
 import java.net.URISyntaxException;
 
@@ -36,10 +39,12 @@ import javax.ws.rs.QueryParam;
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
+import javax.ws.rs.core.StreamingOutput;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.namenode.JspHelper;
@@ -54,7 +59,9 @@ import org.apache.hadoop.hdfs.web.resour
 import org.apache.hadoop.hdfs.web.resources.GetOpParam;
 import org.apache.hadoop.hdfs.web.resources.GroupParam;
 import org.apache.hadoop.hdfs.web.resources.HttpOpParam;
+import org.apache.hadoop.hdfs.web.resources.LengthParam;
 import org.apache.hadoop.hdfs.web.resources.ModificationTimeParam;
+import org.apache.hadoop.hdfs.web.resources.OffsetParam;
 import org.apache.hadoop.hdfs.web.resources.OverwriteParam;
 import org.apache.hadoop.hdfs.web.resources.OwnerParam;
 import org.apache.hadoop.hdfs.web.resources.Param;
@@ -73,15 +80,23 @@ public class NamenodeWebHdfsMethods {
   private @Context ServletContext context;
 
   private static DatanodeInfo chooseDatanode(final NameNode namenode,
-      final String path, final HttpOpParam.Op op) throws IOException {
-    if (op == PostOpParam.Op.APPEND) {
+      final String path, final HttpOpParam.Op op, final long openOffset
+      ) throws IOException {
+    if (op == GetOpParam.Op.OPEN || op == PostOpParam.Op.APPEND) {
       final HdfsFileStatus status = namenode.getFileInfo(path);
       final long len = status.getLen();
+      if (op == GetOpParam.Op.OPEN && (openOffset < 0L || openOffset >= len)) {
+        throw new IOException("Offset=" + openOffset + " out of the range [0, "
+          + len + "); " + op + ", path=" + path);
+      }
+
       if (len > 0) {
-        final LocatedBlocks locations = namenode.getBlockLocations(path, len-1, 1);
+        final long offset = op == GetOpParam.Op.OPEN? openOffset: len - 1;
+        final LocatedBlocks locations = namenode.getBlockLocations(
+            path, offset, 1);
         final int count = locations.locatedBlockCount();
         if (count > 0) {
-          return JspHelper.bestNode(locations.get(count - 1));
+          return JspHelper.bestNode(locations.get(0));
         }
       }
     } 
@@ -90,9 +105,9 @@ public class NamenodeWebHdfsMethods {
   }
 
   private static URI redirectURI(final NameNode namenode,
-      final String path, final HttpOpParam.Op op,
+      final String path, final HttpOpParam.Op op, final long openOffset,
       final Param<?, ?>... parameters) throws URISyntaxException, IOException {
-    final DatanodeInfo dn = chooseDatanode(namenode, path, op);
+    final DatanodeInfo dn = chooseDatanode(namenode, path, op, openOffset);
     final String query = op.toQueryString() + Param.toSortedString("&", parameters);
     final String uripath = "/" + WebHdfsFileSystem.PATH_PREFIX + path;
 
@@ -138,8 +153,9 @@ public class NamenodeWebHdfsMethods {
 
     if (LOG.isTraceEnabled()) {
       LOG.trace(op + ": " + path
-            + Param.toSortedString(", ", dstPath, owner, group, permission,
-                overwrite, bufferSize, replication, blockSize));
+          + Param.toSortedString(", ", dstPath, owner, group, permission,
+              overwrite, bufferSize, replication, blockSize,
+              modificationTime, accessTime));
     }
 
     final String fullpath = path.getAbsolutePath();
@@ -148,7 +164,7 @@ public class NamenodeWebHdfsMethods {
     switch(op.getValue()) {
     case CREATE:
     {
-      final URI uri = redirectURI(namenode, fullpath, op.getValue(),
+      final URI uri = redirectURI(namenode, fullpath, op.getValue(), -1L,
           permission, overwrite, bufferSize, replication, blockSize);
       return Response.temporaryRedirect(uri).build();
     } 
@@ -215,7 +231,8 @@ public class NamenodeWebHdfsMethods {
     switch(op.getValue()) {
     case APPEND:
     {
-      final URI uri = redirectURI(namenode, fullpath, op.getValue(), bufferSize);
+      final URI uri = redirectURI(namenode, fullpath, op.getValue(), -1L,
+          bufferSize);
       return Response.temporaryRedirect(uri).build();
     }
     default:
@@ -231,9 +248,15 @@ public class NamenodeWebHdfsMethods {
   @Produces({MediaType.APPLICATION_OCTET_STREAM, MediaType.APPLICATION_JSON})
   public Response root(
       @QueryParam(GetOpParam.NAME) @DefaultValue(GetOpParam.DEFAULT)
-          final GetOpParam op
-      ) throws IOException {
-    return get(ROOT, op);
+          final GetOpParam op,
+      @QueryParam(OffsetParam.NAME) @DefaultValue(OffsetParam.DEFAULT)
+          final OffsetParam offset,
+      @QueryParam(LengthParam.NAME) @DefaultValue(LengthParam.DEFAULT)
+          final LengthParam length,
+      @QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT)
+          final BufferSizeParam bufferSize
+      ) throws IOException, URISyntaxException {
+    return get(ROOT, op, offset, length, bufferSize);
   }
 
   /** Handle HTTP GET request. */
@@ -243,27 +266,88 @@ public class NamenodeWebHdfsMethods {
   public Response get(
       @PathParam(UriFsPathParam.NAME) final UriFsPathParam path,
       @QueryParam(GetOpParam.NAME) @DefaultValue(GetOpParam.DEFAULT)
-          final GetOpParam op
-      ) throws IOException {
+          final GetOpParam op,
+      @QueryParam(OffsetParam.NAME) @DefaultValue(OffsetParam.DEFAULT)
+          final OffsetParam offset,
+      @QueryParam(LengthParam.NAME) @DefaultValue(LengthParam.DEFAULT)
+          final LengthParam length,
+      @QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT)
+          final BufferSizeParam bufferSize
+      ) throws IOException, URISyntaxException {
 
     if (LOG.isTraceEnabled()) {
       LOG.trace(op + ", " + path
-          + Param.toSortedString(", "));
+          + Param.toSortedString(", ", offset, length, bufferSize));
     }
 
+    final NameNode namenode = (NameNode)context.getAttribute("name.node");
+    final String fullpath = path.getAbsolutePath();
+
     switch(op.getValue()) {
+    case OPEN:
+    {
+      final URI uri = redirectURI(namenode, fullpath, op.getValue(),
+          offset.getValue(), offset, length, bufferSize);
+      return Response.temporaryRedirect(uri).build();
+    }
     case GETFILESTATUS:
-      final NameNode namenode = (NameNode)context.getAttribute("name.node");
-      final String fullpath = path.getAbsolutePath();
+    {
       final HdfsFileStatus status = namenode.getFileInfo(fullpath);
       final String js = JsonUtil.toJsonString(status);
       return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
-
+    }
+    case LISTSTATUS:
+    {
+      final StreamingOutput streaming = getListingStream(namenode, fullpath);
+      return Response.ok(streaming).type(MediaType.APPLICATION_JSON).build();
+    }
     default:
       throw new UnsupportedOperationException(op + " is not supported");
     }    
   }
 
+  private static DirectoryListing getDirectoryListing(final NameNode np,
+      final String p, byte[] startAfter) throws IOException {
+    final DirectoryListing listing = np.getListing(p, startAfter);
+    if (listing == null) { // the directory does not exist
+      throw new FileNotFoundException("File " + p + " does not exist.");
+    }
+    return listing;
+  }
+  
+  private static StreamingOutput getListingStream(final NameNode np, 
+      final String p) throws IOException {
+    final DirectoryListing first = getDirectoryListing(np, p,
+        HdfsFileStatus.EMPTY_NAME);
+
+    return new StreamingOutput() {
+      @Override
+      public void write(final OutputStream outstream) throws IOException {
+        final PrintStream out = new PrintStream(outstream);
+        out.print('[');
+
+        final HdfsFileStatus[] partial = first.getPartialListing();
+        if (partial.length > 0) {
+          out.print(JsonUtil.toJsonString(partial[0]));
+        }
+        for(int i = 1; i < partial.length; i++) {
+          out.println(',');
+          out.print(JsonUtil.toJsonString(partial[i]));
+        }
+
+        for(DirectoryListing curr = first; curr.hasMore(); ) { 
+          curr = getDirectoryListing(np, p, curr.getLastName());
+          for(HdfsFileStatus s : curr.getPartialListing()) {
+            out.println(',');
+            out.print(JsonUtil.toJsonString(s));
+          }
+        }
+        
+        out.println(']');
+      }
+    };
+  }
+
   /** Handle HTTP DELETE request. */
   @DELETE
   @Path("{path:.*}")

Modified: hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java?rev=1170087&r1=1170086&r2=1170087&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java (original)
+++ hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java Tue Sep 13 08:37:38 2011
@@ -26,12 +26,15 @@ import java.io.InputStreamReader;
 import java.net.HttpURLConnection;
 import java.net.URI;
 import java.net.URISyntaxException;
+import java.net.URL;
 import java.util.Map;
 
+import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.ByteRangeInputStream;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HftpFileSystem;
 import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
@@ -102,12 +105,11 @@ public class WebHdfsFileSystem extends H
   }
 
   @SuppressWarnings("unchecked")
-  private static Map<String, Object> jsonParse(final InputStream in
-      ) throws IOException {
+  private static <T> T jsonParse(final InputStream in) throws IOException {
     if (in == null) {
       throw new IOException("The input stream is null.");
     }
-    return (Map<String, Object>)JSON.parse(new InputStreamReader(in));
+    return (T)JSON.parse(new InputStreamReader(in));
   }
 
   private static void validateResponse(final HttpOpParam.Op op,
@@ -118,7 +120,7 @@ public class WebHdfsFileSystem extends H
       try {
         m = jsonParse(conn.getErrorStream());
       } catch(IOException e) {
-        throw new IOException("Unexpected HTTP response: code = " + code + " != "
+        throw new IOException("Unexpected HTTP response: code=" + code + " != "
             + op.getExpectedHttpResponseCode() + ", " + op.toQueryString()
             + ", message=" + conn.getResponseMessage(), e);
       }
@@ -132,14 +134,43 @@ public class WebHdfsFileSystem extends H
     }
   }
 
-  private HttpURLConnection httpConnect(final HttpOpParam.Op op, final Path fspath,
+  /**
+   * Return a URL pointing to given path on the namenode.
+   *
+   * @param path to obtain the URL for
+   * @param query string to append to the path
+   * @return namenode URL referring to the given path
+   * @throws IOException on error constructing the URL
+   */
+  protected URL getNamenodeURL(String path, String query) throws IOException {
+    final URL url = new URL("http", nnAddr.getHostName(),
+          nnAddr.getPort(), path + '?' + query);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("url=" + url);
+    }
+    return url;
+  }
+
+  private URL toUrl(final HttpOpParam.Op op, final Path fspath,
       final Param<?,?>... parameters) throws IOException {
     //initialize URI path and query
-    final String uripath = "/" + PATH_PREFIX + makeQualified(fspath).toUri().getPath();
-    final String query = op.toQueryString() + Param.toSortedString("&", parameters);
+    final String path = "/" + PATH_PREFIX
+        + makeQualified(fspath).toUri().getPath();
+    final String query = op.toQueryString()
+        + Param.toSortedString("&", parameters);
+    final URL url = getNamenodeURL(path, query);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("url=" + url);
+    }
+    return url;
+  }
+
+  private HttpURLConnection httpConnect(final HttpOpParam.Op op, final Path fspath,
+      final Param<?,?>... parameters) throws IOException {
+    final URL url = toUrl(op, fspath, parameters);
 
     //connect and get response
-    final HttpURLConnection conn = openConnection(uripath, query);
+    final HttpURLConnection conn = (HttpURLConnection)url.openConnection();
     try {
       conn.setRequestMethod(op.getType().toString());
       conn.setDoOutput(op.getDoOutput());
@@ -155,7 +186,17 @@ public class WebHdfsFileSystem extends H
     }
   }
 
-  private Map<String, Object> run(final HttpOpParam.Op op, final Path fspath,
+  /**
+   * Run a http operation.
+   * Connect to the http server, validate response, and obtain the JSON output.
+   * 
+   * @param op http operation
+   * @param fspath file system path
+   * @param parameters parameters for the operation
+   * @return a JSON object, e.g. Object[], Map<String, Object>, etc.
+   * @throws IOException
+   */
+  private <T> T run(final HttpOpParam.Op op, final Path fspath,
       final Param<?,?>... parameters) throws IOException {
     final HttpURLConnection conn = httpConnect(op, fspath, parameters);
     validateResponse(op, conn);
@@ -301,4 +342,30 @@ public class WebHdfsFileSystem extends H
     final Map<String, Object> json = run(op, f, new RecursiveParam(recursive));
     return (Boolean)json.get(op.toString());
   }
+
+  @Override
+  public FSDataInputStream open(final Path f, final int buffersize
+      ) throws IOException {
+    statistics.incrementReadOps(1);
+    final HttpOpParam.Op op = GetOpParam.Op.OPEN;
+    final URL url = toUrl(op, f, new BufferSizeParam(buffersize));
+    return new FSDataInputStream(new ByteRangeInputStream(url));
+  }
+
+  @Override
+  public FileStatus[] listStatus(final Path f) throws IOException {
+    statistics.incrementReadOps(1);
+
+    final HttpOpParam.Op op = GetOpParam.Op.LISTSTATUS;
+    final Object[] array = run(op, f);
+
+    //convert FileStatus
+    final FileStatus[] statuses = new FileStatus[array.length];
+    for(int i = 0; i < array.length; i++) {
+      @SuppressWarnings("unchecked")
+      final Map<String, Object> m = (Map<String, Object>)array[i];
+      statuses[i] = makeQualified(JsonUtil.toFileStatus(m), f);
+    }
+    return statuses;
+  }
 }
\ No newline at end of file

Modified: hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/web/resources/GetOpParam.java?rev=1170087&r1=1170086&r2=1170087&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/web/resources/GetOpParam.java (original)
+++ hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/web/resources/GetOpParam.java Tue Sep 13 08:37:38 2011
@@ -26,7 +26,11 @@ public class GetOpParam extends HttpOpPa
 
   /** Get operations. */
   public static enum Op implements HttpOpParam.Op {
+    OPEN(HttpURLConnection.HTTP_OK),
+
     GETFILESTATUS(HttpURLConnection.HTTP_OK),
+    LISTSTATUS(HttpURLConnection.HTTP_OK),
+
     NULL(HttpURLConnection.HTTP_NOT_IMPLEMENTED);
 
     final int expectedHttpResponseCode;

Added: hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/web/resources/LengthParam.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/web/resources/LengthParam.java?rev=1170087&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/web/resources/LengthParam.java (added)
+++ hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/web/resources/LengthParam.java Tue Sep 13 08:37:38 2011
@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.web.resources;
+
+/** Length parameter. */
+public class LengthParam extends LongParam {
+  /** Parameter name. */
+  public static final String NAME = "length";
+  /** Default parameter value. */
+  public static final String DEFAULT = NULL;
+
+  private static final Domain DOMAIN = new Domain(NAME);
+
+  /**
+   * Constructor.
+   * @param value the parameter value.
+   */
+  public LengthParam(final Long value) {
+    super(DOMAIN, value);
+  }
+
+  /**
+   * Constructor.
+   * @param str a string representation of the parameter value.
+   */
+  public LengthParam(final String str) {
+    this(DOMAIN.parse(str));
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+}
\ No newline at end of file

Added: hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/web/resources/OffsetParam.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/web/resources/OffsetParam.java?rev=1170087&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/web/resources/OffsetParam.java (added)
+++ hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/web/resources/OffsetParam.java Tue Sep 13 08:37:38 2011
@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.web.resources;
+
+/** Offset parameter. */
+public class OffsetParam extends LongParam {
+  /** Parameter name. */
+  public static final String NAME = "offset";
+  /** Default parameter value. */
+  public static final String DEFAULT = "0";
+
+  private static final Domain DOMAIN = new Domain(NAME);
+
+  /**
+   * Constructor.
+   * @param value the parameter value.
+   */
+  public OffsetParam(final Long value) {
+    super(DOMAIN, value);
+  }
+
+  /**
+   * Constructor.
+   * @param str a string representation of the parameter value.
+   */
+  public OffsetParam(final String str) {
+    this(DOMAIN.parse(str));
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+}
\ No newline at end of file

Modified: hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/web/resources/UriFsPathParam.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/web/resources/UriFsPathParam.java?rev=1170087&r1=1170086&r2=1170087&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/web/resources/UriFsPathParam.java (original)
+++ hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/web/resources/UriFsPathParam.java Tue Sep 13 08:37:38 2011
@@ -39,7 +39,7 @@ public class UriFsPathParam extends Stri
 
   /** @return the absolute path. */
   public final String getAbsolutePath() {
-    final String path = getValue();
+    final String path = getValue(); //The first / has been stripped out.
     return path == null? null: "/" + path;
   }
 }

Modified: hadoop/common/branches/branch-0.20-security/src/test/org/apache/hadoop/hdfs/MiniDFSCluster.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security/src/test/org/apache/hadoop/hdfs/MiniDFSCluster.java?rev=1170087&r1=1170086&r2=1170087&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security/src/test/org/apache/hadoop/hdfs/MiniDFSCluster.java (original)
+++ hadoop/common/branches/branch-0.20-security/src/test/org/apache/hadoop/hdfs/MiniDFSCluster.java Tue Sep 13 08:37:38 2011
@@ -42,7 +42,6 @@ import org.apache.hadoop.hdfs.server.dat
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
-import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.net.DNSToSwitchMapping;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.StaticMapping;
@@ -807,18 +806,6 @@ public class MiniDFSCluster {
   }
 
   /**
-   * @return a {@link WebHdfsFileSystem} object.
-   */
-  public WebHdfsFileSystem getWebHdfsFileSystem() throws IOException {
-    final String str = WebHdfsFileSystem.SCHEME  + "://" + conf.get("dfs.http.address");
-    try {
-      return (WebHdfsFileSystem)FileSystem.get(new URI(str), conf); 
-    } catch (URISyntaxException e) {
-      throw new IOException(e);
-    }
-  }
-
-  /**
    *  @return a {@link HftpFileSystem} object as specified user. 
    */
   public HftpFileSystem getHftpFileSystemAs(final String username,

Modified: hadoop/common/branches/branch-0.20-security/src/test/org/apache/hadoop/hdfs/web/TestFSMainOperationsWebHdfs.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security/src/test/org/apache/hadoop/hdfs/web/TestFSMainOperationsWebHdfs.java?rev=1170087&r1=1170086&r2=1170087&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security/src/test/org/apache/hadoop/hdfs/web/TestFSMainOperationsWebHdfs.java (original)
+++ hadoop/common/branches/branch-0.20-security/src/test/org/apache/hadoop/hdfs/web/TestFSMainOperationsWebHdfs.java Tue Sep 13 08:37:38 2011
@@ -19,69 +19,61 @@ package org.apache.hadoop.hdfs.web;
 
 
 import static org.apache.hadoop.fs.FileSystemTestHelper.exists;
-import static org.apache.hadoop.fs.FileSystemTestHelper.getDefaultBlockSize;
 import static org.apache.hadoop.fs.FileSystemTestHelper.getTestRootPath;
 
 import java.io.IOException;
+import java.net.URI;
 
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSMainOperationsBaseTest;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.datanode.web.resources.DatanodeWebHdfsMethods;
 import org.apache.hadoop.hdfs.web.resources.ExceptionHandler;
 import org.apache.log4j.Level;
+import org.junit.AfterClass;
 import org.junit.Assert;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 public class TestFSMainOperationsWebHdfs extends FSMainOperationsBaseTest {
   {
     ((Log4JLogger)ExceptionHandler.LOG).getLogger().setLevel(Level.ALL);
+    ((Log4JLogger)DatanodeWebHdfsMethods.LOG).getLogger().setLevel(Level.ALL);
   }
 
-  private static final MiniDFSCluster cluster;
-  private static final Path defaultWorkingDirectory;
+  private static MiniDFSCluster cluster = null;
+  private static Path defaultWorkingDirectory;
 
-  static {
+  @BeforeClass
+  public static void setupCluster() {
     Configuration conf = new Configuration();
     try {
       cluster = new MiniDFSCluster(conf, 2, true, null);
-      fSys = cluster.getWebHdfsFileSystem();
+      cluster.waitActive();
+
+      final String uri = WebHdfsFileSystem.SCHEME  + "://"
+          + conf.get("dfs.http.address");
+      fSys = FileSystem.get(new URI(uri), conf); 
       defaultWorkingDirectory = fSys.getWorkingDirectory();
-    } catch (IOException e) {
+    } catch (Exception e) {
       throw new RuntimeException(e);
     }
   }
 
-  @Override
-  protected Path getDefaultWorkingDirectory() {
-    return defaultWorkingDirectory;
+  @AfterClass
+  public static void shutdownCluster() {
+    if (cluster != null) {
+      cluster.shutdown();
+      cluster = null;
+    }
   }
 
-  /** Override the following method without using position read. */
   @Override
-  protected void writeReadAndDelete(int len) throws IOException {
-    Path path = getTestRootPath(fSys, "test/hadoop/file");
-    fSys.mkdirs(path.getParent());
-
-    FSDataOutputStream out = 
-      fSys.create(path, false, 4096, (short) 1, getDefaultBlockSize() );
-    out.write(data, 0, len);
-    out.close();
-
-    Assert.assertTrue("Exists", exists(fSys, path));
-    Assert.assertEquals("Length", len, fSys.getFileStatus(path).getLen());
-
-    FSDataInputStream in = fSys.open(path);
-    for (int i = 0; i < len; i++) {
-      final int b  = in.read();
-      Assert.assertEquals("Position " + i, data[i], b);
-    }
-    in.close();
-    Assert.assertTrue("Deleted", fSys.delete(path, false));
-    Assert.assertFalse("No longer exists", exists(fSys, path));
+  protected Path getDefaultWorkingDirectory() {
+    return defaultWorkingDirectory;
   }
 
   //copied from trunk.

Modified: hadoop/common/branches/branch-0.20-security/src/test/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security/src/test/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java?rev=1170087&r1=1170086&r2=1170087&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security/src/test/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java (original)
+++ hadoop/common/branches/branch-0.20-security/src/test/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java Tue Sep 13 08:37:38 2011
@@ -18,24 +18,24 @@
 
 package org.apache.hadoop.hdfs.web;
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.net.URI;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystemContractBaseTest;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.security.UserGroupInformation;
 
 public class TestWebHdfsFileSystemContract extends FileSystemContractBaseTest {
+  private static final Configuration conf = new Configuration();
   private static final MiniDFSCluster cluster;
   private String defaultWorkingDirectory;
 
   static {
-    Configuration conf = new Configuration();
     try {
       cluster = new MiniDFSCluster(conf, 2, true, null);
+      cluster.waitActive();
     } catch (IOException e) {
       throw new RuntimeException(e);
     }
@@ -43,9 +43,10 @@ public class TestWebHdfsFileSystemContra
 
   @Override
   protected void setUp() throws Exception {
-    fs = cluster.getWebHdfsFileSystem();
-    defaultWorkingDirectory = "/user/"
-        + UserGroupInformation.getCurrentUser().getShortUserName();
+    final String uri = WebHdfsFileSystem.SCHEME  + "://"
+        + conf.get("dfs.http.address");
+    fs = FileSystem.get(new URI(uri), conf); 
+    defaultWorkingDirectory = fs.getWorkingDirectory().toUri().getPath();
   }
 
   @Override
@@ -53,34 +54,21 @@ public class TestWebHdfsFileSystemContra
     return defaultWorkingDirectory;
   }
 
-  /** Override the following method without using position read. */
-  @Override
-  protected void writeReadAndDelete(int len) throws IOException {
-    Path path = path("/test/hadoop/file");
-    
-    fs.mkdirs(path.getParent());
-
-    FSDataOutputStream out = fs.create(path, false,
-        fs.getConf().getInt("io.file.buffer.size", 4096), 
-        (short) 1, getBlockSize());
-    out.write(data, 0, len);
-    out.close();
-
-    assertTrue("Exists", fs.exists(path));
-    assertEquals("Length", len, fs.getFileStatus(path).getLen());
-
-    FSDataInputStream in = fs.open(path);
-    for (int i = 0; i < len; i++) {
-      final int b = in.read();
-      assertEquals("Position " + i, data[i], b);
-    }
-    in.close();
-    
-    assertTrue("Deleted", fs.delete(path, false));
-    assertFalse("No longer exists", fs.exists(path));
-  }
 
-  //The following test failed for HftpFileSystem,
-  //Disable it for WebHdfsFileSystem
+  //In trunk, testListStatusReturnsNullForNonExistentFile was replaced by
+  //testListStatusThrowsExceptionForNonExistentFile.
+  //
+  //For WebHdfsFileSystem,
+  //disable testListStatusReturnsNullForNonExistentFile
+  //and add testListStatusThrowsExceptionForNonExistentFile below.
   public void testListStatusReturnsNullForNonExistentFile() {}
+
+  public void testListStatusThrowsExceptionForNonExistentFile() throws Exception {
+    try {
+      fs.listStatus(path("/test/hadoop/file"));
+      fail("Should throw FileNotFoundException");
+    } catch (FileNotFoundException fnfe) {
+      // expected
+    }
+  }
 }