You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ar...@apache.org on 2015/06/26 23:34:25 UTC

hadoop git commit: HDFS-8644. OzoneHandler : Add volume handler. (Contributed by Anu Engineer)

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-7240 845a71063 -> ae655ea00


HDFS-8644. OzoneHandler : Add volume handler. (Contributed by Anu Engineer)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/ae655ea0
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ae655ea0
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ae655ea0

Branch: refs/heads/HDFS-7240
Commit: ae655ea00b006fbe2ec0fd868819e5d3360371bd
Parents: 845a710
Author: Arpit Agarwal <ar...@apache.org>
Authored: Fri Jun 26 14:33:51 2015 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Fri Jun 26 14:33:51 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-7240.txt           |   3 +
 .../apache/hadoop/ozone/OzoneConfigKeys.java    |  37 +++
 .../ozone/StorageContainerConfiguration.java    |  35 +++
 .../hadoop/ozone/web/exceptions/ErrorTable.java |  10 +-
 .../ozone/web/exceptions/OzoneException.java    |   2 +
 .../web/handlers/StorageHandlerBuilder.java     |  73 +++++
 .../hadoop/ozone/web/handlers/UserArgs.java     |  33 ++-
 .../ozone/web/handlers/UserHandlerBuilder.java  |  75 ++++++
 .../hadoop/ozone/web/handlers/VolumeArgs.java   |  23 +-
 .../ozone/web/handlers/VolumeHandler.java       | 228 ++++++++++++++++
 .../web/handlers/VolumeProcessTemplate.java     | 235 ++++++++++++++++
 .../apache/hadoop/ozone/web/headers/Header.java |  10 +
 .../ozone/web/interfaces/StorageHandler.java    |   2 +
 .../hadoop/ozone/web/interfaces/UserAuth.java   |   2 +
 .../hadoop/ozone/web/interfaces/Volume.java     |   2 +
 .../web/localstorage/LocalStorageHandler.java   | 150 +++++++++++
 .../hadoop/ozone/web/request/OzoneQuota.java    |   2 +
 .../hadoop/ozone/web/response/ListVolumes.java  |   2 +
 .../hadoop/ozone/web/response/VolumeInfo.java   |   2 +
 .../hadoop/ozone/web/response/VolumeOwner.java  |   2 +
 .../hadoop/ozone/web/userauth/Simple.java       |   2 +
 .../hadoop/ozone/web/utils/OzoneConsts.java     |   9 +
 .../hadoop/ozone/web/utils/OzoneUtils.java      | 268 +++++++++++++++++++
 .../apache/hadoop/ozone/web/TestErrorCode.java  |  50 ++++
 .../org/apache/hadoop/ozone/web/TestUtils.java  | 100 +++++++
 25 files changed, 1346 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae655ea0/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-7240.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-7240.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-7240.txt
index c466495..82e4718 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-7240.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-7240.txt
@@ -20,3 +20,6 @@
     HDFS-8634. OzoneHandler: Add userAuth Interface and Simple userAuth
     handler. (Anu Engineer via Arpit Agarwal)
 
+    HDFS-8644. OzoneHandler : Add volume handler. (Anu Engineer via
+    Arpit Agarwal)
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae655ea0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
new file mode 100644
index 0000000..cf9fad6
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
@@ -0,0 +1,37 @@
+/**
+ * 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.ozone;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * This class contains constants for configuration keys used in Ozone.
+ */
+@InterfaceAudience.Private
+public final class OzoneConfigKeys {
+  public static final String DFS_STORAGE_LOCAL_ROOT =
+      "dfs.ozone.localstorage.root";
+  public static final String DFS_STORAGE_LOCAL_ROOT_DEFAULT = "/tmp/ozone";
+
+  /**
+   * There is no need to instantiate this class.
+   */
+  private OzoneConfigKeys() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae655ea0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/StorageContainerConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/StorageContainerConfiguration.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/StorageContainerConfiguration.java
new file mode 100644
index 0000000..a83fa33
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/StorageContainerConfiguration.java
@@ -0,0 +1,35 @@
+/**
+ * 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.ozone;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Configuration for ozone.
+ */
+@InterfaceAudience.Private
+public class StorageContainerConfiguration extends Configuration {
+  static {
+    // adds the default resources
+    Configuration.addDefaultResource("hdfs-default.xml");
+    Configuration.addDefaultResource("hdfs-site.xml");
+    Configuration.addDefaultResource("ozone-site.xml");
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae655ea0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/exceptions/ErrorTable.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/exceptions/ErrorTable.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/exceptions/ErrorTable.java
index be49d04..a51dac5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/exceptions/ErrorTable.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/exceptions/ErrorTable.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.ozone.web.exceptions;
 
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.ozone.web.handlers.UserArgs;
 
 import static java.net.HttpURLConnection.HTTP_BAD_REQUEST;
@@ -33,6 +34,7 @@ import static java.net.HttpURLConnection.HTTP_UNAUTHORIZED;
  * Please note : The errors in this table are sorted by the HTTP_ERROR codes
  * if you add new error codes to this table please follow the same convention.
  */
+@InterfaceAudience.Private
 public final class ErrorTable {
 
   /* Error 400 */
@@ -149,12 +151,12 @@ public final class ErrorTable {
    *
    * @return creates a new instance of error based on the template
    */
-  public static OzoneException newError(OzoneException e, long requestID,
+  public static OzoneException newError(OzoneException e, String requestID,
                                         String resource, String hostID) {
     OzoneException err =
         new OzoneException(e.getHttpCode(), e.getShortMessage(),
                            e.getMessage());
-    err.setRequestId(Long.toString(requestID));
+    err.setRequestId(requestID);
     err.setResource(resource);
     err.setHostID(hostID);
     return err;
@@ -172,7 +174,7 @@ public final class ErrorTable {
     OzoneException err =
         new OzoneException(e.getHttpCode(), e.getShortMessage(),
                            e.getMessage());
-    err.setRequestId(Long.toString(args.getRequestID()));
+    err.setRequestId(args.getRequestID());
     err.setResource(args.getResourceName());
     err.setHostID(args.getHostName());
     return err;
@@ -191,7 +193,7 @@ public final class ErrorTable {
                                         Exception ex) {
     OzoneException err =
         new OzoneException(e.getHttpCode(), e.getShortMessage(), ex);
-    err.setRequestId(Long.toString(args.getRequestID()));
+    err.setRequestId(args.getRequestID());
     err.setResource(args.getResourceName());
     err.setHostID(args.getHostName());
     err.setMessage(ex.getMessage());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae655ea0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/exceptions/OzoneException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/exceptions/OzoneException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/exceptions/OzoneException.java
index e4a2b2f..576966f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/exceptions/OzoneException.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/exceptions/OzoneException.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.ozone.web.exceptions;
 
 
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.codehaus.jackson.annotate.JsonAutoDetect;
 import org.codehaus.jackson.annotate.JsonProperty;
 import org.codehaus.jackson.map.ObjectMapper;
@@ -29,6 +30,7 @@ import java.io.IOException;
  * Class the represents various errors returned by the
  * Ozone Layer.
  */
+@InterfaceAudience.Private
 public class OzoneException extends Exception {
 
   @JsonProperty("httpCode")

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae655ea0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/StorageHandlerBuilder.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/StorageHandlerBuilder.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/StorageHandlerBuilder.java
new file mode 100644
index 0000000..37c27da
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/StorageHandlerBuilder.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.ozone.web.handlers;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
+import org.apache.hadoop.ozone.web.localstorage.LocalStorageHandler;
+
+/**
+ * This class is responsible for providing a {@link StorageHandler}
+ * implementation to object store web handlers.
+ */
+@InterfaceAudience.Private
+public final class StorageHandlerBuilder {
+
+  private static final ThreadLocal<StorageHandler>
+      STORAGE_HANDLER_THREAD_LOCAL = new ThreadLocal<>();
+
+  /**
+   * Returns the configured StorageHandler from thread-local storage for this
+   * thread.
+   *
+   * @return StorageHandler from thread-local storage
+   */
+  public static StorageHandler getStorageHandler() {
+    StorageHandler storageHandler = STORAGE_HANDLER_THREAD_LOCAL.get();
+    if (storageHandler != null) {
+      return storageHandler;
+    } else {
+      // This only happens while using mvn jetty:run for testing.
+      return new LocalStorageHandler();
+    }
+  }
+
+  /**
+   * Removes the configured StorageHandler from thread-local storage for this
+   * thread.
+   */
+  public static void removeStorageHandler() {
+    STORAGE_HANDLER_THREAD_LOCAL.remove();
+  }
+
+  /**
+   * Sets the configured StorageHandler in thread-local storage for this thread.
+   *
+   * @param storageHandler StorageHandler to set in thread-local storage
+   */
+  public static void setStorageHandler(StorageHandler storageHandler) {
+    STORAGE_HANDLER_THREAD_LOCAL.set(storageHandler);
+  }
+
+  /**
+   * There is no reason to instantiate this class.
+   */
+  private StorageHandlerBuilder() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae655ea0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/UserArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/UserArgs.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/UserArgs.java
index b80de3c..b83303e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/UserArgs.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/UserArgs.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.ozone.web.handlers;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+
 import javax.ws.rs.core.HttpHeaders;
 import javax.ws.rs.core.Request;
 import javax.ws.rs.core.UriInfo;
@@ -25,9 +27,10 @@ import javax.ws.rs.core.UriInfo;
  * UserArgs is used to package caller info
  * and pass it down to file system.
  */
+@InterfaceAudience.Private
 public class UserArgs {
   private String userName;
-  private final long requestID;
+  private final String requestID;
   private final String hostName;
   private final UriInfo uri;
   private final Request request;
@@ -38,14 +41,14 @@ public class UserArgs {
    * Constructs  user args.
    *
    * @param userName - User name
-   * @param requestID _ Request ID
+   * @param requestID - Request ID
    * @param hostName - Host Name
-   * @param req  - Request
+   * @param req - Request
    * @param info - Uri Info
    * @param httpHeaders - http headers
    */
-  public UserArgs(String userName, long requestID, String hostName, Request req,
-                  UriInfo info, HttpHeaders httpHeaders) {
+  public UserArgs(String userName, String requestID, String hostName,
+                  Request req, UriInfo info, HttpHeaders httpHeaders) {
     this.hostName = hostName;
     this.userName = userName;
     this.requestID = requestID;
@@ -55,6 +58,24 @@ public class UserArgs {
   }
 
   /**
+   * Constructs  user args when we don't know the user name yet.
+   *
+   * @param requestID _ Request ID
+   * @param hostName - Host Name
+   * @param req - Request
+   * @param info - UriInfo
+   * @param httpHeaders - http headers
+   */
+  public UserArgs(String requestID, String hostName, Request req, UriInfo info,
+                  HttpHeaders httpHeaders) {
+    this.hostName = hostName;
+    this.requestID = requestID;
+    this.uri = info;
+    this.request = req;
+    this.headers = httpHeaders;
+  }
+
+  /**
    * Returns hostname.
    *
    * @return String
@@ -68,7 +89,7 @@ public class UserArgs {
    *
    * @return Long
    */
-  public long getRequestID() {
+  public String getRequestID() {
     return requestID;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae655ea0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/UserHandlerBuilder.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/UserHandlerBuilder.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/UserHandlerBuilder.java
new file mode 100644
index 0000000..d9051f3
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/UserHandlerBuilder.java
@@ -0,0 +1,75 @@
+/*
+ * 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.ozone.web.handlers;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.ozone.web.interfaces.UserAuth;
+import org.apache.hadoop.ozone.web.userauth.Simple;
+
+/**
+ * This class is responsible for providing a
+ * {@link org.apache.hadoop.ozone.web.interfaces.UserAuth}
+ * implementation to object store web handlers.
+ */
+@InterfaceAudience.Private
+public final class UserHandlerBuilder {
+
+  private static final ThreadLocal<UserAuth> USER_AUTH_THREAD_LOCAL =
+      new ThreadLocal<UserAuth>();
+
+  /**
+   * Returns the configured UserAuth from thread-local storage for this
+   * thread.
+   *
+   * @return UserAuth from thread-local storage
+   */
+  public static UserAuth getAuthHandler() {
+    UserAuth authHandler = USER_AUTH_THREAD_LOCAL.get();
+    if (authHandler != null) {
+      return authHandler;
+    } else {
+      // This only happens while using mvn jetty:run for testing.
+      return new Simple();
+    }
+  }
+
+  /**
+   * Removes the configured UserAuth from thread-local storage for this
+   * thread.
+   */
+  public static void removeAuthHandler() {
+    USER_AUTH_THREAD_LOCAL.remove();
+  }
+
+  /**
+   * Sets the configured UserAuthHandler in thread-local storage for this
+   * thread.
+   *
+   * @param authHandler authHandler to set in thread-local storage
+   */
+  public static void setAuthHandler(UserAuth authHandler) {
+    USER_AUTH_THREAD_LOCAL.set(authHandler);
+  }
+
+  /**
+   * There is no reason to instantiate this class.
+   */
+  private UserHandlerBuilder() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae655ea0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/VolumeArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/VolumeArgs.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/VolumeArgs.java
index 5bb6255..def1f0e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/VolumeArgs.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/VolumeArgs.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.ozone.web.handlers;
 
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.ozone.web.request.OzoneQuota;
 
 import javax.ws.rs.core.HttpHeaders;
@@ -29,6 +30,7 @@ import javax.ws.rs.core.UriInfo;
  * related arguments in the call to underlying
  * file system.
  */
+@InterfaceAudience.Private
 public class VolumeArgs extends UserArgs {
   private String adminName;
   private final String volumeName;
@@ -63,7 +65,7 @@ public class VolumeArgs extends UserArgs {
    * @param info - URI info
    * @param headers - http headers
    */
-  public VolumeArgs(String userName, String volumeName, long requestID,
+  public VolumeArgs(String userName, String volumeName, String requestID,
                     String hostName, Request request, UriInfo info,
                     HttpHeaders headers) {
     super(userName, requestID, hostName, request, info, headers);
@@ -71,6 +73,25 @@ public class VolumeArgs extends UserArgs {
   }
 
   /**
+   * Constructs  volume Args.
+   *
+   * @param volumeName - volume Name
+   * @param userArgs - userArgs
+   */
+  public VolumeArgs(String volumeName, UserArgs userArgs) {
+    this(userArgs.getUserName(), volumeName, userArgs.getRequestID(),
+         userArgs.getHostName(), userArgs.getRequest(), userArgs.getUri(),
+         userArgs.getHeaders());
+  }
+
+  /**
+   * Creates VolumeArgs from another VolumeArgs.
+   */
+  public VolumeArgs(VolumeArgs volArgs) {
+    this(volArgs.getVolumeName(), volArgs);
+  }
+
+  /**
    * Sets Quota information.
    *
    * @param quota - Quota Sting

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae655ea0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/VolumeHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/VolumeHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/VolumeHandler.java
new file mode 100644
index 0000000..73a7a07
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/VolumeHandler.java
@@ -0,0 +1,228 @@
+/*
+ * 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.ozone.web.handlers;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.ozone.web.exceptions.ErrorTable;
+import org.apache.hadoop.ozone.web.exceptions.OzoneException;
+import org.apache.hadoop.ozone.web.headers.Header;
+import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
+import org.apache.hadoop.ozone.web.interfaces.UserAuth;
+import org.apache.hadoop.ozone.web.interfaces.Volume;
+import org.apache.hadoop.ozone.web.utils.OzoneUtils;
+
+import javax.ws.rs.core.HttpHeaders;
+import javax.ws.rs.core.Request;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.UriInfo;
+import java.io.IOException;
+
+import static java.net.HttpURLConnection.HTTP_CREATED;
+import static java.net.HttpURLConnection.HTTP_OK;
+
+/**
+ * VolumeHandler handles volume specific HTTP calls.
+ *
+ * Most functions in this file follow a simple pattern.
+ * All calls are handled by VolumeProcessTemplate.handleCall, which
+ * calls back into doProcess function.
+ *
+ * Everything common to volume handling is abstracted out in handleCall function
+ * For Example : Checking that volume name is sane, we have a supported
+ * ozone version number and a valid date. That is everything common is in
+ * handleCall and actions specific to a call is inside doProcess callback.
+ */
+@InterfaceAudience.Private
+public class VolumeHandler implements Volume {
+  /**
+   * Creates a volume.
+   *
+   * @param volume Volume Name, this has to be unique at Ozone cluster level
+   * @param quota Quota for this Storage Volume - <int>(<BYTES|MB|GB|TB>)
+   * @param req Request Object
+   * @param uriInfo URI info
+   * @param headers Http Headers
+   *
+   * @return Standard JAX-RS Response
+   *
+   * @throws OzoneException
+   */
+  @Override
+  public Response createVolume(String volume, final String quota, Request req,
+                               UriInfo uriInfo, HttpHeaders headers)
+      throws OzoneException {
+    return new VolumeProcessTemplate() {
+      @Override
+      public Response doProcess(VolumeArgs args)
+          throws IOException, OzoneException {
+        UserAuth auth = UserHandlerBuilder.getAuthHandler();
+        if (auth.isAdmin(args)) {
+          args.setAdminName(args.getUserName());
+          String volumeOwner = auth.getOzoneUser(args);
+
+          if (volumeOwner == null) {
+            throw ErrorTable.newError(ErrorTable.USER_NOT_FOUND, args);
+          }
+
+          if (!auth.isUser(volumeOwner, args)) {
+            throw ErrorTable.newError(ErrorTable.USER_NOT_FOUND, args);
+          }
+
+          args.setUserName(volumeOwner);
+          if (!quota.equals(Header.OZONE_QUOTA_UNDEFINED)) {
+            setQuotaArgs(args, quota);
+          }
+          StorageHandler fs = StorageHandlerBuilder.getStorageHandler();
+          fs.createVolume(args);
+          return OzoneUtils.getResponse(args, HTTP_CREATED, "");
+        } else {
+          throw ErrorTable.newError(ErrorTable.ACCESS_DENIED, args);
+        }
+      }
+    }.handleCall(volume, req, uriInfo, headers);
+  }
+
+  /**
+   * Updates  volume metadata.
+   *
+   * There are only two actions possible currently with updateVolume.
+   * Change the volume ownership or update quota. if you make a call
+   * with neither of these actions, update just returns 200 OK.
+   *
+   * @param volume Volume Name, this has to be unique at Ozone Level
+   * @param quota Quota for this volume - <int>(<BYTES|MB|GB|TB>)|remove
+   * @param req - Request Object
+   * @param uriInfo - URI info
+   * @param headers Http Headers
+   *
+   * @return Standard JAX-RS Response
+   *
+   * @throws OzoneException
+   */
+  @Override
+  public Response updateVolume(String volume, final String quota, Request req,
+                               UriInfo uriInfo, HttpHeaders headers)
+      throws OzoneException {
+    return new VolumeProcessTemplate() {
+      @Override
+      public Response doProcess(VolumeArgs args)
+          throws IOException, OzoneException {
+        UserAuth auth = UserHandlerBuilder.getAuthHandler();
+        if (auth.isAdmin(args)) {
+          StorageHandler fs = StorageHandlerBuilder.getStorageHandler();
+          args.setAdminName(args.getUserName());
+          String newVolumeOwner = auth.getOzoneUser(args);
+
+          if (newVolumeOwner != null) {
+            if (!auth.isUser(newVolumeOwner, args)) {
+              throw ErrorTable.newError(ErrorTable.USER_NOT_FOUND, args);
+            }
+            args.setUserName(newVolumeOwner);
+            fs.setVolumeOwner(args);
+          }
+
+          if (!quota.equals(Header.OZONE_QUOTA_UNDEFINED)) {
+            if (quota.equals(Header.OZONE_QUOTA_REMOVE)) {
+              // if it is remove, just tell the file system to remove quota
+              fs.setVolumeQuota(args, true);
+            } else {
+              setQuotaArgs(args, quota);
+              fs.setVolumeQuota(args, false);
+            }
+          }
+          return OzoneUtils.getResponse(args, HTTP_OK, "");
+        } else {
+          // Only Admins are allowed to update volumes
+          throw ErrorTable.newError(ErrorTable.ACCESS_DENIED, args);
+        }
+      }
+    }.handleCall(volume, req, uriInfo, headers);
+  }
+
+
+  /**
+   * Deletes a volume if it is empty.
+   *
+   * @param volume Volume Name
+   * @param req - Http Request
+   * @param uriInfo - http URI
+   * @param headers - http headers
+   *
+   * @return Standard JAX-RS Response
+   *
+   * @throws OzoneException
+   */
+  @Override
+  public Response deleteVolume(String volume, Request req, UriInfo uriInfo,
+                               HttpHeaders headers) throws OzoneException {
+    return new VolumeProcessTemplate() {
+      @Override
+      public Response doProcess(VolumeArgs args)
+          throws IOException, OzoneException {
+        UserAuth auth = UserHandlerBuilder.getAuthHandler();
+        if (auth.isAdmin(args)) {
+          StorageHandler fs = StorageHandlerBuilder.getStorageHandler();
+          fs.deleteVolume(args);
+          return OzoneUtils.getResponse(args, HTTP_OK, "");
+        } else {
+          throw ErrorTable.newError(ErrorTable.ACCESS_DENIED, args);
+        }
+      }
+    }.handleCall(volume, req, uriInfo, headers);
+  }
+
+  /**
+   * Returns Volume info. This API can be invoked either
+   * by admin or the owner
+   *
+   * @param volume - Storage Volume Name
+   * @param req - Http Req
+   * @param uriInfo - http URI
+   * @param headers - Http headers  @return - Response
+   *
+   * @throws OzoneException
+   */
+  @Override
+  public Response getVolumeInfo(String volume, final String info, Request req,
+                                final UriInfo uriInfo, HttpHeaders headers)
+      throws OzoneException {
+    return new VolumeProcessTemplate() {
+      @Override
+      public Response doProcess(VolumeArgs args)
+          throws IOException, OzoneException {
+
+        switch (info) {
+          case Header.OZONE_LIST_QUERY_BUCKET:
+            // TODO : Resolve this dependency when we bring
+            // in bucket code.
+            // return getBucketsInVolume(args); // Return list of Buckets
+          case Header.OZONE_LIST_QUERY_VOLUME:
+            return getVolumeInfoResponse(args); // Return volume info
+          case Header.OZONE_LIST_QUERY_SERVICE:
+            return getVolumesByUser(args); // Return list of volumes
+          default:
+            OzoneException ozoneException =
+                ErrorTable.newError(ErrorTable.INVALID_QUERY_PARAM, args);
+            ozoneException.setMessage("Unrecognized query param : " + info);
+            throw ozoneException;
+        }
+      }
+    }.handleCall(volume, req, uriInfo, headers);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae655ea0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/VolumeProcessTemplate.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/VolumeProcessTemplate.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/VolumeProcessTemplate.java
new file mode 100644
index 0000000..8f4d4a6
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/VolumeProcessTemplate.java
@@ -0,0 +1,235 @@
+/*
+ * 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.ozone.web.handlers;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.ozone.web.exceptions.ErrorTable;
+import org.apache.hadoop.ozone.web.exceptions.OzoneException;
+import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
+import org.apache.hadoop.ozone.web.interfaces.UserAuth;
+import org.apache.hadoop.ozone.web.response.ListVolumes;
+import org.apache.hadoop.ozone.web.response.VolumeInfo;
+import org.apache.hadoop.ozone.web.utils.OzoneUtils;
+
+import javax.ws.rs.core.HttpHeaders;
+import javax.ws.rs.core.Request;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.UriInfo;
+import java.io.IOException;
+import java.nio.file.DirectoryNotEmptyException;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.NoSuchFileException;
+
+import static java.net.HttpURLConnection.HTTP_OK;
+
+/**
+ * This class abstracts way the repetitive tasks in
+ * handling volume related code.
+ */
+@InterfaceAudience.Private
+public abstract class VolumeProcessTemplate {
+
+
+  /**
+   * The handle call is the common functionality for Volume
+   * handling code.
+   *
+   * @param volume - Name of the Volume
+   * @param request - request
+   * @param info - UriInfo
+   * @param headers - Http Headers
+   *
+   * @return Response
+   *
+   * @throws OzoneException
+   */
+  public Response handleCall(String volume, Request request, UriInfo info,
+                             HttpHeaders headers) throws OzoneException {
+    String reqID = OzoneUtils.getRequestID();
+    String hostName = OzoneUtils.getHostName();
+    try {
+
+      OzoneUtils.validate(request, headers, reqID, volume, hostName);
+
+      // we use the same logic for both bucket and volume names
+      OzoneUtils.verifyBucketName(volume);
+      UserAuth auth = UserHandlerBuilder.getAuthHandler();
+      UserArgs userArgs = new UserArgs(reqID, hostName, request, info, headers);
+
+      userArgs.setUserName(auth.getUser(userArgs));
+      VolumeArgs args = new VolumeArgs(volume, userArgs);
+
+      return doProcess(args);
+    } catch (IllegalArgumentException ex) {
+      OzoneException exp = ErrorTable
+          .newError(ErrorTable.INVALID_VOLUME_NAME, reqID, volume, hostName);
+      exp.setMessage(ex.getMessage());
+      throw exp;
+    } catch (IOException ex) {
+      handleIOException(volume, reqID, hostName, ex);
+    }
+    return null;
+  }
+
+  /**
+   * Specific handler for each call.
+   *
+   * @param args - Volume Args
+   *
+   * @return - Response
+   *
+   * @throws IOException
+   * @throws OzoneException
+   */
+  public abstract Response doProcess(VolumeArgs args)
+      throws IOException, OzoneException;
+
+  /**
+   * Maps Java File System Exceptions to Ozone Exceptions in the Volume path.
+   *
+   * @param volume - Name of the Volume
+   * @param reqID - Request ID
+   * @param hostName - HostName
+   * @param fsExp - Exception
+   *
+   * @throws OzoneException
+   */
+  private void handleIOException(String volume, String reqID, String hostName,
+                                 IOException fsExp) throws OzoneException {
+    OzoneException exp = null;
+
+    if (fsExp instanceof FileAlreadyExistsException) {
+      exp = ErrorTable
+          .newError(ErrorTable.VOLUME_ALREADY_EXISTS, reqID, volume, hostName);
+    }
+
+    if (fsExp instanceof DirectoryNotEmptyException) {
+      exp = ErrorTable
+          .newError(ErrorTable.VOLUME_NOT_EMPTY, reqID, volume, hostName);
+    }
+
+    if (fsExp instanceof NoSuchFileException) {
+      exp = ErrorTable
+          .newError(ErrorTable.INVALID_VOLUME_NAME, reqID, volume, hostName);
+    }
+
+    if ((fsExp != null) && (exp != null)) {
+      exp.setMessage(fsExp.getMessage());
+    }
+
+    // We don't handle that FS error yet, report a Server Internal Error
+    if (exp == null) {
+      exp =
+          ErrorTable.newError(ErrorTable.SERVER_ERROR, reqID, volume, hostName);
+      if (fsExp != null) {
+        exp.setMessage(fsExp.getMessage());
+      }
+    }
+    throw exp;
+  }
+
+  /**
+   * Set the user provided string into args and throw ozone exception
+   * if needed.
+   *
+   * @param args - volume args
+   * @param quota - quota sting
+   *
+   * @throws OzoneException
+   */
+  void setQuotaArgs(VolumeArgs args, String quota) throws OzoneException {
+    try {
+      args.setQuota(quota);
+    } catch (IllegalArgumentException ex) {
+      throw ErrorTable.newError(ErrorTable.MALFORMED_QUOTA, args, ex);
+    }
+  }
+
+  /**
+   * Wraps calls into volumeInfo data.
+   *
+   * @param args - volumeArgs
+   *
+   * @return - VolumeInfo
+   *
+   * @throws IOException
+   * @throws OzoneException
+   */
+  Response getVolumeInfoResponse(VolumeArgs args)
+      throws IOException, OzoneException {
+    StorageHandler fs = StorageHandlerBuilder.getStorageHandler();
+    VolumeInfo info = fs.getVolumeInfo(args);
+    return OzoneUtils.getResponse(args, HTTP_OK, info.toJsonString());
+  }
+
+
+  /**
+   * Returns all the volumes belonging to a user.
+   *
+   * @param user - userArgs
+   *
+   * @return - Response
+   *
+   * @throws OzoneException
+   * @throws IOException
+   */
+  Response getVolumesByUser(UserArgs user) throws OzoneException, IOException {
+    StorageHandler fs = StorageHandlerBuilder.getStorageHandler();
+    ListVolumes volumes = fs.listVolumes(user);
+    return OzoneUtils.getResponse(user, HTTP_OK, volumes.toJsonString());
+  }
+
+
+  /**
+   * This call can also be invoked by Admins of the system where they can
+   * get the list of buckets of any user.
+   *
+   * User makes a call like
+   * GET / HTTP/1.1
+   * Host: ozone.self
+   *
+   * @param args - volumeArgs
+   *
+   * @return Response - A list of buckets owned this user
+   *
+   * @throws OzoneException
+   */
+  Response getVolumesByUser(VolumeArgs args) throws OzoneException {
+    String validatedUser = args.getUserName();
+    try {
+      UserAuth auth = UserHandlerBuilder.getAuthHandler();
+      if (auth.isAdmin(args)) {
+        validatedUser = auth.getOzoneUser(args);
+        if (validatedUser == null) {
+          validatedUser = auth.getUser(args);
+        }
+      }
+
+      UserArgs user =
+          new UserArgs(validatedUser, args.getRequestID(), args.getHostName(),
+                       args.getRequest(), args.getUri(), args.getHeaders());
+      return getVolumesByUser(user);
+    } catch (IOException ex) {
+      OzoneException exp = ErrorTable.newError(ErrorTable.SERVER_ERROR, args);
+      exp.setMessage("unable to get the volume list for the user");
+      throw exp;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae655ea0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/headers/Header.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/headers/Header.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/headers/Header.java
index 53cf0f9..f217038 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/headers/Header.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/headers/Header.java
@@ -18,9 +18,12 @@
 
 package org.apache.hadoop.ozone.web.headers;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+
 /**
  * OZONE specific HTTP headers.
  */
+@InterfaceAudience.Private
 public final class Header {
   public static final String OZONE_QUOTA_BYTES = "BYTES";
   public static final String OZONE_QUOTA_MB = "MB";
@@ -33,6 +36,13 @@ public final class Header {
 
   public static final String OZONE_USER = "x-ozone-user";
   public static final String OZONE_SIMPLE_AUTHENTICATION_SCHEME = "OZONE";
+  public static final String OZONE_VERSION_HEADER = "x-ozone-version";
+
+  public static final String OZONE_LIST_QUERY_SERVICE = "service";
+  public static final String OZONE_LIST_QUERY_VOLUME = "volume";
+
+  public static final String OZONE_REQUEST_ID = "x-ozone-request-id";
+  public static final String OZONE_SERVER_NAME = "x-ozone-server-name";
 
   private Header() {
     // Never constructed.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae655ea0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/StorageHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/StorageHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/StorageHandler.java
index 2088409..043b59e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/StorageHandler.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/StorageHandler.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.ozone.web.interfaces;
 
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.ozone.web.exceptions.OzoneException;
 import org.apache.hadoop.ozone.web.handlers.UserArgs;
 import org.apache.hadoop.ozone.web.handlers.VolumeArgs;
@@ -34,6 +35,7 @@ import java.io.IOException;
  * One for the local file system that is handy while testing
  * and another which will point to the HDFS backend.
  */
+@InterfaceAudience.Private
 public interface StorageHandler {
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae655ea0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/UserAuth.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/UserAuth.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/UserAuth.java
index 1e2c3b3..2ada012 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/UserAuth.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/UserAuth.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.ozone.web.interfaces;
 
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.ozone.web.exceptions.OzoneException;
 import org.apache.hadoop.ozone.web.handlers.UserArgs;
 
@@ -26,6 +27,7 @@ import org.apache.hadoop.ozone.web.handlers.UserArgs;
  *
  * Please see concrete implementations for more information
  */
+@InterfaceAudience.Private
 public interface UserAuth {
   /**
    * Returns the user name as a string from the URI and HTTP headers.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae655ea0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/Volume.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/Volume.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/Volume.java
index 6ad742a..db4bda6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/Volume.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/Volume.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.ozone.web.interfaces;
 
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.ozone.web.exceptions.OzoneException;
 import org.apache.hadoop.ozone.web.headers.Header;
 
@@ -39,6 +40,7 @@ import javax.ws.rs.core.UriInfo;
  * Volume Interface acts as the HTTP entry point for
  * volume related functionality.
  */
+@InterfaceAudience.Private
 @Path("/{volume}")
 public interface Volume {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae655ea0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/LocalStorageHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/LocalStorageHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/LocalStorageHandler.java
new file mode 100644
index 0000000..8655edc
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/LocalStorageHandler.java
@@ -0,0 +1,150 @@
+/*
+ * 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.ozone.web.localstorage;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.ozone.StorageContainerConfiguration;
+import org.apache.hadoop.ozone.web.exceptions.OzoneException;
+import org.apache.hadoop.ozone.web.handlers.UserArgs;
+import org.apache.hadoop.ozone.web.handlers.VolumeArgs;
+import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
+import org.apache.hadoop.ozone.web.response.ListVolumes;
+import org.apache.hadoop.ozone.web.response.VolumeInfo;
+
+import java.io.IOException;
+
+
+/**
+ * PLEASE NOTE : This file is a dummy backend for test purposes
+ * and prototyping effort only. It does not handle any Object semantics
+ * correctly, neither does it take care of security.
+ */
+@InterfaceAudience.Private
+public class LocalStorageHandler implements StorageHandler {
+  private String storageRoot = null;
+
+  /**
+   * Constructs LocalStorageHandler.
+   */
+  public LocalStorageHandler() {
+    StorageContainerConfiguration conf = new StorageContainerConfiguration();
+    storageRoot = conf.getTrimmed(
+        OzoneConfigKeys.DFS_STORAGE_LOCAL_ROOT,
+        OzoneConfigKeys.DFS_STORAGE_LOCAL_ROOT_DEFAULT);
+  }
+
+  /**
+   * Creates Storage Volume.
+   *
+   * @param args - volumeArgs
+   *
+   * @throws IOException
+   */
+  @Override
+  public void createVolume(VolumeArgs args) throws IOException, OzoneException {
+  }
+
+  /**
+   * setVolumeOwner - sets the owner of the volume.
+   *
+   * @param args volumeArgs
+   *
+   * @throws IOException
+   */
+  @Override
+  public void setVolumeOwner(VolumeArgs args)
+      throws IOException, OzoneException {
+  }
+
+  /**
+   * Set Volume Quota Info.
+   *
+   * @param args - volumeArgs
+   * @param remove - true if the request is to remove the quota
+   *
+   * @throws IOException
+   */
+  @Override
+  public void setVolumeQuota(VolumeArgs args, boolean remove)
+      throws IOException, OzoneException {
+  }
+
+
+  /**
+   * Checks if a Volume exists and the user specified has access to the
+   * volume.
+   *
+   * @param args - volumeArgs
+   *
+   * @return - Boolean - True if the user can modify the volume.
+   * This is possible for owners of the volume and admin users
+   *
+   * @throws FileSystemException
+   */
+  @Override
+  public boolean checkVolumeAccess(VolumeArgs args)
+      throws IOException, OzoneException {
+    return true;
+  }
+
+
+  /**
+   * Returns Info about the specified Volume.
+   *
+   * @param args - volumeArgs
+   *
+   * @return VolumeInfo
+   *
+   * @throws IOException
+   */
+  @Override
+  public VolumeInfo getVolumeInfo(VolumeArgs args)
+      throws IOException, OzoneException {
+    return null;
+  }
+
+
+  /**
+   * Deletes an Empty Volume.
+   *
+   * @param args - Volume Args
+   *
+   * @throws IOException
+   */
+  @Override
+  public void deleteVolume(VolumeArgs args) throws IOException, OzoneException {
+  }
+
+  /**
+   * Returns the List of Volumes owned by the specific user.
+   *
+   * @param args - UserArgs
+   *
+   * @return - List of Volumes
+   *
+   * @throws IOException
+   */
+  @Override
+  public ListVolumes listVolumes(UserArgs args)
+      throws IOException, OzoneException {
+    return null;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae655ea0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/request/OzoneQuota.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/request/OzoneQuota.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/request/OzoneQuota.java
index f0f6b98..ca6ddeb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/request/OzoneQuota.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/request/OzoneQuota.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.ozone.web.request;
 
 
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.ozone.web.headers.Header;
 import org.codehaus.jackson.annotate.JsonIgnore;
 
@@ -26,6 +27,7 @@ import org.codehaus.jackson.annotate.JsonIgnore;
  * represents an OzoneQuota Object that can be applied to
  * a storage volume.
  */
+@InterfaceAudience.Private
 public class OzoneQuota {
   private static final long MB_IN_BYTES = 1048576L;
   private static final long GB_IN_BYTES = 1073741824L;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae655ea0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/response/ListVolumes.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/response/ListVolumes.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/response/ListVolumes.java
index 9c2ec9f..89cfebd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/response/ListVolumes.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/response/ListVolumes.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.ozone.web.response;
 
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.codehaus.jackson.annotate.JsonAutoDetect;
 import org.codehaus.jackson.annotate.JsonMethod;
 import org.codehaus.jackson.map.ObjectMapper;
@@ -35,6 +36,7 @@ import java.util.List;
  * List Volume Class is the class that is returned in JSON format to
  * users when they call ListVolumes.
  */
+@InterfaceAudience.Private
 public class ListVolumes {
   private List<VolumeInfo> volumes;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae655ea0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/response/VolumeInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/response/VolumeInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/response/VolumeInfo.java
index e3ef533..5717368 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/response/VolumeInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/response/VolumeInfo.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.ozone.web.response;
 
 
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.ozone.web.request.OzoneQuota;
 import org.codehaus.jackson.annotate.JsonAutoDetect;
 import org.codehaus.jackson.annotate.JsonMethod;
@@ -35,6 +36,7 @@ import java.io.IOException;
  * VolumeInfo Class is the Java class that represents
  * Json when VolumeInfo Call is made.
  */
+@InterfaceAudience.Private
 public class VolumeInfo implements Comparable<VolumeInfo> {
 
   static final String VOLUME_INFO = "VOLUME_INFO_FILTER";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae655ea0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/response/VolumeOwner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/response/VolumeOwner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/response/VolumeOwner.java
index 02c4d04..1dadf79 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/response/VolumeOwner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/response/VolumeOwner.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.ozone.web.response;
 
 
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
 /**
@@ -27,6 +28,7 @@ import org.codehaus.jackson.map.annotate.JsonSerialize;
  * This is a class instead of a string since we might need to extend this class
  * to support other forms of authentication.
  */
+@InterfaceAudience.Private
 public class VolumeOwner {
   @JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
   private String name;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae655ea0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/userauth/Simple.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/userauth/Simple.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/userauth/Simple.java
index 1c41cee..85b67fb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/userauth/Simple.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/userauth/Simple.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.ozone.web.userauth;
 
 
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.ozone.web.exceptions.ErrorTable;
 import org.apache.hadoop.ozone.web.exceptions.OzoneException;
 import org.apache.hadoop.ozone.web.handlers.UserArgs;
@@ -33,6 +34,7 @@ import java.util.List;
  * mode of ozone. This maps more or less to the simple user scheme in
  * HDFS.
  */
+@InterfaceAudience.Private
 public class Simple implements UserAuth {
   /**
    * Returns the x-ozone-user or the user on behalf of, This is

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae655ea0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/utils/OzoneConsts.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/utils/OzoneConsts.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/utils/OzoneConsts.java
index f54e593..56c814e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/utils/OzoneConsts.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/utils/OzoneConsts.java
@@ -18,13 +18,22 @@
 
 package org.apache.hadoop.ozone.web.utils;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+
 /**
  * Set of constants used in Ozone implementation.
  */
+@InterfaceAudience.Private
 public final class OzoneConsts {
   public static final String OZONE_SIMPLE_ROOT_USER = "root";
   public static final String OZONE_SIMPLE_HDFS_USER = "hdfs";
 
+  /*
+   * BucketName length is used for both buckets and volume lengths
+   */
+  public static final int OZONE_MIN_BUCKET_NAME_LENGTH = 3;
+  public static final int OZONE_MAX_BUCKET_NAME_LENGTH = 63;
+
   private OzoneConsts() {
     // Never Constructed
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae655ea0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/utils/OzoneUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/utils/OzoneUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/utils/OzoneUtils.java
new file mode 100644
index 0000000..c454cc4
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/utils/OzoneUtils.java
@@ -0,0 +1,268 @@
+/*
+ * 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.ozone.web.utils;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.ozone.web.exceptions.ErrorTable;
+import org.apache.hadoop.ozone.web.exceptions.OzoneException;
+import org.apache.hadoop.ozone.web.handlers.UserArgs;
+import org.apache.hadoop.ozone.web.headers.Header;
+import org.apache.hadoop.util.Time;
+
+import javax.ws.rs.core.HttpHeaders;
+import javax.ws.rs.core.Request;
+import javax.ws.rs.core.Response;
+import java.io.InputStream;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.List;
+import java.util.Locale;
+import java.util.TimeZone;
+import java.util.UUID;
+
+/**
+ * Set of Utility functions used in ozone.
+ */
+@InterfaceAudience.Private
+public final class OzoneUtils {
+
+  /**
+   * verifies that bucket name / volume name is a valid DNS name.
+   *
+   * @param bucketName Bucket Name to be validated
+   *
+   * @throws IllegalArgumentException
+   */
+  public static void verifyBucketName(String bucketName)
+      throws IllegalArgumentException {
+
+    if (bucketName == null) {
+      throw new IllegalArgumentException("Bucket or Volume name is null");
+    }
+
+    if ((bucketName.length() < OzoneConsts.OZONE_MIN_BUCKET_NAME_LENGTH) ||
+        (bucketName.length() > OzoneConsts.OZONE_MAX_BUCKET_NAME_LENGTH)) {
+      throw new IllegalArgumentException(
+          "Bucket or Volume length is illegal, " +
+              "valid length is 3-63 characters");
+    }
+
+    if ((bucketName.charAt(0) == '.') || (bucketName.charAt(0) == '-')) {
+      throw new IllegalArgumentException(
+          "Bucket or Volume name cannot start with a period or dash");
+    }
+
+    if ((bucketName.charAt(bucketName.length() - 1) == '.') ||
+        (bucketName.charAt(bucketName.length() - 1) == '-')) {
+      throw new IllegalArgumentException(
+          "Bucket or Volume name cannot end with a period or dash");
+    }
+
+    boolean isIPv4 = true;
+    char prev = (char) 0;
+
+    for (int index = 0; index < bucketName.length(); index++) {
+      char currChar = bucketName.charAt(index);
+
+      if (currChar != '.') {
+        isIPv4 = ((currChar >= '0') && (currChar <= '9')) && isIPv4;
+      }
+
+      if (currChar > 'A' && currChar < 'Z') {
+        throw new IllegalArgumentException(
+            "Bucket or Volume name does not support uppercase characters");
+      }
+
+
+      if ((currChar != '.') && (currChar != '-')) {
+        if ((currChar < '0') || (currChar > '9' && currChar < 'a') ||
+            (currChar > 'z')) {
+          throw new IllegalArgumentException("Bucket or Volume name has an " +
+                                                 "unsupported character : " +
+                                                 currChar);
+        }
+      }
+
+      if ((prev == '.') && (currChar == '.')) {
+        throw new IllegalArgumentException("Bucket or Volume name should not " +
+                                               "have two contiguous periods");
+      }
+
+      if ((prev == '-') && (currChar == '.')) {
+        throw new IllegalArgumentException(
+            "Bucket or Volume name should not have period after dash");
+      }
+
+      if ((prev == '.') && (currChar == '-')) {
+        throw new IllegalArgumentException(
+            "Bucket or Volume name should not have dash after period");
+      }
+      prev = currChar;
+    }
+
+    if (isIPv4) {
+      throw new IllegalArgumentException(
+          "Bucket or Volume name cannot be an IPv4 address or all numeric");
+    }
+  }
+
+  /**
+   * Returns a random Request ID.
+   *
+   * Request ID is returned to the client as well as flows thru the system
+   * facilitating debugging on why a certain request failed.
+   *
+   * @return String random request ID
+   */
+  public static String getRequestID() {
+    return UUID.randomUUID().toString();
+  }
+
+  /**
+   * Return host name if possible.
+   *
+   * @return Host Name or localhost
+   */
+  public static String getHostName() {
+    String host = "localhost";
+    try {
+      host = InetAddress.getLocalHost().getHostName();
+    } catch (UnknownHostException e) {
+      // Ignore the error
+    }
+    return host;
+  }
+
+  /**
+   * Basic validate routine to make sure that all the
+   * required headers are in place.
+   *
+   * @param request - http request
+   * @param headers - http headers
+   * @param reqId - request id
+   * @param resource - Resource Name
+   * @param hostname - Hostname
+   *
+   * @throws OzoneException
+   */
+  public static void validate(Request request, HttpHeaders headers,
+                              String reqId, String resource, String hostname)
+      throws OzoneException {
+
+    List<String> ozHeader =
+        headers.getRequestHeader(Header.OZONE_VERSION_HEADER);
+    if (ozHeader == null) {
+      throw ErrorTable
+          .newError(ErrorTable.MISSING_VERSION, reqId, resource, hostname);
+    }
+
+    List<String> date = headers.getRequestHeader(HttpHeaders.DATE);
+    if (date == null) {
+      throw ErrorTable
+          .newError(ErrorTable.MISSING_DATE, reqId, resource, hostname);
+    }
+
+    /*
+    TODO :
+    Ignore the results for time being. Eventually we can validate if the
+    request Date time is too skewed and reject if it is so.
+    */
+    parseDate(date.get(0), reqId, resource, hostname);
+
+  }
+
+  /**
+   * Parses the Date String coming from the Users.
+   *
+   * @param dateString - Date String
+   * @param reqID - Ozone Request ID
+   * @param resource - Resource Name
+   * @param hostname - HostName
+   *
+   * @return - Date
+   *
+   * @throws OzoneException - in case of parsing error
+   */
+  public static synchronized Date parseDate(String dateString, String reqID,
+                                            String resource, String hostname)
+      throws OzoneException {
+    SimpleDateFormat format =
+        new SimpleDateFormat("EEE, dd MMM yyyy HH:mm:ss zzz", Locale.US);
+    format.setTimeZone(TimeZone.getTimeZone("GMT"));
+
+    try {
+      return format.parse(dateString);
+    } catch (ParseException ex) {
+      OzoneException exp =
+          ErrorTable.newError(ErrorTable.BAD_DATE, reqID, resource, hostname);
+      exp.setMessage(ex.getMessage());
+      throw exp;
+    }
+  }
+
+  /**
+   * Returns a response with appropriate OZONE headers and payload.
+   *
+   * @param args - UserArgs or Inherited class
+   * @param statusCode - HttpStatus code
+   * @param payload - Content Body
+   *
+   * @return JAX-RS Response
+   */
+  public static Response getResponse(UserArgs args, int statusCode,
+                                     String payload) {
+    SimpleDateFormat format =
+        new SimpleDateFormat("EEE, dd MMM yyyy HH:mm:ss zzz", Locale.US);
+    format.setTimeZone(TimeZone.getTimeZone("GMT"));
+    String date = format.format(new Date(Time.monotonicNow()));
+    return Response.ok(payload)
+        .header(Header.OZONE_SERVER_NAME, args.getHostName())
+        .header(Header.OZONE_REQUEST_ID, args.getRequestID())
+        .header(HttpHeaders.DATE, date).status(statusCode).build();
+  }
+
+  /**
+   * Returns a response with appropriate OZONE headers and payload.
+   *
+   * @param args - UserArgs or Inherited class
+   * @param statusCode - HttpStatus code
+   * @param stream InputStream
+   *
+   * @return JAX-RS Response
+   */
+  public static Response getResponse(UserArgs args, int statusCode,
+                                     InputStream stream) {
+    SimpleDateFormat format =
+        new SimpleDateFormat("EEE, dd MMM yyyy HH:mm:ss zzz", Locale.US);
+    format.setTimeZone(TimeZone.getTimeZone("GMT"));
+    String date = format.format(new Date(Time.monotonicNow()));
+    return Response.ok(stream)
+        .header(Header.OZONE_SERVER_NAME, args.getHostName())
+        .header(Header.OZONE_REQUEST_ID, args.getRequestID())
+        .header(HttpHeaders.DATE, date).status(statusCode)
+        .header(HttpHeaders.CONTENT_TYPE, "application/octet-stream").build();
+  }
+
+  private OzoneUtils() {
+    // Never constructed
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae655ea0/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestErrorCode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestErrorCode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestErrorCode.java
new file mode 100644
index 0000000..6639e48
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestErrorCode.java
@@ -0,0 +1,50 @@
+/*
+ * 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.ozone.web;
+
+import org.apache.hadoop.ozone.web.exceptions.ErrorTable;
+import org.apache.hadoop.ozone.web.exceptions.OzoneException;
+import org.junit.Test;
+
+import static junit.framework.TestCase.assertEquals;
+import static org.apache.hadoop.ozone.web.utils.OzoneUtils.getRequestID;
+
+public class TestErrorCode {
+  /**
+   * Test Error Generator functions.
+   */
+  @Test
+  public void TestErrorGen() {
+    OzoneException e = ErrorTable
+        .newError(ErrorTable.ACCESS_DENIED, getRequestID(), "/test/path",
+                  "localhost");
+    assertEquals(e.getHostID(), "localhost");
+    assertEquals(e.getShortMessage(),
+                 ErrorTable.ACCESS_DENIED.getShortMessage());
+  }
+
+  @Test
+  public void TestErrorGenWithException() {
+    OzoneException e =
+        new OzoneException(ErrorTable.ACCESS_DENIED.getHttpCode(),
+                           "short message", new Exception("Hello"));
+    assertEquals("short message", e.getShortMessage());
+    assertEquals("Hello", e.getMessage());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae655ea0/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestUtils.java
new file mode 100644
index 0000000..f63e369
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestUtils.java
@@ -0,0 +1,100 @@
+/*
+ * 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.ozone.web;
+
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.web.utils.OzoneUtils.getRequestID;
+import static org.apache.hadoop.ozone.web.utils.OzoneUtils.verifyBucketName;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class TestUtils {
+
+  /**
+   * Tests if the bucket name handling is correct.
+   */
+  @Test
+  public  void TestValidBucketNames() {
+    HashMap<String, Boolean> testMatrix;
+    // Init the Table with Strings and Expected Return values
+    testMatrix = new HashMap<String, Boolean>();
+
+    testMatrix.put("bucket-.ozone.self", Boolean.FALSE);
+    testMatrix.put("bucket.-ozone.self", Boolean.FALSE);
+    testMatrix.put(".bucket.ozone.self", Boolean.FALSE);
+    testMatrix.put("bucket.ozone.self.", Boolean.FALSE);
+    testMatrix.put("bucket..ozone.self", Boolean.FALSE);
+    testMatrix.put("192.1.1.1", Boolean.FALSE);
+    testMatrix.put("ab", Boolean.FALSE);
+    testMatrix.put("bucket.ozone.self.this.is.a.really.long.name.that." +
+        "is.more.than.sixty.three.characters.long.for.sure",
+      Boolean.FALSE);
+    testMatrix.put(null, Boolean.FALSE);
+    testMatrix.put("bucket@$", Boolean.FALSE);
+    testMatrix.put("BUCKET", Boolean.FALSE);
+    testMatrix.put("bucket .ozone.self", Boolean.FALSE);
+    testMatrix.put("       bucket.ozone.self", Boolean.FALSE);
+    testMatrix.put("bucket.ozone.self-", Boolean.FALSE);
+    testMatrix.put("-bucket.ozone.self", Boolean.FALSE);
+
+    testMatrix.put("bucket", Boolean.TRUE);
+    testMatrix.put("bucket.ozone.self", Boolean.TRUE);
+    testMatrix.put("bucket.ozone.self", Boolean.TRUE);
+    testMatrix.put("bucket-name.ozone.self", Boolean.TRUE);
+    testMatrix.put("bucket.1.ozone.self", Boolean.TRUE);
+
+    Set<String> keys = testMatrix.keySet();
+    for (String key : keys) {
+      if(testMatrix.get(key)) {
+
+        // For valid names there should be no exceptions at all
+        verifyBucketName(key);
+      } else {
+        try {
+          verifyBucketName(key);
+          // should never get here since the isValid call will throw
+         fail("An exception was expected but did not happen.");
+        } catch(IllegalArgumentException e){
+
+        }
+      }
+    }
+  }
+
+  /**
+   *  Just calls Request ID many times and assert we
+   *  got different values, ideally this should be
+   *  run under parallel threads. Since the function under
+   *  test has no external dependencies it is assumed
+   *  that this test is good enough.
+   */
+  @Test
+  public void TestRequestIDisRandom(){
+    HashSet<String> set = new HashSet<>();
+    for (int i = 0; i < 1000; i ++){
+      assertTrue(set.add(getRequestID()));
+    }
+  }
+
+}