Browse Source

HDFS-2284. Add a new FileSystem, webhdfs://, for supporting write Http access to HDFS.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1167662 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 13 years ago
parent
commit
61d0b7530c
44 changed files with 2962 additions and 8 deletions
  1. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  2. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  3. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
  4. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java
  5. 53 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  6. 169 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/resources/DatanodeWebHdfsMethods.java
  7. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
  8. 9 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java
  9. 315 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
  10. 133 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
  11. 345 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
  12. 49 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/AccessTimeParam.java
  13. 49 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/BlockSizeParam.java
  14. 51 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/BooleanParam.java
  15. 49 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/BufferSizeParam.java
  16. 74 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DeleteOpParam.java
  17. 43 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DstPathParam.java
  18. 46 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/EnumParam.java
  19. 86 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/EnumSetParam.java
  20. 59 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ExceptionHandler.java
  21. 73 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
  22. 41 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/GroupParam.java
  23. 52 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/HttpOpParam.java
  24. 60 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/IntegerParam.java
  25. 60 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/LongParam.java
  26. 49 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ModificationTimeParam.java
  27. 49 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/OverwriteParam.java
  28. 41 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/OwnerParam.java
  29. 104 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/Param.java
  30. 57 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/PermissionParam.java
  31. 74 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/PostOpParam.java
  32. 84 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/PutOpParam.java
  33. 49 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/RecursiveParam.java
  34. 52 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/RenameOptionSetParam.java
  35. 49 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ReplicationParam.java
  36. 60 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ShortParam.java
  37. 54 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/StringParam.java
  38. 45 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/UriFsPathParam.java
  39. 41 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/UserParam.java
  40. 73 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/UserProvider.java
  41. 15 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
  42. 96 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestFSMainOperationsWebHdfs.java
  43. 55 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java
  44. 86 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java

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

@@ -5,6 +5,9 @@ Trunk (unreleased changes)
     HDFS-395.  DFS Scalability: Incremental block reports. (Tomasz Nykiel
                via hairong)
 
+    HDFS-2284. Add a new FileSystem, webhdfs://, for supporting write Http
+    access to HDFS.  (szetszwo)
+
   IMPROVEMENTS
 
     HADOOP-7524 Change RPC to allow multiple protocols including multuple versions of the same protocol (sanjay Radia)

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -802,7 +802,7 @@ public class DFSClient implements java.io.Closeable {
    * 
    * @see ClientProtocol#append(String, String) 
    */
-  DFSOutputStream append(String src, int buffersize, Progressable progress) 
+  public DFSOutputStream append(String src, int buffersize, Progressable progress) 
       throws IOException {
     checkOpen();
     HdfsFileStatus stat = getFileInfo(src);

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java

@@ -98,7 +98,7 @@ import org.apache.hadoop.util.PureJavaCrc32;
  * datanode from the original pipeline. The DataStreamer now
  * starts sending packets from the dataQueue.
 ****************************************************************/
-class DFSOutputStream extends FSOutputSummer implements Syncable {
+public class DFSOutputStream extends FSOutputSummer implements Syncable {
   private final DFSClient dfsClient;
   private static final int MAX_PACKETS = 80; // each packet 64K, total 5MB
   private Socket s;
@@ -1707,7 +1707,7 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
   /**
    * Returns the size of a file as it was when this stream was opened
    */
-  long getInitialLen() {
+  public long getInitialLen() {
     return initialFileSize;
   }
 

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java

@@ -250,7 +250,7 @@ public class HftpFileSystem extends FileSystem {
    * @return namenode URL referring to the given path
    * @throws IOException on error constructing the URL
    */
-  URL getNamenodeURL(String path, String query) throws IOException {
+  protected URL getNamenodeURL(String path, String query) throws IOException {
     final URL url = new URL("http", nnAddr.getHostName(),
           nnAddr.getPort(), path + '?' + query);
     if (LOG.isTraceEnabled()) {
@@ -317,6 +317,7 @@ public class HftpFileSystem extends FileSystem {
 
   @Override
   public FSDataInputStream open(Path f, int buffersize) throws IOException {
+    f = f.makeQualified(getUri(), getWorkingDirectory());
     String path = "/data" + ServletUtil.encodePath(f.toUri().getPath());
     String query = addDelegationTokenParam("ugi=" + getEncodedUgiParameter());
     URL u = getNamenodeURL(path, query);    

+ 53 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -18,7 +18,50 @@
 package org.apache.hadoop.hdfs.server.datanode;
 
 
-import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ADMIN;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_HTTPS_NEED_AUTH_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_HTTPS_NEED_AUTH_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_ADDRESS_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_PERMISSION_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_PERMISSION_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DNS_INTERFACE_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DNS_INTERFACE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DNS_NAMESERVER_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DNS_NAMESERVER_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HANDLER_COUNT_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HANDLER_COUNT_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HOST_NAME_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_KEYTAB_FILE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_PLUGINS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SIMULATEDDATASTORAGE_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SIMULATEDDATASTORAGE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_STARTUP_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_STORAGEID_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SYNCONCLOSE_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SYNCONCLOSE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_TRANSFERTO_ALLOWED_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_TRANSFERTO_ALLOWED_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_FEDERATION_NAMESERVICES;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY;
 import static org.apache.hadoop.hdfs.server.common.Util.now;
 
 import java.io.BufferedOutputStream;
@@ -93,9 +136,11 @@ import org.apache.hadoop.hdfs.server.common.Util;
 import org.apache.hadoop.hdfs.server.datanode.FSDataset.VolumeInfo;
 import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter.SecureResources;
 import org.apache.hadoop.hdfs.server.datanode.metrics.DataNodeMetrics;
+import org.apache.hadoop.hdfs.server.datanode.web.resources.DatanodeWebHdfsMethods;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.FileChecksumServlets;
 import org.apache.hadoop.hdfs.server.namenode.StreamFile;
+import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
@@ -109,7 +154,8 @@ import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
 import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
-import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand;
+import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
+import org.apache.hadoop.hdfs.web.resources.Param;
 import org.apache.hadoop.http.HttpServer;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.ProtocolSignature;
@@ -502,6 +548,11 @@ public class DataNode extends Configured
     this.infoServer.setAttribute(JspHelper.CURRENT_CONF, conf);
     this.infoServer.addServlet(null, "/blockScannerReport", 
                                DataBlockScanner.Servlet.class);
+
+    infoServer.addJerseyResourcePackage(
+        DatanodeWebHdfsMethods.class.getPackage().getName()
+        + ";" + Param.class.getPackage().getName(),
+        "/" + WebHdfsFileSystem.PATH_PREFIX + "/*");
     this.infoServer.start();
   }
   

+ 169 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/resources/DatanodeWebHdfsMethods.java

@@ -0,0 +1,169 @@
+/**
+ * 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.server.datanode.web.resources;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.EnumSet;
+
+import javax.servlet.ServletContext;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DefaultValue;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.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.OverwriteParam;
+import org.apache.hadoop.hdfs.web.resources.Param;
+import org.apache.hadoop.hdfs.web.resources.PermissionParam;
+import org.apache.hadoop.hdfs.web.resources.PostOpParam;
+import org.apache.hadoop.hdfs.web.resources.PutOpParam;
+import org.apache.hadoop.hdfs.web.resources.ReplicationParam;
+import org.apache.hadoop.hdfs.web.resources.UriFsPathParam;
+import org.apache.hadoop.io.IOUtils;
+
+/** Web-hdfs DataNode implementation. */
+@Path("")
+public class DatanodeWebHdfsMethods {
+  private static final Log LOG = LogFactory.getLog(DatanodeWebHdfsMethods.class);
+
+  private @Context ServletContext context;
+
+  /** Handle HTTP PUT request. */
+  @PUT
+  @Path("{" + UriFsPathParam.NAME + ":.*}")
+  @Consumes({"*/*"})
+  @Produces({MediaType.APPLICATION_JSON})
+  public Response put(
+      final InputStream in,
+      @PathParam(UriFsPathParam.NAME) final UriFsPathParam path,
+      @QueryParam(PutOpParam.NAME) @DefaultValue(PutOpParam.DEFAULT)
+          final PutOpParam op,
+      @QueryParam(PermissionParam.NAME) @DefaultValue(PermissionParam.DEFAULT)
+          final PermissionParam permission,
+      @QueryParam(OverwriteParam.NAME) @DefaultValue(OverwriteParam.DEFAULT)
+          final OverwriteParam overwrite,
+      @QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT)
+          final BufferSizeParam bufferSize,
+      @QueryParam(ReplicationParam.NAME) @DefaultValue(ReplicationParam.DEFAULT)
+          final ReplicationParam replication,
+      @QueryParam(BlockSizeParam.NAME) @DefaultValue(BlockSizeParam.DEFAULT)
+          final BlockSizeParam blockSize
+      ) throws IOException, URISyntaxException {
+
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(op + ": " + path
+            + Param.toSortedString(", ", permission, overwrite, bufferSize,
+                replication, blockSize));
+    }
+
+    final String fullpath = path.getAbsolutePath();
+    final DataNode datanode = (DataNode)context.getAttribute("datanode");
+
+    switch(op.getValue()) {
+    case CREATE:
+    {
+      final Configuration conf = new Configuration(datanode.getConf());
+      final InetSocketAddress nnRpcAddr = NameNode.getAddress(conf);
+      final DFSClient dfsclient = new DFSClient(nnRpcAddr, conf);
+      final FSDataOutputStream out = new FSDataOutputStream(dfsclient.create(
+          fullpath, permission.getFsPermission(), 
+          overwrite.getValue() ? EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE)
+              : EnumSet.of(CreateFlag.CREATE),
+          replication.getValue(), blockSize.getValue(), null,
+          bufferSize.getValue()), null);
+      try {
+        IOUtils.copyBytes(in, out, bufferSize.getValue());
+      } finally {
+        out.close();
+      }
+      final InetSocketAddress nnHttpAddr = NameNode.getHttpAddress(conf);
+      final URI uri = new URI(WebHdfsFileSystem.SCHEME, null,
+          nnHttpAddr.getHostName(), nnHttpAddr.getPort(), fullpath, null, null);
+      return Response.created(uri).type(MediaType.APPLICATION_JSON).build();
+    }
+    default:
+      throw new UnsupportedOperationException(op + " is not supported");
+    }
+  }
+
+  /** Handle HTTP POST request. */
+  @POST
+  @Path("{" + UriFsPathParam.NAME + ":.*}")
+  @Consumes({"*/*"})
+  @Produces({MediaType.APPLICATION_JSON})
+  public Response post(
+      final InputStream in,
+      @PathParam(UriFsPathParam.NAME) final UriFsPathParam path,
+      @QueryParam(PostOpParam.NAME) @DefaultValue(PostOpParam.DEFAULT)
+          final PostOpParam op,
+      @QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT)
+          final BufferSizeParam bufferSize
+      ) throws IOException, URISyntaxException {
+
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(op + ": " + path
+            + Param.toSortedString(", ", bufferSize));
+    }
+
+    final String fullpath = path.getAbsolutePath();
+    final DataNode datanode = (DataNode)context.getAttribute("datanode");
+
+    switch(op.getValue()) {
+    case APPEND:
+    {
+      final Configuration conf = new Configuration(datanode.getConf());
+      final InetSocketAddress nnRpcAddr = NameNode.getAddress(conf);
+      final DFSClient dfsclient = new DFSClient(nnRpcAddr, conf);
+      final DFSOutputStream dfsout = dfsclient.append(fullpath,
+          bufferSize.getValue(), null);
+      final FSDataOutputStream out = new FSDataOutputStream(dfsout, null,
+          dfsout.getInitialLen());
+      try {
+        IOUtils.copyBytes(in, out, bufferSize.getValue());
+      } finally {
+        out.close();
+      }
+      return Response.ok().type(MediaType.APPLICATION_JSON).build();
+    }
+    default:
+      throw new UnsupportedOperationException(op + " is not supported");
+    }
+  }
+}

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java

@@ -279,6 +279,11 @@ public class NameNode {
   }
 
   protected InetSocketAddress getHttpServerAddress(Configuration conf) {
+    return getHttpAddress(conf);
+  }
+
+  /** @return the NameNode HTTP address set in the conf. */
+  public static InetSocketAddress getHttpAddress(Configuration conf) {
     return  NetUtils.createSocketAddr(
         conf.get(DFS_NAMENODE_HTTP_ADDRESS_KEY, DFS_NAMENODE_HTTP_ADDRESS_DEFAULT));
   }

+ 9 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java

@@ -24,18 +24,20 @@ import java.security.PrivilegedExceptionAction;
 import javax.servlet.ServletContext;
 
 import org.apache.commons.logging.Log;
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
+import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
+import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
+import org.apache.hadoop.hdfs.web.resources.Param;
 import org.apache.hadoop.http.HttpServer;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
 
-import org.apache.hadoop.classification.InterfaceAudience;
-
 /**
  * Encapsulates the HTTP server started by the NameNode. 
  */
@@ -179,6 +181,11 @@ public class NameNodeHttpServer {
         FileChecksumServlets.RedirectServlet.class, false);
     httpServer.addInternalServlet("contentSummary", "/contentSummary/*",
         ContentSummaryServlet.class, false);
+
+    httpServer.addJerseyResourcePackage(
+        NamenodeWebHdfsMethods.class.getPackage().getName()
+        + ";" + Param.class.getPackage().getName(),
+        "/" + WebHdfsFileSystem.PATH_PREFIX + "/*");
   }
 
   public static FSImage getFsImageFromContext(ServletContext context) {

+ 315 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java

@@ -0,0 +1,315 @@
+/**
+ * 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.server.namenode.web.resources;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.EnumSet;
+
+import javax.servlet.ServletContext;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DELETE;
+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;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.common.JspHelper;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
+import org.apache.hadoop.hdfs.web.JsonUtil;
+import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
+import org.apache.hadoop.hdfs.web.resources.AccessTimeParam;
+import org.apache.hadoop.hdfs.web.resources.BlockSizeParam;
+import org.apache.hadoop.hdfs.web.resources.BufferSizeParam;
+import org.apache.hadoop.hdfs.web.resources.DeleteOpParam;
+import org.apache.hadoop.hdfs.web.resources.DstPathParam;
+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.ModificationTimeParam;
+import org.apache.hadoop.hdfs.web.resources.OverwriteParam;
+import org.apache.hadoop.hdfs.web.resources.OwnerParam;
+import org.apache.hadoop.hdfs.web.resources.Param;
+import org.apache.hadoop.hdfs.web.resources.PermissionParam;
+import org.apache.hadoop.hdfs.web.resources.PostOpParam;
+import org.apache.hadoop.hdfs.web.resources.PutOpParam;
+import org.apache.hadoop.hdfs.web.resources.RecursiveParam;
+import org.apache.hadoop.hdfs.web.resources.RenameOptionSetParam;
+import org.apache.hadoop.hdfs.web.resources.ReplicationParam;
+import org.apache.hadoop.hdfs.web.resources.UriFsPathParam;
+import org.apache.hadoop.net.NodeBase;
+
+/** Web-hdfs NameNode implementation. */
+@Path("")
+public class NamenodeWebHdfsMethods {
+  private static final Log LOG = LogFactory.getLog(NamenodeWebHdfsMethods.class);
+
+  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 HdfsFileStatus status = namenode.getRpcServer().getFileInfo(path);
+      final long len = status.getLen();
+      if (len > 0) {
+        final LocatedBlocks locations = namenode.getRpcServer().getBlockLocations(path, len-1, 1);
+        final int count = locations.locatedBlockCount();
+        if (count > 0) {
+          return JspHelper.bestNode(locations.get(count - 1));
+        }
+      }
+    } 
+
+    return (DatanodeDescriptor)namenode.getNamesystem().getBlockManager(
+        ).getDatanodeManager().getNetworkTopology().chooseRandom(
+        NodeBase.ROOT);
+  }
+
+  private static URI redirectURI(final NameNode namenode,
+      final String path, final HttpOpParam.Op op,
+      final Param<?, ?>... parameters) throws URISyntaxException, IOException {
+    final DatanodeInfo dn = chooseDatanode(namenode, path, op);
+    final String query = op.toQueryString() + Param.toSortedString("&", parameters);
+    final String uripath = "/" + WebHdfsFileSystem.PATH_PREFIX + path;
+
+    final URI uri = new URI("http", null, dn.getHostName(), dn.getInfoPort(),
+        uripath, query, null);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("redirectURI=" + uri);
+    }
+    return uri;
+  }
+
+  /** Handle HTTP PUT request. */
+  @PUT
+  @Path("{" + UriFsPathParam.NAME + ":.*}")
+  @Consumes({"*/*"})
+  @Produces({MediaType.APPLICATION_JSON})
+  public Response put(
+      final InputStream in,
+      @PathParam(UriFsPathParam.NAME) final UriFsPathParam path,
+      @QueryParam(PutOpParam.NAME) @DefaultValue(PutOpParam.DEFAULT)
+          final PutOpParam op,
+      @QueryParam(DstPathParam.NAME) @DefaultValue(DstPathParam.DEFAULT)
+          final DstPathParam dstPath,
+      @QueryParam(OwnerParam.NAME) @DefaultValue(OwnerParam.DEFAULT)
+          final OwnerParam owner,
+      @QueryParam(GroupParam.NAME) @DefaultValue(GroupParam.DEFAULT)
+          final GroupParam group,
+      @QueryParam(PermissionParam.NAME) @DefaultValue(PermissionParam.DEFAULT)
+          final PermissionParam permission,
+      @QueryParam(OverwriteParam.NAME) @DefaultValue(OverwriteParam.DEFAULT)
+          final OverwriteParam overwrite,
+      @QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT)
+          final BufferSizeParam bufferSize,
+      @QueryParam(ReplicationParam.NAME) @DefaultValue(ReplicationParam.DEFAULT)
+          final ReplicationParam replication,
+      @QueryParam(BlockSizeParam.NAME) @DefaultValue(BlockSizeParam.DEFAULT)
+          final BlockSizeParam blockSize,
+      @QueryParam(ModificationTimeParam.NAME) @DefaultValue(ModificationTimeParam.DEFAULT)
+          final ModificationTimeParam modificationTime,
+      @QueryParam(AccessTimeParam.NAME) @DefaultValue(AccessTimeParam.DEFAULT)
+          final AccessTimeParam accessTime,
+      @QueryParam(RenameOptionSetParam.NAME) @DefaultValue(RenameOptionSetParam.DEFAULT)
+          final RenameOptionSetParam renameOptions
+      ) throws IOException, URISyntaxException {
+
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(op + ": " + path
+            + Param.toSortedString(", ", dstPath, owner, group, permission,
+                overwrite, bufferSize, replication, blockSize));
+    }
+
+    final String fullpath = path.getAbsolutePath();
+    final NameNode namenode = (NameNode)context.getAttribute("name.node");
+    final NamenodeProtocols np = namenode.getRpcServer();
+
+    switch(op.getValue()) {
+    case CREATE:
+    {
+      final URI uri = redirectURI(namenode, fullpath, op.getValue(),
+          permission, overwrite, bufferSize, replication, blockSize);
+      return Response.temporaryRedirect(uri).build();
+    } 
+    case MKDIRS:
+    {
+      final boolean b = np.mkdirs(fullpath, permission.getFsPermission(), true);
+      final String js = JsonUtil.toJsonString(PutOpParam.Op.MKDIRS, b);
+      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
+    }
+    case RENAME:
+    {
+      final EnumSet<Options.Rename> s = renameOptions.getValue();
+      if (s.isEmpty()) {
+        @SuppressWarnings("deprecation")
+        final boolean b = np.rename(fullpath, dstPath.getValue());
+        final String js = JsonUtil.toJsonString(PutOpParam.Op.RENAME, b);
+        return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
+      } else {
+        np.rename(fullpath, dstPath.getValue(),
+            s.toArray(new Options.Rename[s.size()]));
+        return Response.ok().type(MediaType.APPLICATION_JSON).build();
+      }
+    }
+    case SETREPLICATION:
+    {
+      final boolean b = np.setReplication(fullpath, replication.getValue());
+      final String js = JsonUtil.toJsonString(PutOpParam.Op.SETREPLICATION, b);
+      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
+    }
+    case SETOWNER:
+    {
+      np.setOwner(fullpath, owner.getValue(), group.getValue());
+      return Response.ok().type(MediaType.APPLICATION_JSON).build();
+    }
+    case SETPERMISSION:
+    {
+      np.setPermission(fullpath, permission.getFsPermission());
+      return Response.ok().type(MediaType.APPLICATION_JSON).build();
+    }
+    case SETTIMES:
+    {
+      np.setTimes(fullpath, modificationTime.getValue(), accessTime.getValue());
+      return Response.ok().type(MediaType.APPLICATION_JSON).build();
+    }
+    default:
+      throw new UnsupportedOperationException(op + " is not supported");
+    }
+  }
+
+  /** Handle HTTP POST request. */
+  @POST
+  @Path("{" + UriFsPathParam.NAME + ":.*}")
+  @Consumes({"*/*"})
+  @Produces({MediaType.APPLICATION_JSON})
+  public Response post(
+      final InputStream in,
+      @PathParam(UriFsPathParam.NAME) final UriFsPathParam path,
+      @QueryParam(PostOpParam.NAME) @DefaultValue(PostOpParam.DEFAULT)
+          final PostOpParam op,
+      @QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT)
+          final BufferSizeParam bufferSize
+      ) throws IOException, URISyntaxException {
+
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(op + ": " + path
+            + Param.toSortedString(", ", bufferSize));
+    }
+
+    final String fullpath = path.getAbsolutePath();
+    final NameNode namenode = (NameNode)context.getAttribute("name.node");
+
+    switch(op.getValue()) {
+    case APPEND:
+    {
+      final URI uri = redirectURI(namenode, fullpath, op.getValue(), bufferSize);
+      return Response.temporaryRedirect(uri).build();
+    }
+    default:
+      throw new UnsupportedOperationException(op + " is not supported");
+    }
+  }
+
+  private static final UriFsPathParam ROOT = new UriFsPathParam("");
+
+  /** Handle HTTP GET request for the root. */
+  @GET
+  @Path("/")
+  @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);
+  }
+
+  /** 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
+      ) throws IOException {
+
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(op + ", " + path
+          + Param.toSortedString(", "));
+    }
+
+    switch(op.getValue()) {
+    case GETFILESTATUS:
+      final NameNode namenode = (NameNode)context.getAttribute("name.node");
+      final String fullpath = path.getAbsolutePath();
+      final HdfsFileStatus status = namenode.getRpcServer().getFileInfo(fullpath);
+      final String js = JsonUtil.toJsonString(status);
+      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
+
+    default:
+      throw new UnsupportedOperationException(op + " is not supported");
+    }    
+  }
+
+  /** Handle HTTP DELETE request. */
+  @DELETE
+  @Path("{path:.*}")
+  @Produces(MediaType.APPLICATION_JSON)
+  public Response delete(
+      @PathParam(UriFsPathParam.NAME) final UriFsPathParam path,
+      @QueryParam(DeleteOpParam.NAME) @DefaultValue(DeleteOpParam.DEFAULT)
+          final DeleteOpParam op,
+      @QueryParam(RecursiveParam.NAME) @DefaultValue(RecursiveParam.DEFAULT)
+          final RecursiveParam recursive
+      ) throws IOException {
+
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(op + ", " + path
+        + Param.toSortedString(", ", recursive));
+    }
+
+    switch(op.getValue()) {
+    case DELETE:
+      final NameNode namenode = (NameNode)context.getAttribute("name.node");
+      final String fullpath = path.getAbsolutePath();
+      final boolean b = namenode.getRpcServer().delete(fullpath, recursive.getValue());
+      final String js = JsonUtil.toJsonString(DeleteOpParam.Op.DELETE, b);
+      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
+
+    default:
+      throw new UnsupportedOperationException(op + " is not supported");
+    }    
+  }
+}

+ 133 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java

@@ -0,0 +1,133 @@
+/**
+ * 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;
+
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.ipc.RemoteException;
+import org.mortbay.util.ajax.JSON;
+
+/** JSON Utilities */
+public class JsonUtil {
+  private static final ThreadLocal<Map<String, Object>> jsonMap
+      = new ThreadLocal<Map<String, Object>>() {
+    @Override
+    protected Map<String, Object> initialValue() {
+      return new TreeMap<String, Object>();
+    }
+
+    @Override
+    public Map<String, Object> get() {
+      final Map<String, Object> m = super.get();
+      m.clear();
+      return m;
+    }
+  };
+
+  /** Convert an exception object to a Json string. */
+  public static String toJsonString(final Exception e) {
+    final Map<String, Object> m = jsonMap.get();
+    m.put("className", e.getClass().getName());
+    m.put("message", e.getMessage());
+    return JSON.toString(m);
+  }
+
+  /** Convert a Json map to a RemoteException. */
+  public static RemoteException toRemoteException(final Map<String, Object> m) {
+    final String className = (String)m.get("className");
+    final String message = (String)m.get("message");
+    return new RemoteException(className, message);
+  }
+
+  /** Convert a key-value pair to a Json string. */
+  public static String toJsonString(final Object key, final Object value) {
+    final Map<String, Object> m = jsonMap.get();
+    m.put(key instanceof String ? (String) key : key.toString(), value);
+    return JSON.toString(m);
+  }
+
+  /** Convert a FsPermission object to a string. */
+  public static String toString(final FsPermission permission) {
+    return String.format("%o", permission.toShort());
+  }
+
+  /** Convert a string to a FsPermission object. */
+  public static FsPermission toFsPermission(final String s) {
+    return new FsPermission(Short.parseShort(s, 8));
+  }
+
+  /** Convert a HdfsFileStatus object to a Json string. */
+  public static String toJsonString(final HdfsFileStatus status) {
+    final Map<String, Object> m = jsonMap.get();
+    if (status == null) {
+      m.put("isNull", true);
+    } else {
+      m.put("isNull", false);
+      m.put("localName", status.getLocalName());
+      m.put("isDir", status.isDir());
+      m.put("isSymlink", status.isSymlink());
+      if (status.isSymlink()) {
+        m.put("symlink", status.getSymlink());
+      }
+
+      m.put("len", status.getLen());
+      m.put("owner", status.getOwner());
+      m.put("group", status.getGroup());
+      m.put("permission", toString(status.getPermission()));
+      m.put("accessTime", status.getAccessTime());
+      m.put("modificationTime", status.getModificationTime());
+      m.put("blockSize", status.getBlockSize());
+      m.put("replication", status.getReplication());
+    }
+    return JSON.toString(m);
+  }
+
+  @SuppressWarnings("unchecked")
+  static Map<String, Object> parse(String jsonString) {
+    return (Map<String, Object>) JSON.parse(jsonString);
+  }
+
+  /** Convert a Json string to a HdfsFileStatus object. */
+  public static HdfsFileStatus toFileStatus(final Map<String, Object> m) {
+    if ((Boolean)m.get("isNull")) {
+      return null;
+    }
+
+    final String localName = (String) m.get("localName");
+    final boolean isDir = (Boolean) m.get("isDir");
+    final boolean isSymlink = (Boolean) m.get("isSymlink");
+    final byte[] symlink = isSymlink?
+        DFSUtil.string2Bytes((String)m.get("symlink")): null;
+
+    final long len = (Long) m.get("len");
+    final String owner = (String) m.get("owner");
+    final String group = (String) m.get("group");
+    final FsPermission permission = toFsPermission((String) m.get("permission"));
+    final long aTime = (Long) m.get("accessTime");
+    final long mTime = (Long) m.get("modificationTime");
+    final long blockSize = (Long) m.get("blockSize");
+    final short replication = (short) (long) (Long) m.get("replication");
+    return new HdfsFileStatus(len, isDir, replication, blockSize, mTime, aTime,
+        permission, owner, group,
+        symlink, DFSUtil.string2Bytes(localName));
+  }
+}

+ 345 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java

@@ -0,0 +1,345 @@
+/**
+ * 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;
+
+import java.io.BufferedOutputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+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.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.fs.ParentNotDirectoryException;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.HftpFileSystem;
+import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
+import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
+import org.apache.hadoop.hdfs.web.resources.AccessTimeParam;
+import org.apache.hadoop.hdfs.web.resources.BlockSizeParam;
+import org.apache.hadoop.hdfs.web.resources.BufferSizeParam;
+import org.apache.hadoop.hdfs.web.resources.DeleteOpParam;
+import org.apache.hadoop.hdfs.web.resources.DstPathParam;
+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.ModificationTimeParam;
+import org.apache.hadoop.hdfs.web.resources.OverwriteParam;
+import org.apache.hadoop.hdfs.web.resources.OwnerParam;
+import org.apache.hadoop.hdfs.web.resources.Param;
+import org.apache.hadoop.hdfs.web.resources.PermissionParam;
+import org.apache.hadoop.hdfs.web.resources.PostOpParam;
+import org.apache.hadoop.hdfs.web.resources.PutOpParam;
+import org.apache.hadoop.hdfs.web.resources.RecursiveParam;
+import org.apache.hadoop.hdfs.web.resources.RenameOptionSetParam;
+import org.apache.hadoop.hdfs.web.resources.ReplicationParam;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Progressable;
+import org.mortbay.util.ajax.JSON;
+
+/** A FileSystem for HDFS over the web. */
+public class WebHdfsFileSystem extends HftpFileSystem {
+  /** File System URI: {SCHEME}://namenode:port/path/to/file */
+  public static final String SCHEME = "webhdfs";
+  /** Http URI: http://namenode:port/{PATH_PREFIX}/path/to/file */
+  public static final String PATH_PREFIX = SCHEME;
+
+  private UserGroupInformation ugi;
+  protected Path workingDir;
+
+  @Override
+  public void initialize(URI uri, Configuration conf) throws IOException {
+    super.initialize(uri, conf);
+    setConf(conf);
+
+    ugi = UserGroupInformation.getCurrentUser();
+    this.workingDir = getHomeDirectory();
+  }
+
+  @Override
+  public URI getUri() {
+    try {
+      return new URI(SCHEME, null, nnAddr.getHostName(), nnAddr.getPort(),
+          null, null, null);
+    } catch (URISyntaxException e) {
+      return null;
+    }
+  }
+
+  @Override
+  public Path getHomeDirectory() {
+    return makeQualified(new Path("/user/" + ugi.getShortUserName()));
+  }
+
+  @Override
+  public synchronized Path getWorkingDirectory() {
+    return workingDir;
+  }
+
+  @Override
+  public synchronized void setWorkingDirectory(final Path dir) {
+    String result = makeAbsolute(dir).toUri().getPath();
+    if (!DFSUtil.isValidName(result)) {
+      throw new IllegalArgumentException("Invalid DFS directory name " + 
+                                         result);
+    }
+    workingDir = makeAbsolute(dir);
+  }
+
+  private Path makeAbsolute(Path f) {
+    return f.isAbsolute()? f: new Path(workingDir, f);
+  }
+
+  @SuppressWarnings("unchecked")
+  private static Map<String, Object> 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));
+  }
+
+  private static void validateResponse(final HttpOpParam.Op op,
+      final HttpURLConnection conn) throws IOException {
+    final int code = conn.getResponseCode();
+    if (code != op.getExpectedHttpResponseCode()) {
+      final Map<String, Object> m;
+      try {
+        m = jsonParse(conn.getErrorStream());
+      } catch(IOException e) {
+        throw new IOException("Unexpected HTTP response: code = " + code + " != "
+            + op.getExpectedHttpResponseCode() + ", " + op.toQueryString()
+            + ", message=" + conn.getResponseMessage(), e);
+      }
+
+      final RemoteException re = JsonUtil.toRemoteException(m);
+      throw re.unwrapRemoteException(AccessControlException.class,
+          DSQuotaExceededException.class,
+          FileAlreadyExistsException.class,
+          FileNotFoundException.class,
+          ParentNotDirectoryException.class,
+          SafeModeException.class,
+          NSQuotaExceededException.class,
+          UnresolvedPathException.class);
+    }
+  }
+
+  @Override
+  protected HttpURLConnection openConnection(String path, String query)
+      throws IOException {
+    query = addDelegationTokenParam(query);
+    final URL url = getNamenodeURL(path, query);
+    return  (HttpURLConnection)url.openConnection();
+  }
+
+  private HttpURLConnection httpConnect(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);
+
+    //connect and get response
+    final HttpURLConnection conn = openConnection(uripath, query);
+    try {
+      conn.setRequestMethod(op.getType().toString());
+      conn.setDoOutput(op.getDoOutput());
+      if (op.getDoOutput()) {
+        conn.setRequestProperty("Expect", "100-Continue");
+        conn.setInstanceFollowRedirects(true);
+      }
+      conn.connect();
+      return conn;
+    } catch(IOException e) {
+      conn.disconnect();
+      throw e;
+    }
+  }
+
+  private Map<String, Object> run(final HttpOpParam.Op op, final Path fspath,
+      final Param<?,?>... parameters) throws IOException {
+    final HttpURLConnection conn = httpConnect(op, fspath, parameters);
+    validateResponse(op, conn);
+    try {
+      return jsonParse(conn.getInputStream());
+    } finally {
+      conn.disconnect();
+    }
+  }
+
+  private FsPermission applyUMask(FsPermission permission) {
+    if (permission == null) {
+      permission = FsPermission.getDefault();
+    }
+    return permission.applyUMask(FsPermission.getUMask(getConf()));
+  }
+
+  private HdfsFileStatus getHdfsFileStatus(Path f) throws IOException {
+    final HttpOpParam.Op op = GetOpParam.Op.GETFILESTATUS;
+    final Map<String, Object> json = run(op, f);
+    final HdfsFileStatus status = JsonUtil.toFileStatus(json);
+    if (status == null) {
+      throw new FileNotFoundException("File does not exist: " + f);
+    }
+    return status;
+  }
+
+  @Override
+  public FileStatus getFileStatus(Path f) throws IOException {
+    statistics.incrementReadOps(1);
+    return makeQualified(getHdfsFileStatus(f), f);
+  }
+
+  private FileStatus makeQualified(HdfsFileStatus f, Path parent) {
+    return new FileStatus(f.getLen(), f.isDir(), f.getReplication(),
+        f.getBlockSize(), f.getModificationTime(),
+        f.getAccessTime(),
+        f.getPermission(), f.getOwner(), f.getGroup(),
+        f.getFullPath(parent).makeQualified(getUri(), getWorkingDirectory()));
+  }
+
+  @Override
+  public boolean mkdirs(Path f, FsPermission permission) throws IOException {
+    statistics.incrementWriteOps(1);
+    final HttpOpParam.Op op = PutOpParam.Op.MKDIRS;
+    final Map<String, Object> json = run(op, f,
+        new PermissionParam(applyUMask(permission)));
+    return (Boolean)json.get(op.toString());
+  }
+
+  @Override
+  public boolean rename(final Path src, final Path dst) throws IOException {
+    statistics.incrementWriteOps(1);
+    final HttpOpParam.Op op = PutOpParam.Op.RENAME;
+    final Map<String, Object> json = run(op, src,
+        new DstPathParam(makeQualified(dst).toUri().getPath()));
+    return (Boolean)json.get(op.toString());
+  }
+
+  @SuppressWarnings("deprecation")
+  @Override
+  public void rename(final Path src, final Path dst,
+      final Options.Rename... options) throws IOException {
+    statistics.incrementWriteOps(1);
+    final HttpOpParam.Op op = PutOpParam.Op.RENAME;
+    run(op, src, new DstPathParam(makeQualified(dst).toUri().getPath()),
+        new RenameOptionSetParam(options));
+  }
+
+  @Override
+  public void setOwner(final Path p, final String owner, final String group
+      ) throws IOException {
+    if (owner == null && group == null) {
+      throw new IOException("owner == null && group == null");
+    }
+
+    statistics.incrementWriteOps(1);
+    final HttpOpParam.Op op = PutOpParam.Op.SETOWNER;
+    run(op, p, new OwnerParam(owner), new GroupParam(group));
+  }
+
+  @Override
+  public void setPermission(final Path p, final FsPermission permission
+      ) throws IOException {
+    statistics.incrementWriteOps(1);
+    final HttpOpParam.Op op = PutOpParam.Op.SETPERMISSION;
+    run(op, p, new PermissionParam(permission));
+  }
+
+  @Override
+  public boolean setReplication(final Path p, final short replication
+     ) throws IOException {
+    statistics.incrementWriteOps(1);
+    final HttpOpParam.Op op = PutOpParam.Op.SETREPLICATION;
+    final Map<String, Object> json = run(op, p,
+        new ReplicationParam(replication));
+    return (Boolean)json.get(op.toString());
+  }
+
+  @Override
+  public void setTimes(final Path p, final long mtime, final long atime
+      ) throws IOException {
+    statistics.incrementWriteOps(1);
+    final HttpOpParam.Op op = PutOpParam.Op.SETTIMES;
+    run(op, p, new ModificationTimeParam(mtime), new AccessTimeParam(atime));
+  }
+
+  private FSDataOutputStream write(final HttpOpParam.Op op,
+      final HttpURLConnection conn, final int bufferSize) throws IOException {
+    return new FSDataOutputStream(new BufferedOutputStream(
+        conn.getOutputStream(), bufferSize), statistics) {
+      @Override
+      public void close() throws IOException {
+        try {
+          super.close();
+        } finally {
+          validateResponse(op, conn);
+        }
+      }
+    };
+  }
+
+  @Override
+  public FSDataOutputStream create(final Path f, final FsPermission permission,
+      final boolean overwrite, final int bufferSize, final short replication,
+      final long blockSize, final Progressable progress) throws IOException {
+    statistics.incrementWriteOps(1);
+
+    final HttpOpParam.Op op = PutOpParam.Op.CREATE;
+    final HttpURLConnection conn = httpConnect(op, f, 
+        new PermissionParam(applyUMask(permission)),
+        new OverwriteParam(overwrite),
+        new BufferSizeParam(bufferSize),
+        new ReplicationParam(replication),
+        new BlockSizeParam(blockSize));
+    return write(op, conn, bufferSize);
+  }
+
+  @Override
+  public FSDataOutputStream append(final Path f, final int bufferSize,
+      final Progressable progress) throws IOException {
+    statistics.incrementWriteOps(1);
+
+    final HttpOpParam.Op op = PostOpParam.Op.APPEND;
+    final HttpURLConnection conn = httpConnect(op, f, 
+        new BufferSizeParam(bufferSize));
+    return write(op, conn, bufferSize);
+  }
+
+  @Override
+  public boolean delete(Path f, boolean recursive) throws IOException {
+    final HttpOpParam.Op op = DeleteOpParam.Op.DELETE;
+    final Map<String, Object> json = run(op, f, new RecursiveParam(recursive));
+    return (Boolean)json.get(op.toString());
+  }
+}

+ 49 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/AccessTimeParam.java

@@ -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;
+
+/** Access time parameter. */
+public class AccessTimeParam extends LongParam {
+  /** Parameter name. */
+  public static final String NAME = "accessTime";
+  /** Default parameter value. */
+  public static final String DEFAULT = "-1";
+
+  private static final Domain DOMAIN = new Domain(NAME);
+
+  /**
+   * Constructor.
+   * @param value the parameter value.
+   */
+  public AccessTimeParam(final Long value) {
+    super(DOMAIN, value);
+  }
+
+  /**
+   * Constructor.
+   * @param str a string representation of the parameter value.
+   */
+  public AccessTimeParam(final String str) {
+    this(DOMAIN.parse(str));
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+}

+ 49 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/BlockSizeParam.java

@@ -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;
+
+/** Block size parameter. */
+public class BlockSizeParam extends LongParam {
+  /** Parameter name. */
+  public static final String NAME = "blockSize";
+  /** Default parameter value. */
+  public static final String DEFAULT = NULL;
+
+  private static final Domain DOMAIN = new Domain(NAME);
+
+  /**
+   * Constructor.
+   * @param value the parameter value.
+   */
+  public BlockSizeParam(final Long value) {
+    super(DOMAIN, value);
+  }
+
+  /**
+   * Constructor.
+   * @param str a string representation of the parameter value.
+   */
+  public BlockSizeParam(final String str) {
+    this(DOMAIN.parse(str));
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+}

+ 51 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/BooleanParam.java

@@ -0,0 +1,51 @@
+/**
+ * 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;
+
+/** Boolean parameter. */
+abstract class BooleanParam extends Param<Boolean, BooleanParam.Domain> {
+  static final String TRUE = "true";
+  static final String FALSE = "false";
+
+  BooleanParam(final Domain domain, final Boolean value) {
+    super(domain, value);
+  }
+
+  /** The domain of the parameter. */
+  static final class Domain extends Param.Domain<Boolean> {
+    Domain(final String paramName) {
+      super(paramName);
+    }
+
+    @Override
+    public String getDomain() {
+      return "<" + NULL + " | boolean>";
+    }
+
+    @Override
+    Boolean parse(final String str) {
+      if (TRUE.equalsIgnoreCase(str)) {
+        return true;
+      } else if (FALSE.equalsIgnoreCase(str)) {
+        return false;
+      }
+      throw new IllegalArgumentException("Failed to parse \"" + str
+          + "\" to Boolean.");
+    }
+  }
+}

+ 49 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/BufferSizeParam.java

@@ -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;
+
+/** Buffer size parameter. */
+public class BufferSizeParam extends IntegerParam {
+  /** Parameter name. */
+  public static final String NAME = "bufferSize";
+  /** Default parameter value. */
+  public static final String DEFAULT = NULL;
+
+  private static final Domain DOMAIN = new Domain(NAME);
+
+  /**
+   * Constructor.
+   * @param value the parameter value.
+   */
+  public BufferSizeParam(final Integer value) {
+    super(DOMAIN, value);
+  }
+
+  /**
+   * Constructor.
+   * @param str a string representation of the parameter value.
+   */
+  public BufferSizeParam(final String str) {
+    this(DOMAIN.parse(str));
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+}

+ 74 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DeleteOpParam.java

@@ -0,0 +1,74 @@
+/**
+ * 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;
+
+import java.net.HttpURLConnection;
+
+/** Http DELETE operation parameter. */
+public class DeleteOpParam extends HttpOpParam<DeleteOpParam.Op> {
+  /** Parameter name. */
+  public static final String NAME = "deleteOp";
+
+  /** Delete operations. */
+  public static enum Op implements HttpOpParam.Op {
+    DELETE(HttpURLConnection.HTTP_OK),
+
+    NULL(HttpURLConnection.HTTP_NOT_IMPLEMENTED);
+
+    final int expectedHttpResponseCode;
+
+    Op(final int expectedHttpResponseCode) {
+      this.expectedHttpResponseCode = expectedHttpResponseCode;
+    }
+
+    @Override
+    public HttpOpParam.Type getType() {
+      return HttpOpParam.Type.DELETE;
+    }
+
+    @Override
+    public boolean getDoOutput() {
+      return false;
+    }
+
+    @Override
+    public int getExpectedHttpResponseCode() {
+      return expectedHttpResponseCode;
+    }
+
+    @Override
+    public String toQueryString() {
+      return NAME + "=" + this;
+    }
+  }
+
+  private static final Domain<Op> DOMAIN = new Domain<Op>(NAME, Op.class);
+
+  /**
+   * Constructor.
+   * @param str a string representation of the parameter value.
+   */
+  public DeleteOpParam(final String str) {
+    super(DOMAIN, DOMAIN.parse(str));
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+}

+ 43 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DstPathParam.java

@@ -0,0 +1,43 @@
+/**
+ * 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;
+
+import org.apache.hadoop.fs.Path;
+
+/** Destination path parameter. */
+public class DstPathParam extends StringParam {
+  /** Parameter name. */
+  public static final String NAME = "dstPath";
+  /** Default parameter value. */
+  public static final String DEFAULT = "";
+
+  private static final Domain DOMAIN = new Domain(NAME, null);
+
+  /**
+   * Constructor.
+   * @param str a string representation of the parameter value.
+   */
+  public DstPathParam(final String str) {
+    super(DOMAIN, str == null || str.equals(DEFAULT)? null: new Path(str).toUri().getPath());
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+}

+ 46 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/EnumParam.java

@@ -0,0 +1,46 @@
+/**
+ * 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;
+
+import java.util.Arrays;
+
+abstract class EnumParam<E extends Enum<E>> extends Param<E, EnumParam.Domain<E>> {
+  EnumParam(final Domain<E> domain, final E value) {
+    super(domain, value);
+  }
+
+  /** The domain of the parameter. */
+  static final class Domain<E extends Enum<E>> extends Param.Domain<E> {
+    private final Class<E> enumClass;
+
+    Domain(String name, final Class<E> enumClass) {
+      super(name);
+      this.enumClass = enumClass;
+    }
+
+    @Override
+    public final String getDomain() {
+      return Arrays.asList(enumClass.getEnumConstants()).toString();
+    }
+
+    @Override
+    final E parse(final String str) {
+      return Enum.valueOf(enumClass, str.toUpperCase());
+    }
+  }
+}

+ 86 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/EnumSetParam.java

@@ -0,0 +1,86 @@
+/**
+ * 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;
+
+import java.util.Arrays;
+import java.util.EnumSet;
+import java.util.Iterator;
+
+abstract class EnumSetParam<E extends Enum<E>> extends Param<EnumSet<E>, EnumSetParam.Domain<E>> {
+  /** Convert an EnumSet to a string of comma separated values. */
+  static <E extends Enum<E>> String toString(EnumSet<E> set) {
+    if (set == null || set.isEmpty()) {
+      return "";
+    } else {
+      final StringBuilder b = new StringBuilder();
+      final Iterator<E> i = set.iterator();
+      b.append(i.next());
+      for(; i.hasNext(); ) {
+        b.append(',').append(i.next());
+      }
+      return b.toString();
+    }
+  }
+
+  static <E extends Enum<E>> EnumSet<E> toEnumSet(final Class<E> clazz,
+      final E... values) {
+    final EnumSet<E> set = EnumSet.noneOf(clazz);
+    set.addAll(Arrays.asList(values));
+    return set;
+  }
+
+  EnumSetParam(final Domain<E> domain, final EnumSet<E> value) {
+    super(domain, value);
+  }
+
+  @Override
+  public String toString() {
+    return getName() + "=" + toString(value);
+  }
+
+  
+  /** The domain of the parameter. */
+  static final class Domain<E extends Enum<E>> extends Param.Domain<EnumSet<E>> {
+    private final Class<E> enumClass;
+
+    Domain(String name, final Class<E> enumClass) {
+      super(name);
+      this.enumClass = enumClass;
+    }
+
+    @Override
+    public final String getDomain() {
+      return Arrays.asList(enumClass.getEnumConstants()).toString();
+    }
+
+    /** The string contains a comma separated values. */
+    @Override
+    final EnumSet<E> parse(final String str) {
+      final EnumSet<E> set = EnumSet.noneOf(enumClass);
+      if (!str.isEmpty()) {
+        for(int i, j = 0; j >= 0; ) {
+          i = j;
+          j = str.indexOf(',', i+1);
+          final String sub = j >= 0? str.substring(i, j): str.substring(i);
+          set.add(Enum.valueOf(enumClass, sub.trim().toUpperCase()));
+        }
+      }
+      return set;
+    }
+  }
+}

+ 59 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ExceptionHandler.java

@@ -0,0 +1,59 @@
+/**
+ * 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;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.ext.ExceptionMapper;
+import javax.ws.rs.ext.Provider;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hdfs.web.JsonUtil;
+
+/** Handle exceptions. */
+@Provider
+public class ExceptionHandler implements ExceptionMapper<Exception> {
+  public static final Log LOG = LogFactory.getLog(ExceptionHandler.class);
+
+  @Override
+  public Response toResponse(final Exception e) {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("GOT EXCEPITION", e);
+    }
+
+    final Response.Status s;
+    if (e instanceof SecurityException) {
+      s = Response.Status.UNAUTHORIZED;
+    } else if (e instanceof FileNotFoundException) {
+      s = Response.Status.NOT_FOUND;
+    } else if (e instanceof IOException) {
+      s = Response.Status.FORBIDDEN;
+    } else if (e instanceof UnsupportedOperationException) {
+      s = Response.Status.BAD_REQUEST;
+    } else {
+      s = Response.Status.INTERNAL_SERVER_ERROR;
+    }
+ 
+    final String js = JsonUtil.toJsonString(e);
+    return Response.status(s).type(MediaType.APPLICATION_JSON).entity(js).build();
+  }
+}

+ 73 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java

@@ -0,0 +1,73 @@
+/**
+ * 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;
+
+import java.net.HttpURLConnection;
+
+/** Http GET operation parameter. */
+public class GetOpParam extends HttpOpParam<GetOpParam.Op> {
+  /** Parameter name. */
+  public static final String NAME = "getOp";
+
+  /** Get operations. */
+  public static enum Op implements HttpOpParam.Op {
+    GETFILESTATUS(HttpURLConnection.HTTP_OK),
+    NULL(HttpURLConnection.HTTP_NOT_IMPLEMENTED);
+
+    final int expectedHttpResponseCode;
+
+    Op(final int expectedHttpResponseCode) {
+      this.expectedHttpResponseCode = expectedHttpResponseCode;
+    }
+
+    @Override
+    public HttpOpParam.Type getType() {
+      return HttpOpParam.Type.GET;
+    }
+
+    @Override
+    public boolean getDoOutput() {
+      return false;
+    }
+
+    @Override
+    public int getExpectedHttpResponseCode() {
+      return expectedHttpResponseCode;
+    }
+
+    @Override
+    public String toQueryString() {
+      return NAME + "=" + this;
+    }
+  }
+
+  private static final Domain<Op> DOMAIN = new Domain<Op>(NAME, Op.class);
+
+  /**
+   * Constructor.
+   * @param str a string representation of the parameter value.
+   */
+  public GetOpParam(final String str) {
+    super(DOMAIN, DOMAIN.parse(str));
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+}

+ 41 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/GroupParam.java

@@ -0,0 +1,41 @@
+/**
+ * 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;
+
+/** Group parameter. */
+public class GroupParam extends StringParam {
+  /** Parameter name. */
+  public static final String NAME = "group";
+  /** Default parameter value. */
+  public static final String DEFAULT = "";
+
+  private static final Domain DOMAIN = new Domain(NAME, null);
+
+  /**
+   * Constructor.
+   * @param str a string representation of the parameter value.
+   */
+  public GroupParam(final String str) {
+    super(DOMAIN, str == null || str.equals(DEFAULT)? null: str);
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+}

+ 52 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/HttpOpParam.java

@@ -0,0 +1,52 @@
+/**
+ * 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;
+
+/** Http operation parameter. */
+public abstract class HttpOpParam<E extends Enum<E> & HttpOpParam.Op> extends EnumParam<E> {
+  /** Default parameter value. */
+  public static final String DEFAULT = NULL;
+
+  /** Http operation types */
+  public static enum Type {
+    GET, PUT, POST, DELETE;
+  }
+
+  /** Http operation interface. */
+  public static interface Op {
+    /** @return the Http operation type. */
+    public Type getType();
+
+    /** @return true if the operation has output. */
+    public boolean getDoOutput();
+
+    /** @return true if the operation has output. */
+    public int getExpectedHttpResponseCode();
+
+    /** @return a URI query string. */
+    public String toQueryString();
+  }
+
+  /**
+   * Constructor.
+   * @param str a string representation of the parameter value.
+   */
+  HttpOpParam(final Domain<E> domain, final E value) {
+    super(domain, value);
+  }
+}

+ 60 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/IntegerParam.java

@@ -0,0 +1,60 @@
+/**
+ * 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;
+
+/** Integer parameter. */
+abstract class IntegerParam extends Param<Integer, IntegerParam.Domain> {
+  IntegerParam(final Domain domain, final Integer value) {
+    super(domain, value);
+  }
+  
+  @Override
+  public String toString() {
+    return getName() + "=" + domain.toString(getValue());
+  }
+
+  /** The domain of the parameter. */
+  static final class Domain extends Param.Domain<Integer> {
+    /** The radix of the number. */
+    final int radix;
+
+    Domain(final String paramName) {
+      this(paramName, 10);
+    }
+
+    Domain(final String paramName, final int radix) {
+      super(paramName);
+      this.radix = radix;
+    }
+
+    @Override
+    public String getDomain() {
+      return "<" + NULL + " | int in radix " + radix + ">";
+    }
+
+    @Override
+    Integer parse(final String str) {
+      return NULL.equals(str)? null: Integer.parseInt(str, radix);
+    }
+
+    /** Convert an Integer to a String. */ 
+    String toString(final Integer n) {
+      return n == null? NULL: Integer.toString(n, radix);
+    }
+  }
+}

+ 60 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/LongParam.java

@@ -0,0 +1,60 @@
+/**
+ * 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;
+
+/** Long parameter. */
+abstract class LongParam extends Param<Long, LongParam.Domain> {
+  LongParam(final Domain domain, final Long value) {
+    super(domain, value);
+  }
+  
+  @Override
+  public String toString() {
+    return getName() + "=" + domain.toString(getValue());
+  }
+
+  /** The domain of the parameter. */
+  static final class Domain extends Param.Domain<Long> {
+    /** The radix of the number. */
+    final int radix;
+
+    Domain(final String paramName) {
+      this(paramName, 10);
+    }
+
+    Domain(final String paramName, final int radix) {
+      super(paramName);
+      this.radix = radix;
+    }
+
+    @Override
+    public String getDomain() {
+      return "<" + NULL + " | short in radix " + radix + ">";
+    }
+
+    @Override
+    Long parse(final String str) {
+      return NULL.equals(str)? null: Long.parseLong(str, radix);
+    }
+
+    /** Convert a Short to a String. */ 
+    String toString(final Long n) {
+      return n == null? NULL: Long.toString(n, radix);
+    }
+  }
+}

+ 49 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ModificationTimeParam.java

@@ -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;
+
+/** Modification time parameter. */
+public class ModificationTimeParam extends LongParam {
+  /** Parameter name. */
+  public static final String NAME = "modificationTime";
+  /** Default parameter value. */
+  public static final String DEFAULT = "-1";
+
+  private static final Domain DOMAIN = new Domain(NAME);
+
+  /**
+   * Constructor.
+   * @param value the parameter value.
+   */
+  public ModificationTimeParam(final Long value) {
+    super(DOMAIN, value);
+  }
+
+  /**
+   * Constructor.
+   * @param str a string representation of the parameter value.
+   */
+  public ModificationTimeParam(final String str) {
+    this(DOMAIN.parse(str));
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+}

+ 49 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/OverwriteParam.java

@@ -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;
+
+/** Recursive parameter. */
+public class OverwriteParam extends BooleanParam {
+  /** Parameter name. */
+  public static final String NAME = "overwrite";
+  /** Default parameter value. */
+  public static final String DEFAULT = FALSE;
+
+  private static final Domain DOMAIN = new Domain(NAME);
+
+  /**
+   * Constructor.
+   * @param value the parameter value.
+   */
+  public OverwriteParam(final Boolean value) {
+    super(DOMAIN, value);
+  }
+
+  /**
+   * Constructor.
+   * @param str a string representation of the parameter value.
+   */
+  public OverwriteParam(final String str) {
+    this(DOMAIN.parse(str));
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+}

+ 41 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/OwnerParam.java

@@ -0,0 +1,41 @@
+/**
+ * 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;
+
+/** Owner parameter. */
+public class OwnerParam extends StringParam {
+  /** Parameter name. */
+  public static final String NAME = "owner";
+  /** Default parameter value. */
+  public static final String DEFAULT = "";
+
+  private static final Domain DOMAIN = new Domain(NAME, null);
+
+  /**
+   * Constructor.
+   * @param str a string representation of the parameter value.
+   */
+  public OwnerParam(final String str) {
+    super(DOMAIN, str == null || str.equals(DEFAULT)? null: str);
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+}

+ 104 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/Param.java

@@ -0,0 +1,104 @@
+/**
+ * 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;
+
+import java.util.Arrays;
+import java.util.Comparator;
+
+
+/** Base class of parameters. */
+public abstract class Param<T, D extends Param.Domain<T>> {
+  static final String NULL = "null";
+  
+  static final Comparator<Param<?,?>> NAME_CMP = new Comparator<Param<?,?>>() {
+    @Override
+    public int compare(Param<?, ?> left, Param<?, ?> right) {
+      return left.getName().compareTo(right.getName());
+    }
+  };
+
+  /** Convert the parameters to a sorted String. */
+  public static String toSortedString(final String separator,
+      final Param<?, ?>... parameters) {
+    Arrays.sort(parameters, NAME_CMP);
+    final StringBuilder b = new StringBuilder();
+    for(Param<?, ?> p : parameters) {
+      if (p.getValue() != null) {
+        b.append(separator).append(p);
+      }
+    }
+    return b.toString();
+  }
+
+  /** The domain of the parameter. */
+  final D domain;
+  /** The actual parameter value. */
+  final T value;
+
+  Param(final D domain, final T value) {
+    this.domain = domain;
+    this.value = value;
+  }
+
+  /** @return the parameter value. */
+  public final T getValue() {
+    return value;
+  }
+
+  /** @return the parameter name. */
+  public abstract String getName();
+
+  @Override
+  public String toString() {
+    return getName() + "=" + value;
+  }
+
+  /** Base class of parameter domains. */
+  static abstract class Domain<T> {
+    /** Parameter name. */
+    final String paramName;
+    
+    Domain(final String paramName) {
+      this.paramName = paramName;
+    }
+ 
+    /** @return the parameter name. */
+    public final String getParamName() {
+      return paramName;
+    }
+
+    /** @return a string description of the domain of the parameter. */
+    public abstract String getDomain();
+
+    /** @return the parameter value represented by the string. */
+    abstract T parse(String str);
+
+    /** Parse the given string.
+     * @return the parameter value represented by the string.
+     */
+    public final T parse(final String varName, final String str) {
+      try {
+        return str != null && str.trim().length() > 0 ? parse(str) : null;
+      } catch(Exception e) {
+        throw new IllegalArgumentException("Failed to parse \"" + str
+            + "\" for the parameter " + varName
+            + ".  The value must be in the domain " + getDomain(), e);
+      }
+    }
+  }
+}

+ 57 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/PermissionParam.java

@@ -0,0 +1,57 @@
+/**
+ * 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;
+
+import org.apache.hadoop.fs.permission.FsPermission;
+
+/** Permission parameter, use a Short to represent a FsPermission. */
+public class PermissionParam extends ShortParam {
+  /** Parameter name. */
+  public static final String NAME = "permission";
+  /** Default parameter value. */
+  public static final String DEFAULT = NULL;
+
+  private static final Domain DOMAIN = new Domain(NAME, 8);
+  
+  /**
+   * Constructor.
+   * @param value the parameter value.
+   */
+  public PermissionParam(final FsPermission value) {
+    super(DOMAIN, value == null? null: value.toShort());
+  }
+
+  /**
+   * Constructor.
+   * @param str a string representation of the parameter value.
+   */
+  public PermissionParam(final String str) {
+    super(DOMAIN, DOMAIN.parse(str));
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+
+  /** @return the represented FsPermission. */
+  public FsPermission getFsPermission() {
+    final Short mode = getValue();
+    return mode == null? FsPermission.getDefault(): new FsPermission(mode);
+  }
+}

+ 74 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/PostOpParam.java

@@ -0,0 +1,74 @@
+/**
+ * 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;
+
+import java.net.HttpURLConnection;
+
+/** Http POST operation parameter. */
+public class PostOpParam extends HttpOpParam<PostOpParam.Op> {
+  /** Parameter name. */
+  public static final String NAME = "postOp";
+
+  /** Post operations. */
+  public static enum Op implements HttpOpParam.Op {
+    APPEND(HttpURLConnection.HTTP_OK),
+
+    NULL(HttpURLConnection.HTTP_NOT_IMPLEMENTED);
+
+    final int expectedHttpResponseCode;
+
+    Op(final int expectedHttpResponseCode) {
+      this.expectedHttpResponseCode = expectedHttpResponseCode;
+    }
+
+    @Override
+    public Type getType() {
+      return Type.POST;
+    }
+
+    @Override
+    public boolean getDoOutput() {
+      return true;
+    }
+
+    @Override
+    public int getExpectedHttpResponseCode() {
+      return expectedHttpResponseCode;
+    }
+
+    /** @return a URI query string. */
+    public String toQueryString() {
+      return NAME + "=" + this;
+    }
+  }
+
+  private static final Domain<Op> DOMAIN = new Domain<PostOpParam.Op>(NAME, Op.class);
+
+  /**
+   * Constructor.
+   * @param str a string representation of the parameter value.
+   */
+  public PostOpParam(final String str) {
+    super(DOMAIN, DOMAIN.parse(str));
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+}

+ 84 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/PutOpParam.java

@@ -0,0 +1,84 @@
+/**
+ * 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;
+
+import java.net.HttpURLConnection;
+
+/** Http POST operation parameter. */
+public class PutOpParam extends HttpOpParam<PutOpParam.Op> {
+  /** Parameter name. */
+  public static final String NAME = "putOp";
+
+  /** Put operations. */
+  public static enum Op implements HttpOpParam.Op {
+    CREATE(true, HttpURLConnection.HTTP_CREATED),
+
+    MKDIRS(false, HttpURLConnection.HTTP_OK),
+    RENAME(false, HttpURLConnection.HTTP_OK),
+    SETREPLICATION(false, HttpURLConnection.HTTP_OK),
+
+    SETOWNER(false, HttpURLConnection.HTTP_OK),
+    SETPERMISSION(false, HttpURLConnection.HTTP_OK),
+    SETTIMES(false, HttpURLConnection.HTTP_OK),
+    
+    NULL(false, HttpURLConnection.HTTP_NOT_IMPLEMENTED);
+
+    final boolean doOutput;
+    final int expectedHttpResponseCode;
+
+    Op(final boolean doOutput, final int expectedHttpResponseCode) {
+      this.doOutput = doOutput;
+      this.expectedHttpResponseCode = expectedHttpResponseCode;
+    }
+
+    @Override
+    public HttpOpParam.Type getType() {
+      return HttpOpParam.Type.PUT;
+    }
+
+    @Override
+    public boolean getDoOutput() {
+      return doOutput;
+    }
+
+    @Override
+    public int getExpectedHttpResponseCode() {
+      return expectedHttpResponseCode;
+    }
+
+    @Override
+    public String toQueryString() {
+      return NAME + "=" + this;
+    }
+  }
+
+  private static final Domain<Op> DOMAIN = new Domain<Op>(NAME, Op.class);
+
+  /**
+   * Constructor.
+   * @param str a string representation of the parameter value.
+   */
+  public PutOpParam(final String str) {
+    super(DOMAIN, DOMAIN.parse(str));
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+}

+ 49 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/RecursiveParam.java

@@ -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;
+
+/** Recursive parameter. */
+public class RecursiveParam extends BooleanParam {
+  /** Parameter name. */
+  public static final String NAME = "recursive";
+  /** Default parameter value. */
+  public static final String DEFAULT = FALSE;
+
+  private static final Domain DOMAIN = new Domain(NAME);
+
+  /**
+   * Constructor.
+   * @param value the parameter value.
+   */
+  public RecursiveParam(final Boolean value) {
+    super(DOMAIN, value);
+  }
+
+  /**
+   * Constructor.
+   * @param str a string representation of the parameter value.
+   */
+  public RecursiveParam(final String str) {
+    this(DOMAIN.parse(str));
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+}

+ 52 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/RenameOptionSetParam.java

@@ -0,0 +1,52 @@
+/**
+ * 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;
+
+import org.apache.hadoop.fs.Options;
+
+/** Rename option set parameter. */
+public class RenameOptionSetParam extends EnumSetParam<Options.Rename> {
+  /** Parameter name. */
+  public static final String NAME = "renameOptions";
+  /** Default parameter value. */
+  public static final String DEFAULT = "";
+
+  private static final Domain<Options.Rename> DOMAIN = new Domain<Options.Rename>(
+      NAME, Options.Rename.class);
+
+  /**
+   * Constructor.
+   * @param options rename options.
+   */
+  public RenameOptionSetParam(final Options.Rename... options) {
+    super(DOMAIN, toEnumSet(Options.Rename.class, options));
+  }
+
+  /**
+   * Constructor.
+   * @param str a string representation of the parameter value.
+   */
+  public RenameOptionSetParam(final String str) {
+    super(DOMAIN, DOMAIN.parse(str));
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+}

+ 49 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ReplicationParam.java

@@ -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;
+
+/** Replication parameter. */
+public class ReplicationParam extends ShortParam {
+  /** Parameter name. */
+  public static final String NAME = "replication";
+  /** Default parameter value. */
+  public static final String DEFAULT = NULL;
+
+  private static final Domain DOMAIN = new Domain(NAME);
+
+  /**
+   * Constructor.
+   * @param value the parameter value.
+   */
+  public ReplicationParam(final Short value) {
+    super(DOMAIN, value);
+  }
+
+  /**
+   * Constructor.
+   * @param str a string representation of the parameter value.
+   */
+  public ReplicationParam(final String str) {
+    this(DOMAIN.parse(str));
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+}

+ 60 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ShortParam.java

@@ -0,0 +1,60 @@
+/**
+ * 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;
+
+/** Short parameter. */
+abstract class ShortParam extends Param<Short, ShortParam.Domain> {
+  ShortParam(final Domain domain, final Short value) {
+    super(domain, value);
+  }
+  
+  @Override
+  public String toString() {
+    return getName() + "=" + domain.toString(getValue());
+  }
+
+  /** The domain of the parameter. */
+  static final class Domain extends Param.Domain<Short> {
+    /** The radix of the number. */
+    final int radix;
+
+    Domain(final String paramName) {
+      this(paramName, 10);
+    }
+
+    Domain(final String paramName, final int radix) {
+      super(paramName);
+      this.radix = radix;
+    }
+
+    @Override
+    public String getDomain() {
+      return "<" + NULL + " | short in radix " + radix + ">";
+    }
+
+    @Override
+    Short parse(final String str) {
+      return NULL.equals(str)? null: Short.parseShort(str, radix);
+    }
+
+    /** Convert a Short to a String. */ 
+    String toString(final Short n) {
+      return n == null? NULL: Integer.toString(n, radix);
+    }
+  }
+}

+ 54 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/StringParam.java

@@ -0,0 +1,54 @@
+/**
+ * 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;
+
+import java.util.regex.Pattern;
+
+/** String parameter. */
+abstract class StringParam extends Param<String, StringParam.Domain> {
+  StringParam(final Domain domain, String str) {
+    super(domain, domain.parse(str));
+  }
+
+  /** The domain of the parameter. */
+  static final class Domain extends Param.Domain<String> {
+    /** The pattern defining the domain; null . */
+    private final Pattern pattern;
+
+    Domain(final String paramName, final Pattern pattern) {
+      super(paramName);
+      this.pattern = pattern;
+    }
+
+    @Override
+    public final String getDomain() {
+      return pattern == null ? "<String>" : pattern.pattern();
+    }
+
+    @Override
+    final String parse(final String str) {
+      if (pattern != null) {
+        if (!pattern.matcher(str).matches()) {
+          throw new IllegalArgumentException("Invalid value: \"" + str
+              + "\" does not belong to the domain " + getDomain());
+        }
+      }
+      return str;
+    }
+  }
+}

+ 45 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/UriFsPathParam.java

@@ -0,0 +1,45 @@
+/**
+ * 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;
+
+/** The FileSystem path parameter. */
+public class UriFsPathParam extends StringParam {
+  /** Parameter name. */
+  public static final String NAME = "path";
+
+  private static final Domain DOMAIN = new Domain(NAME, null);
+
+  /**
+   * Constructor.
+   * @param str a string representation of the parameter value.
+   */
+  public UriFsPathParam(String str) {
+    super(DOMAIN, str);
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+
+  /** @return the absolute path. */
+  public final String getAbsolutePath() {
+    final String path = getValue();
+    return path == null? null: "/" + path;
+  }
+}

+ 41 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/UserParam.java

@@ -0,0 +1,41 @@
+/**
+ * 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;
+
+/** User parameter. */
+public class UserParam extends StringParam {
+  /** Parameter name. */
+  public static final String NAME = "user.name";
+  /** Default parameter value. */
+  public static final String DEFAULT = "";
+
+  private static final Domain DOMAIN = new Domain(NAME, null);
+
+  /**
+   * Constructor.
+   * @param str a string representation of the parameter value.
+   */
+  public UserParam(final String str) {
+    super(DOMAIN, str == null || str.equals(DEFAULT)? null: str);
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+}

+ 73 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/UserProvider.java

@@ -0,0 +1,73 @@
+/**
+ * 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;
+
+import java.lang.reflect.Type;
+import java.security.Principal;
+
+import javax.ws.rs.core.Context;
+import javax.ws.rs.ext.Provider;
+
+import com.sun.jersey.api.core.HttpContext;
+import com.sun.jersey.core.spi.component.ComponentContext;
+import com.sun.jersey.core.spi.component.ComponentScope;
+import com.sun.jersey.server.impl.inject.AbstractHttpContextInjectable;
+import com.sun.jersey.spi.inject.Injectable;
+import com.sun.jersey.spi.inject.InjectableProvider;
+
+@Provider
+public class UserProvider extends AbstractHttpContextInjectable<Principal>
+    implements InjectableProvider<Context, Type> {
+
+  @Override
+  public Principal getValue(final HttpContext context) {
+    //get principal from the request
+    final Principal principal = context.getRequest().getUserPrincipal();
+    if (principal != null) {
+      return principal;
+    }
+
+    //get username from the parameter
+    final String username = context.getRequest().getQueryParameters().getFirst(
+        UserParam.NAME);
+    if (username != null) {
+      final UserParam userparam = new UserParam(username);
+      return new Principal() {
+        @Override
+        public String getName() {
+          return userparam.getValue();
+        }
+      };
+    }
+
+    //user not found
+    return null;
+  }
+
+  @Override
+  public ComponentScope getScope() {
+    return ComponentScope.PerRequest;
+  }
+
+  @Override
+  public Injectable<Principal> getInjectable(
+      final ComponentContext componentContext, final Context context,
+      final Type type) {
+    return type.equals(Principal.class)? this : null;
+  }
+}

+ 15 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java

@@ -63,6 +63,7 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
+import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.net.DNSToSwitchMapping;
 import org.apache.hadoop.net.NetUtils;
@@ -815,6 +816,8 @@ public class MiniDFSCluster {
                              long[] simulatedCapacities,
                              boolean setupHostsFile,
                              boolean checkDataNodeAddrConfig) throws IOException {
+    conf.set(DFSConfigKeys.DFS_DATANODE_HOST_NAME_KEY, "127.0.0.1");
+
     int curDatanodesNum = dataNodes.size();
     // for mincluster's the default initialDelay for BRs is 0
     if (conf.get(DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_KEY) == null) {
@@ -1458,6 +1461,18 @@ 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. 
    */

+ 96 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestFSMainOperationsWebHdfs.java

@@ -0,0 +1,96 @@
+/**
+ * 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;
+
+
+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 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.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.web.resources.ExceptionHandler;
+import org.apache.log4j.Level;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestFSMainOperationsWebHdfs extends FSMainOperationsBaseTest {
+  {
+    ((Log4JLogger)ExceptionHandler.LOG).getLogger().setLevel(Level.ALL);
+  }
+
+  private static final MiniDFSCluster cluster;
+  private static final Path defaultWorkingDirectory;
+
+  static {
+    Configuration conf = new Configuration();
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
+      fSys = cluster.getWebHdfsFileSystem();
+      defaultWorkingDirectory = fSys.getWorkingDirectory();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  protected Path getDefaultWorkingDirectory() {
+    return defaultWorkingDirectory;
+  }
+
+  /** 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));
+  }
+  
+  
+  //The following tests failed for HftpFileSystem,
+  //Disable it for WebHdfsFileSystem
+  @Test
+  public void testListStatusThrowsExceptionForNonExistentFile() {}
+  @Test
+  public void testListStatusThrowsExceptionForUnreadableDir() {}
+  @Test
+  public void testGlobStatusThrowsExceptionForNonExistentFile() {}
+}

+ 55 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java

@@ -0,0 +1,55 @@
+/**
+ * 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;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.web.JsonUtil;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestJsonUtil {
+  static FileStatus toFileStatus(HdfsFileStatus f, String parent) {
+    return new FileStatus(f.getLen(), f.isDir(), f.getReplication(),
+        f.getBlockSize(), f.getModificationTime(), f.getAccessTime(),
+        f.getPermission(), f.getOwner(), f.getGroup(),
+        new Path(f.getFullName(parent)));
+  }
+
+  @Test
+  public void testHdfsFileStatus() {
+    final long now = System.currentTimeMillis();
+    final String parent = "/dir";
+    final HdfsFileStatus status = new HdfsFileStatus(1001L, false, 3, 1L<<26,
+        now, now + 10, new FsPermission((short)0644), "user", "group",
+        DFSUtil.string2Bytes("bar"), DFSUtil.string2Bytes("foo"));
+    final FileStatus fstatus = toFileStatus(status, parent);
+    System.out.println("status  = " + status);
+    System.out.println("fstatus = " + fstatus);
+    final String json = JsonUtil.toJsonString(status);
+    System.out.println("json    = " + json.replace(",", ",\n  "));
+    final HdfsFileStatus s2 = JsonUtil.toFileStatus(JsonUtil.parse(json));
+    final FileStatus fs2 = toFileStatus(s2, parent);
+    System.out.println("s2      = " + s2);
+    System.out.println("fs2     = " + fs2);
+    Assert.assertEquals(fstatus, fs2);
+  }
+}

+ 86 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java

@@ -0,0 +1,86 @@
+/**
+ * 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;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+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 MiniDFSCluster cluster;
+  private String defaultWorkingDirectory;
+
+  static {
+    Configuration conf = new Configuration();
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  protected void setUp() throws Exception {
+    fs = cluster.getWebHdfsFileSystem();
+    defaultWorkingDirectory = "/user/"
+        + UserGroupInformation.getCurrentUser().getShortUserName();
+  }
+
+  @Override
+  protected String getDefaultWorkingDirectory() {
+    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
+  public void testListStatusThrowsExceptionForNonExistentFile() {}
+}