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 cn...@apache.org on 2016/03/03 01:34:06 UTC

hadoop git commit: HDFS-9848. Ozone: Add Ozone Client lib for volume handling. Contributed by Anu Engineer.

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-7240 f3fbae8b8 -> 1244d8ff5


HDFS-9848. Ozone: Add Ozone Client lib for volume handling. 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/1244d8ff
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1244d8ff
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1244d8ff

Branch: refs/heads/HDFS-7240
Commit: 1244d8ff5c3342322bc4b287f0763746cc5e3dc5
Parents: f3fbae8
Author: Chris Nauroth <cn...@apache.org>
Authored: Wed Mar 2 16:31:09 2016 -0800
Committer: Chris Nauroth <cn...@apache.org>
Committed: Wed Mar 2 16:31:09 2016 -0800

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-7240.txt           |   3 +
 .../server/datanode/ObjectStoreHandler.java     |  18 +-
 .../apache/hadoop/ozone/OzoneConfigKeys.java    |   4 +
 .../ozone/web/ObjectStoreApplication.java       |   2 +
 .../hadoop/ozone/web/client/OzoneClient.java    | 544 +++++++++++++++++++
 .../ozone/web/client/OzoneClientException.java  |  44 ++
 .../hadoop/ozone/web/client/OzoneVolume.java    | 122 +++++
 .../hadoop/ozone/web/client/package-info.java   |  34 ++
 .../ozone/web/exceptions/OzoneException.java    |  10 +
 .../web/handlers/BucketProcessTemplate.java     |   4 +-
 .../web/handlers/StorageHandlerBuilder.java     |   5 +-
 .../apache/hadoop/ozone/web/headers/Header.java |   6 +-
 .../web/localstorage/LocalStorageHandler.java   |  47 +-
 .../web/localstorage/OzoneMetadataManager.java  |  11 +-
 .../web/messages/StringMessageBodyWriter.java   |  62 +++
 .../hadoop/ozone/web/utils/OzoneUtils.java      |   3 +
 .../hadoop/ozone/web/TestOzoneVolumes.java      |   9 +-
 .../hadoop/ozone/web/TestOzoneWebAccess.java    |   6 +-
 .../hadoop/ozone/web/client/TestVolume.java     | 162 ++++++
 19 files changed, 1061 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1244d8ff/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 83cfef8..667bc93 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-7240.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-7240.txt
@@ -44,3 +44,6 @@
 
     HDFS-9845. OzoneHandler : Support List and Info Volumes.
     (Anu Engineer via cnauroth)
+
+    HDFS-9848. Ozone: Add Ozone Client lib for volume handling.
+    (Anu Engineer via cnauroth)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1244d8ff/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ObjectStoreHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ObjectStoreHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ObjectStoreHandler.java
index 5b751b3..6413ac0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ObjectStoreHandler.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ObjectStoreHandler.java
@@ -18,11 +18,15 @@
 package org.apache.hadoop.hdfs.server.datanode;
 
 import static com.sun.jersey.api.core.ResourceConfig.PROPERTY_CONTAINER_REQUEST_FILTERS;
+import static com.sun.jersey.api.core.ResourceConfig.FEATURE_TRACE;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.DFS_STORAGE_HANDLER_TYPE_DEFAULT;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.DFS_STORAGE_HANDLER_TYPE_KEY;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.DFS_OBJECTSTORE_TRACE_ENABLED_KEY;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.DFS_OBJECTSTORE_TRACE_ENABLED_DEFAULT;
 
 import java.io.IOException;
-import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
 
 import com.sun.jersey.api.container.ContainerFactory;
 import com.sun.jersey.api.core.ApplicationAdapter;
@@ -53,12 +57,14 @@ public final class ObjectStoreHandler {
   public ObjectStoreHandler(Configuration conf) throws IOException {
     String shType = conf.getTrimmed(DFS_STORAGE_HANDLER_TYPE_KEY,
         DFS_STORAGE_HANDLER_TYPE_DEFAULT);
+    boolean ozoneTrace = conf.getBoolean(DFS_OBJECTSTORE_TRACE_ENABLED_KEY,
+        DFS_OBJECTSTORE_TRACE_ENABLED_DEFAULT);
     final StorageHandler storageHandler;
     if ("distributed".equalsIgnoreCase(shType)) {
       storageHandler = new DistributedStorageHandler();
     } else {
       if ("local".equalsIgnoreCase(shType)) {
-        storageHandler = new LocalStorageHandler();
+        storageHandler = new LocalStorageHandler(conf);
       } else {
         throw new IllegalArgumentException(
             String.format("Unrecognized value for %s: %s",
@@ -67,9 +73,11 @@ public final class ObjectStoreHandler {
     }
     ApplicationAdapter aa =
         new ApplicationAdapter(new ObjectStoreApplication());
-    aa.setPropertiesAndFeatures(Collections.<String, Object>singletonMap(
-        PROPERTY_CONTAINER_REQUEST_FILTERS,
-        ServiceFilter.class.getCanonicalName()));
+    Map<String, Object> settingsMap = new HashMap<>();
+    settingsMap.put(PROPERTY_CONTAINER_REQUEST_FILTERS,
+        ServiceFilter.class.getCanonicalName());
+    settingsMap.put(FEATURE_TRACE, ozoneTrace);
+    aa.setPropertiesAndFeatures(settingsMap);
     this.objectStoreJerseyContainer = ContainerFactory.createContainer(
         ObjectStoreJerseyContainer.class, aa);
     this.objectStoreJerseyContainer.setStorageHandler(storageHandler);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1244d8ff/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
index ee940b6..7d9cdd5 100644
--- 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
@@ -34,6 +34,10 @@ public final class OzoneConfigKeys {
   public static final String DFS_STORAGE_HANDLER_TYPE_KEY =
       "dfs.storage.handler.type";
   public static final String DFS_STORAGE_HANDLER_TYPE_DEFAULT = "distributed";
+  public static final String DFS_OBJECTSTORE_TRACE_ENABLED_KEY =
+      "dfs.objectstore.trace.enabled";
+  public static final boolean DFS_OBJECTSTORE_TRACE_ENABLED_DEFAULT = false;
+
 
   /**
    * There is no need to instantiate this class.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1244d8ff/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ObjectStoreApplication.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ObjectStoreApplication.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ObjectStoreApplication.java
index 900b98c..0c63c11 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ObjectStoreApplication.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ObjectStoreApplication.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.ozone.web.exceptions.OzoneExceptionMapper;
 import org.apache.hadoop.ozone.web.handlers.BucketHandler;
 import org.apache.hadoop.ozone.web.handlers.ServiceFilter;
 import org.apache.hadoop.ozone.web.handlers.VolumeHandler;
+import org.apache.hadoop.ozone.web.messages.StringMessageBodyWriter;
 
 import javax.ws.rs.core.Application;
 import java.util.HashSet;
@@ -41,6 +42,7 @@ public class ObjectStoreApplication extends Application {
     set.add(BucketHandler.class);
     set.add(VolumeHandler.class);
     set.add(OzoneExceptionMapper.class);
+    set.add(StringMessageBodyWriter.class);
     return set;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1244d8ff/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneClient.java
new file mode 100644
index 0000000..e133e2a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneClient.java
@@ -0,0 +1,544 @@
+/*
+ * 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.client;
+
+import org.apache.hadoop.ozone.web.exceptions.OzoneException;
+import org.apache.hadoop.ozone.web.headers.Header;
+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 org.apache.hadoop.util.Time;
+import org.apache.http.HttpEntity;
+import org.apache.http.HttpResponse;
+import org.apache.http.client.methods.HttpDelete;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.client.methods.HttpPost;
+import org.apache.http.client.methods.HttpPut;
+import org.apache.http.client.methods.HttpRequestBase;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.http.impl.client.DefaultHttpClient;
+import org.apache.http.util.EntityUtils;
+
+import javax.ws.rs.core.HttpHeaders;
+import java.io.Closeable;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Locale;
+
+import static java.net.HttpURLConnection.HTTP_CREATED;
+import static java.net.HttpURLConnection.HTTP_OK;
+
+/**
+ * Ozone client that connects to an Ozone server. Please note that this class is
+ * not  thread safe.
+ */
+public class OzoneClient implements Closeable {
+  private URI endPointURI;
+  private String userAuth;
+
+  /**
+   * Constructor for OzoneClient.
+   */
+  public OzoneClient() {
+  }
+
+  /**
+   * Constructor for OzoneClient.
+   */
+  public OzoneClient(String ozoneURI)
+      throws OzoneException, URISyntaxException {
+    setEndPoint(ozoneURI);
+  }
+
+  /**
+   * Constructor for OzoneClient.
+   */
+  public OzoneClient(String ozoneURI, String userAuth)
+      throws OzoneException, URISyntaxException {
+    setEndPoint(ozoneURI);
+    setUserAuth(userAuth);
+  }
+
+  /**
+   * Returns the end Point.
+   *
+   * @return String
+   */
+  public URI getEndPointURI() {
+    return endPointURI;
+
+  }
+
+  /**
+   * Sets the End Point info using an URI.
+   *
+   * @param endPointURI - URI
+   * @throws OzoneException
+   */
+  public void setEndPointURI(URI endPointURI) throws OzoneException {
+    if ((endPointURI == null) || (endPointURI.toString().isEmpty())) {
+      throw new OzoneClientException("Invalid ozone URI");
+    }
+    this.endPointURI = endPointURI;
+  }
+
+  /**
+   * Set endPoint.
+   *
+   * @param clusterFQDN - cluster FQDN.
+   */
+  public void setEndPoint(String clusterFQDN) throws
+      OzoneException, URISyntaxException {
+    setEndPointURI(new URI(clusterFQDN));
+  }
+
+  /**
+   * Get user Auth String.
+   *
+   * @return - User Auth String
+   */
+  public String getUserAuth() {
+    return this.userAuth;
+  }
+
+  /**
+   * Set User Auth.
+   *
+   * @param userAuth - User Auth String
+   */
+  public void setUserAuth(String userAuth) {
+    this.userAuth = userAuth;
+  }
+
+  /**
+   * create volume.
+   *
+   * @param volumeName - volume name 3 - 63 chars, small letters.
+   * @param onBehalfOf - The user on behalf we are making the call for
+   * @param quota      - Quota's are specified in a specific format. it is
+   *                   integer(MB|GB|TB), for example 100TB.
+   * @throws OzoneClientException
+   */
+  public OzoneVolume createVolume(String volumeName, String onBehalfOf,
+                                  String quota) throws OzoneException {
+    try {
+      DefaultHttpClient httpClient = new DefaultHttpClient();
+      OzoneUtils.verifyBucketName(volumeName);
+
+      URIBuilder builder = new URIBuilder(endPointURI);
+      builder.setPath("/" + volumeName);
+      if (quota != null) {
+        builder.setParameter(Header.OZONE_QUOTA_QUERY_TAG, quota);
+      }
+
+      HttpPost httppost = getHttpPost(onBehalfOf, builder.build().toString());
+      executeCreateVolume(httppost, httpClient);
+      return getVolume(volumeName);
+    } catch (IOException | URISyntaxException | IllegalArgumentException ex) {
+      throw new OzoneClientException(ex.getMessage());
+    }
+  }
+
+  /**
+   * Returns information about an existing Volume. if the Volume does not exist,
+   * or if the user does not have access rights OzoneException is thrown
+   *
+   * @param volumeName - volume name 3 - 63 chars, small letters.
+   * @return OzoneVolume Ozone Client Volume Class.
+   * @throws OzoneException
+   */
+  public OzoneVolume getVolume(String volumeName) throws OzoneException {
+    try {
+      DefaultHttpClient httpClient = new DefaultHttpClient();
+
+      OzoneUtils.verifyBucketName(volumeName);
+      URIBuilder builder = new URIBuilder(endPointURI);
+      builder.setPath("/" + volumeName)
+          .setParameter(Header.OZONE_LIST_QUERY_TAG,
+              Header.OZONE_LIST_QUERY_VOLUME)
+          .build();
+
+      HttpGet httpget = getHttpGet(builder.toString());
+      return executeInfoVolume(httpget, httpClient);
+
+    } catch (IOException | URISyntaxException | IllegalArgumentException ex) {
+      throw new OzoneClientException(ex.getMessage());
+    }
+  }
+
+  /**
+   * List all the volumes owned by the user or Owned by the user specified in
+   * the behalf of string.
+   *
+   * @param onBehalfOf - User Name of the user if it is not the caller. for
+   *                   example, an admin wants to list some other users
+   *                   volumes.
+   * @return List of Volumes
+   * @throws OzoneException
+   */
+  public List<OzoneVolume> listVolumes(String onBehalfOf)
+      throws OzoneException {
+    try {
+      DefaultHttpClient httpClient = new DefaultHttpClient();
+
+      URIBuilder builder = new URIBuilder(endPointURI);
+      builder.setPath("/").build();
+
+      HttpGet httpget = getHttpGet(builder.toString());
+      if (onBehalfOf != null) {
+        httpget.addHeader(Header.OZONE_USER, onBehalfOf);
+      }
+      return executeListVolume(httpget, httpClient);
+
+    } catch (IOException | URISyntaxException ex) {
+      throw new OzoneClientException(ex.getMessage());
+    }
+  }
+
+  /**
+   * delete a given volume.
+   *
+   * @param volumeName - volume to be deleted.
+   * @throws OzoneException - Ozone Exception
+   */
+  public void deleteVolume(String volumeName) throws OzoneException {
+    try {
+      DefaultHttpClient httpClient = new DefaultHttpClient();
+
+      OzoneUtils.verifyBucketName(volumeName);
+      URIBuilder builder = new URIBuilder(endPointURI);
+      builder.setPath("/" + volumeName).build();
+
+      HttpDelete httpdelete = getHttpDelete(builder.toString());
+      executeDeleteVolume(httpdelete, httpClient);
+    } catch (IOException | URISyntaxException | IllegalArgumentException ex) {
+      throw new OzoneClientException(ex.getMessage());
+    }
+  }
+
+  /**
+   * Sets the Volume Owner.
+   *
+   * @param volumeName - Volume Name
+   * @param newOwner   - New Owner Name
+   * @throws OzoneException
+   */
+  public void setVolumeOwner(String volumeName, String newOwner)
+      throws OzoneException {
+
+    if (newOwner == null || newOwner.isEmpty()) {
+      throw new OzoneClientException("Invalid new owner name");
+    }
+    try {
+      DefaultHttpClient httpClient = new DefaultHttpClient();
+
+      OzoneUtils.verifyBucketName(volumeName);
+      URIBuilder builder = new URIBuilder(endPointURI);
+      builder.setPath("/" + volumeName).build();
+
+      HttpPut putRequest = getHttpPut(builder.toString());
+      putRequest.addHeader(Header.OZONE_USER, newOwner);
+      executePutVolume(putRequest, httpClient);
+
+    } catch (URISyntaxException | IllegalArgumentException | IOException ex) {
+      throw new OzoneClientException(ex.getMessage());
+    }
+  }
+
+  /**
+   * Sets the Volume Quota. Quota's are specified in a specific format. it is
+   * <integer>|(MB|GB|TB. for example 100TB.
+   * <p>
+   * To Remove a quota you can specify Header.OZONE_QUOTA_REMOVE
+   *
+   * @param volumeName - volume name
+   * @param quota      - Quota String or  Header.OZONE_QUOTA_REMOVE
+   * @throws OzoneException
+   */
+  public void setVolumeQuota(String volumeName, String quota)
+      throws OzoneException {
+    if (quota == null || quota.isEmpty()) {
+      throw new OzoneClientException("Invalid quota");
+    }
+    try {
+      DefaultHttpClient httpClient = new DefaultHttpClient();
+
+      OzoneUtils.verifyBucketName(volumeName);
+      URIBuilder builder = new URIBuilder(endPointURI);
+      builder.setPath("/" + volumeName)
+          .setParameter(Header.OZONE_QUOTA_QUERY_TAG, quota)
+          .build();
+
+      HttpPut putRequest = getHttpPut(builder.toString());
+      executePutVolume(putRequest, httpClient);
+
+    } catch (URISyntaxException | IllegalArgumentException | IOException ex) {
+      throw new OzoneClientException(ex.getMessage());
+    }
+  }
+
+  /**
+   * Sends the create Volume request to the server.
+   *
+   * @param httppost   - http post class
+   * @param httpClient - httpClient
+   * @throws IOException    -
+   * @throws OzoneException
+   */
+  private void executeCreateVolume(HttpPost httppost,
+                                   DefaultHttpClient httpClient)
+      throws IOException, OzoneException {
+    HttpEntity entity = null;
+    try {
+      HttpResponse response = httpClient.execute(httppost);
+      int errorCode = response.getStatusLine().getStatusCode();
+      entity = response.getEntity();
+
+      if ((errorCode == HTTP_OK) || (errorCode == HTTP_CREATED)) {
+        return;
+      }
+
+      if (entity != null) {
+        throw OzoneException.parse(EntityUtils.toString(entity));
+      } else {
+        throw new OzoneClientException("Unexpected null in http payload");
+      }
+    } finally {
+      if (entity != null) {
+        EntityUtils.consume(entity);
+      }
+    }
+  }
+
+  /**
+   * Sends the create Volume request to the server.
+   *
+   * @param httpGet - httpGet
+   * @return OzoneVolume
+   * @throws IOException    -
+   * @throws OzoneException
+   */
+  private OzoneVolume executeInfoVolume(HttpGet httpGet,
+                                        DefaultHttpClient httpClient)
+      throws IOException, OzoneException {
+    HttpEntity entity = null;
+    try {
+      HttpResponse response = httpClient.execute(httpGet);
+      int errorCode = response.getStatusLine().getStatusCode();
+
+      entity = response.getEntity();
+      if (entity == null) {
+        throw new OzoneClientException("Unexpected null in http payload");
+      }
+
+      if (errorCode == HTTP_OK) {
+        OzoneVolume volume = new OzoneVolume(this);
+        volume.setVolumeInfo(EntityUtils.toString(entity));
+        return volume;
+      } else {
+        throw OzoneException.parse(EntityUtils.toString(entity));
+      }
+    } finally {
+      if (entity != null) {
+        EntityUtils.consumeQuietly(entity);
+      }
+    }
+  }
+
+  /**
+   * Sends update volume requests to the server.
+   *
+   * @param putRequest http request
+   * @throws IOException
+   * @throws OzoneException
+   */
+  private void executePutVolume(HttpPut putRequest,
+                                DefaultHttpClient httpClient)
+      throws IOException, OzoneException {
+    HttpEntity entity = null;
+    try {
+      HttpResponse response = httpClient.execute(putRequest);
+      int errorCode = response.getStatusLine().getStatusCode();
+      entity = response.getEntity();
+      if (errorCode != HTTP_OK) {
+        throw OzoneException.parse(EntityUtils.toString(entity));
+      }
+    } finally {
+      if (entity != null) {
+        EntityUtils.consume(entity);
+      }
+    }
+  }
+
+  /**
+   * List Volumes.
+   *
+   * @param httpGet - httpGet
+   * @return OzoneVolume
+   * @throws IOException    -
+   * @throws OzoneException
+   */
+  private List<OzoneVolume> executeListVolume(HttpGet httpGet,
+                                              DefaultHttpClient httpClient)
+      throws IOException, OzoneException {
+    HttpEntity entity = null;
+    List<OzoneVolume> volList = new LinkedList<>();
+    try {
+      HttpResponse response = httpClient.execute(httpGet);
+      int errorCode = response.getStatusLine().getStatusCode();
+      entity = response.getEntity();
+
+      if (entity == null) {
+        throw new OzoneClientException("Unexpected null in http payload");
+      }
+
+      String temp = EntityUtils.toString(entity);
+      if (errorCode == HTTP_OK) {
+        ListVolumes listVolumes =
+            ListVolumes.parse(temp);
+
+        for (VolumeInfo info : listVolumes.getVolumes()) {
+          volList.add(new OzoneVolume(info, this));
+        }
+        return volList;
+
+      } else {
+        throw OzoneException.parse(EntityUtils.toString(entity));
+      }
+    } finally {
+      if (entity != null) {
+        EntityUtils.consumeQuietly(entity);
+      }
+    }
+  }
+
+  /**
+   * Delete Volume.
+   *
+   * @param httpDelete - Http Delete Request
+   * @throws IOException
+   * @throws OzoneException
+   */
+  private void executeDeleteVolume(HttpDelete httpDelete,
+                                   DefaultHttpClient httpClient)
+      throws IOException, OzoneException {
+    HttpEntity entity = null;
+    try {
+      HttpResponse response = httpClient.execute(httpDelete);
+      int errorCode = response.getStatusLine().getStatusCode();
+      entity = response.getEntity();
+
+      if (errorCode != HTTP_OK) {
+        throw OzoneException.parse(EntityUtils.toString(entity));
+      }
+    } finally {
+      if (entity != null) {
+        EntityUtils.consumeQuietly(entity);
+      }
+    }
+  }
+
+  /**
+   * Returns a standard HttpPost Object to use for ozone post requests.
+   *
+   * @param onBehalfOf - If the use is being made on behalf of user, that user
+   * @param uriString  - UriString
+   * @return HttpPost
+   */
+  public HttpPost getHttpPost(String onBehalfOf, String uriString) {
+    HttpPost httppost = new HttpPost(uriString);
+    addOzoneHeaders(httppost);
+    if (onBehalfOf != null) {
+      httppost.addHeader(Header.OZONE_USER, onBehalfOf);
+    }
+    return httppost;
+  }
+
+  /**
+   * Returns a standard HttpGet Object to use for ozone Get requests.
+   *
+   * @param uriString - The full Uri String
+   * @return HttpGet
+   */
+  public HttpGet getHttpGet(String uriString) {
+    HttpGet httpget = new HttpGet(uriString);
+    addOzoneHeaders(httpget);
+    return httpget;
+  }
+
+  /**
+   * Returns httpDelete.
+   *
+   * @param uriString - uri
+   * @return HttpDelete
+   */
+  public HttpDelete getHttpDelete(String uriString) {
+    HttpDelete httpDel = new HttpDelete(uriString);
+    addOzoneHeaders(httpDel);
+    return httpDel;
+  }
+
+  /**
+   * returns an HttpPut Object.
+   *
+   * @param uriString - Uri
+   * @return HttpPut
+   */
+  public HttpPut getHttpPut(String uriString) {
+    HttpPut httpPut = new HttpPut(uriString);
+    addOzoneHeaders(httpPut);
+    return httpPut;
+  }
+
+  /**
+   * Add Ozone Headers.
+   *
+   * @param httpRequest - Http Request
+   */
+  private void addOzoneHeaders(HttpRequestBase httpRequest) {
+    SimpleDateFormat format =
+        new SimpleDateFormat("EEE, dd MMM yyyy HH:mm:ss ZZZ", Locale.US);
+
+    httpRequest.addHeader(Header.OZONE_VERSION_HEADER,
+        Header.OZONE_V1_VERSION_HEADER);
+    httpRequest.addHeader(HttpHeaders.DATE,
+        format.format(new Date(Time.monotonicNow())));
+    if (getUserAuth() != null) {
+      httpRequest.addHeader(HttpHeaders.AUTHORIZATION,
+          Header.OZONE_SIMPLE_AUTHENTICATION_SCHEME + " " +
+              getUserAuth());
+    }
+  }
+
+  /**
+   * Closes this stream and releases any system resources associated with it. If
+   * the stream is already closed then invoking this method has no effect.
+   *
+   * @throws IOException if an I/O error occurs
+   */
+  @Override
+  public void close() throws IOException {
+    // TODO : Currently we create a new HTTP client. We should switch
+    // This to a Pool and cleanup the pool here.
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1244d8ff/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneClientException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneClientException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneClientException.java
new file mode 100644
index 0000000..854b47a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneClientException.java
@@ -0,0 +1,44 @@
+/*
+ * 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.client;
+
+import org.apache.hadoop.ozone.web.exceptions.OzoneException;
+
+/**
+ * This exception is thrown by the Ozone Clients.
+ */
+public class OzoneClientException extends OzoneException {
+  /**
+   * Constructor that allows the shortMessage.
+   *
+   * @param shortMessage Short Message
+   */
+  public OzoneClientException(String shortMessage) {
+    super(0, shortMessage, shortMessage);
+  }
+
+  /**
+   * Constructor that allows the shortMessage and a longer message.
+   *
+   * @param shortMessage Short Message
+   * @param message long error message
+   */
+  public OzoneClientException(String shortMessage, String message) {
+    super(0, shortMessage, message);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1244d8ff/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneVolume.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneVolume.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneVolume.java
new file mode 100644
index 0000000..46033cc
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneVolume.java
@@ -0,0 +1,122 @@
+/*
+ * 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.client;
+
+import org.apache.hadoop.ozone.web.request.OzoneQuota;
+import org.apache.hadoop.ozone.web.response.VolumeInfo;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Ozone Volume Class.
+ */
+public class OzoneVolume {
+  private VolumeInfo volumeInfo;
+  private Map<String, String> headerMap;
+  private final OzoneClient client;
+
+  /**
+   * Constructor for OzoneVolume.
+   */
+  public OzoneVolume(OzoneClient client) {
+    this.client = client;
+    this.headerMap = new HashMap<>();
+  }
+
+  /**
+   * Constructor for OzoneVolume.
+   *
+   * @param volInfo - volume Info.
+   * @param client  Client
+   */
+  public OzoneVolume(VolumeInfo volInfo, OzoneClient client) {
+    this.volumeInfo = volInfo;
+    this.client = client;
+  }
+
+  public String getJsonString() throws IOException {
+    return volumeInfo.toJsonString();
+  }
+
+  /**
+   * sets the Volume Info.
+   *
+   * @param volInfoString - Volume Info String
+   */
+  public void setVolumeInfo(String volInfoString) throws IOException {
+    this.volumeInfo = VolumeInfo.parse(volInfoString);
+  }
+
+  /**
+   * Returns volume Name.
+   *
+   * @return Volume Name.
+   */
+  public String getVolumeName() {
+    return this.volumeInfo.getVolumeName();
+  }
+
+  /**
+   * Get created by.
+   *
+   * @return String
+   */
+  public String getCreatedby() {
+    return this.volumeInfo.getCreatedBy();
+  }
+
+  /**
+   * returns the Owner name.
+   *
+   * @return String
+   */
+  public String getOwnerName() {
+    return this.volumeInfo.getOwner().getName();
+  }
+
+  /**
+   * Returns Quota Info.
+   *
+   * @return Quota
+   */
+  public OzoneQuota getQuota() {
+    return volumeInfo.getQuota();
+  }
+
+  /**
+   * Returns a Http header from the Last Volume related call.
+   *
+   * @param headerName - Name of the header
+   * @return - Header Value
+   */
+  public String getHeader(String headerName) {
+    return headerMap.get(headerName);
+  }
+
+  /**
+   * Gets the Client, this is used by Bucket and Key Classes.
+   *
+   * @return - Ozone Client
+   */
+  OzoneClient getClient() {
+    return client;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1244d8ff/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/package-info.java
new file mode 100644
index 0000000..046568b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/package-info.java
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+
+/**
+ * Ozone client library is a java client for the Ozone
+ * Object Store.
+ */
+package org.apache.hadoop.ozone.web.client;
+
+/**
+ This library is  a simple Ozone REST Library.
+
+ This library is a very *minimal* client written for tests and
+ command line utils that work against Ozone. It does not have
+ things like thread-pools and support for extended security models yet.
+
+ OzoneClients return OzoneVolumes and OzoneVolumes return OzoneBuckets.
+ **/

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1244d8ff/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 576966f..2bd4a69 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
@@ -46,6 +46,16 @@ public class OzoneException extends Exception {
   @JsonProperty("hostName")
   private String hostID;
 
+  /**
+   * Constructs a new exception with {@code null} as its detail message. The
+   * cause is not initialized, and may subsequently be initialized by a call
+   * to {@link #initCause}.
+   *
+   * This constructor is needed by Json Serializer.
+   */
+  public OzoneException() {
+  }
+
 
   /**
    * Constructor that allows a shortMessage and exception.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1244d8ff/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/BucketProcessTemplate.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/BucketProcessTemplate.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/BucketProcessTemplate.java
index 73827db..a312939 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/BucketProcessTemplate.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/BucketProcessTemplate.java
@@ -181,7 +181,7 @@ public abstract class BucketProcessTemplate {
    *
    * @throws OzoneException
    */
-  List<String> getAcls(BucketArgs args, String tag) throws OzoneException {
+  List<String> getAcls(BucketArgs args, String tag)  {
     List<String> aclStrings =
         args.getHeaders().getRequestHeader(Header.OZONE_ACLS);
     List<String> filteredSet = null;
@@ -242,7 +242,7 @@ public abstract class BucketProcessTemplate {
 
     try {
       List<String> storageClassString =
-          args.getHeaders().getRequestHeader(Header.OZONE_STORAGE_CLASS);
+          args.getHeaders().getRequestHeader(Header.OZONE_STORAGE_TYPE);
       if (storageClassString == null) {
         return null;
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1244d8ff/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
index 37c27da..6482f2f 100644
--- 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
@@ -19,6 +19,8 @@
 package org.apache.hadoop.ozone.web.handlers;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ozone.OzoneConfiguration;
 import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
 import org.apache.hadoop.ozone.web.localstorage.LocalStorageHandler;
 
@@ -44,7 +46,8 @@ public final class StorageHandlerBuilder {
       return storageHandler;
     } else {
       // This only happens while using mvn jetty:run for testing.
-      return new LocalStorageHandler();
+      Configuration conf = new OzoneConfiguration();
+      return new LocalStorageHandler(conf);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1244d8ff/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 3569a3f..45f36ae 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
@@ -45,7 +45,7 @@ public final class Header {
   public static final String OZONE_REQUEST_ID = "x-ozone-request-id";
   public static final String OZONE_SERVER_NAME = "x-ozone-server-name";
 
-  public static final String OZONE_STORAGE_CLASS = "x-ozone-storage-type";
+  public static final String OZONE_STORAGE_TYPE = "x-ozone-storage-type";
 
   public static final String OZONE_BUCKET_VERSIONING =
       "x-ozone-bucket-versioning";
@@ -54,6 +54,10 @@ public final class Header {
   public static final String OZONE_ACL_ADD = "ADD";
   public static final String OZONE_ACL_REMOVE = "REMOVE";
 
+  public static final String OZONE_LIST_QUERY_TAG ="info";
+  public static final String OZONE_QUOTA_QUERY_TAG ="quota";
+
+
 
   private Header() {
     // Never constructed.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1244d8ff/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
index 8d00e66..9850551 100644
--- 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
@@ -19,6 +19,7 @@
 package org.apache.hadoop.ozone.web.localstorage;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ozone.web.exceptions.OzoneException;
 import org.apache.hadoop.ozone.web.handlers.BucketArgs;
 import org.apache.hadoop.ozone.web.handlers.UserArgs;
@@ -40,10 +41,12 @@ import java.io.IOException;
  */
 @InterfaceAudience.Private
 public class LocalStorageHandler implements StorageHandler {
+  private final Configuration conf;
   /**
    * Constructs LocalStorageHandler.
    */
-  public LocalStorageHandler() {
+  public LocalStorageHandler(Configuration conf) {
+    this.conf = conf;
   }
 
   /**
@@ -55,7 +58,8 @@ public class LocalStorageHandler implements StorageHandler {
    */
   @Override
   public void createVolume(VolumeArgs args) throws IOException, OzoneException {
-    OzoneMetadataManager oz = OzoneMetadataManager.getOzoneMetadataManager();
+    OzoneMetadataManager oz =
+        OzoneMetadataManager.getOzoneMetadataManager(conf);
     oz.createVolume(args);
 
   }
@@ -70,7 +74,8 @@ public class LocalStorageHandler implements StorageHandler {
   @Override
   public void setVolumeOwner(VolumeArgs args)
       throws IOException, OzoneException {
-    OzoneMetadataManager oz = OzoneMetadataManager.getOzoneMetadataManager();
+    OzoneMetadataManager oz =
+        OzoneMetadataManager.getOzoneMetadataManager(conf);
     oz.setVolumeProperty(args, OzoneMetadataManager.VolumeProperty.OWNER);
   }
 
@@ -85,7 +90,8 @@ public class LocalStorageHandler implements StorageHandler {
   @Override
   public void setVolumeQuota(VolumeArgs args, boolean remove)
       throws IOException, OzoneException {
-    OzoneMetadataManager oz = OzoneMetadataManager.getOzoneMetadataManager();
+    OzoneMetadataManager oz =
+        OzoneMetadataManager.getOzoneMetadataManager(conf);
 
     if(remove) {
       OzoneQuota quota = new OzoneQuota();
@@ -109,7 +115,8 @@ public class LocalStorageHandler implements StorageHandler {
   @Override
   public boolean checkVolumeAccess(VolumeArgs args)
       throws IOException, OzoneException {
-    OzoneMetadataManager oz = OzoneMetadataManager.getOzoneMetadataManager();
+    OzoneMetadataManager oz =
+        OzoneMetadataManager.getOzoneMetadataManager(conf);
     return oz.checkVolumeAccess(args);
   }
 
@@ -126,7 +133,8 @@ public class LocalStorageHandler implements StorageHandler {
   @Override
   public VolumeInfo getVolumeInfo(VolumeArgs args)
       throws IOException, OzoneException {
-    OzoneMetadataManager oz = OzoneMetadataManager.getOzoneMetadataManager();
+    OzoneMetadataManager oz =
+        OzoneMetadataManager.getOzoneMetadataManager(conf);
     return oz.getVolumeInfo(args);
   }
 
@@ -140,7 +148,8 @@ public class LocalStorageHandler implements StorageHandler {
    */
   @Override
   public void deleteVolume(VolumeArgs args) throws IOException, OzoneException {
-    OzoneMetadataManager oz = OzoneMetadataManager.getOzoneMetadataManager();
+    OzoneMetadataManager oz =
+        OzoneMetadataManager.getOzoneMetadataManager(conf);
     oz.deleteVolume(args);
 
   }
@@ -157,7 +166,8 @@ public class LocalStorageHandler implements StorageHandler {
   @Override
   public ListVolumes listVolumes(UserArgs args)
       throws IOException, OzoneException {
-    OzoneMetadataManager oz = OzoneMetadataManager.getOzoneMetadataManager();
+    OzoneMetadataManager oz =
+        OzoneMetadataManager.getOzoneMetadataManager(conf);
     return oz.listVolumes(args);
   }
 
@@ -184,7 +194,8 @@ public class LocalStorageHandler implements StorageHandler {
    */
   @Override
   public void createBucket(BucketArgs args) throws IOException, OzoneException {
-    OzoneMetadataManager oz = OzoneMetadataManager.getOzoneMetadataManager();
+    OzoneMetadataManager oz =
+        OzoneMetadataManager.getOzoneMetadataManager(conf);
     oz.createBucket(args);
   }
 
@@ -198,7 +209,8 @@ public class LocalStorageHandler implements StorageHandler {
   @Override
   public void setBucketAcls(BucketArgs args)
       throws IOException, OzoneException {
-    OzoneMetadataManager oz = OzoneMetadataManager.getOzoneMetadataManager();
+    OzoneMetadataManager oz =
+        OzoneMetadataManager.getOzoneMetadataManager(conf);
     oz.setBucketProperty(args, OzoneMetadataManager.BucketProperty.ACLS);
   }
 
@@ -212,7 +224,8 @@ public class LocalStorageHandler implements StorageHandler {
   @Override
   public void setBucketVersioning(BucketArgs args)
       throws IOException, OzoneException {
-    OzoneMetadataManager oz = OzoneMetadataManager.getOzoneMetadataManager();
+    OzoneMetadataManager oz =
+        OzoneMetadataManager.getOzoneMetadataManager(conf);
     oz.setBucketProperty(args, OzoneMetadataManager.BucketProperty.VERSIONING);
 
   }
@@ -227,7 +240,8 @@ public class LocalStorageHandler implements StorageHandler {
   @Override
   public void setBucketStorageClass(BucketArgs args)
       throws IOException, OzoneException {
-    OzoneMetadataManager oz = OzoneMetadataManager.getOzoneMetadataManager();
+    OzoneMetadataManager oz =
+        OzoneMetadataManager.getOzoneMetadataManager(conf);
     oz.setBucketProperty(args, OzoneMetadataManager.BucketProperty.STORAGETYPE);
 
   }
@@ -241,7 +255,8 @@ public class LocalStorageHandler implements StorageHandler {
    */
   @Override
   public void deleteBucket(BucketArgs args) throws IOException, OzoneException {
-    OzoneMetadataManager oz = OzoneMetadataManager.getOzoneMetadataManager();
+    OzoneMetadataManager oz =
+        OzoneMetadataManager.getOzoneMetadataManager(conf);
     oz.deleteBucket(args);
   }
 
@@ -257,7 +272,8 @@ public class LocalStorageHandler implements StorageHandler {
   @Override
   public ListBuckets listBuckets(VolumeArgs args)
       throws IOException, OzoneException {
-    OzoneMetadataManager oz = OzoneMetadataManager.getOzoneMetadataManager();
+    OzoneMetadataManager oz =
+        OzoneMetadataManager.getOzoneMetadataManager(conf);
     return oz.listBuckets(args);
   }
 
@@ -273,7 +289,8 @@ public class LocalStorageHandler implements StorageHandler {
   @Override
   public BucketInfo getBucketInfo(BucketArgs args)
       throws IOException, OzoneException {
-    OzoneMetadataManager oz = OzoneMetadataManager.getOzoneMetadataManager();
+    OzoneMetadataManager oz =
+        OzoneMetadataManager.getOzoneMetadataManager(conf);
     return oz.getBucketInfo(args);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1244d8ff/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/OzoneMetadataManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/OzoneMetadataManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/OzoneMetadataManager.java
index d14baed..db9903c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/OzoneMetadataManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/OzoneMetadataManager.java
@@ -19,8 +19,8 @@ package org.apache.hadoop.ozone.web.localstorage;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
-import org.apache.hadoop.ozone.OzoneConfiguration;
 import org.apache.hadoop.ozone.web.exceptions.ErrorTable;
 import org.apache.hadoop.ozone.web.exceptions.OzoneException;
 import org.apache.hadoop.ozone.web.handlers.BucketArgs;
@@ -123,11 +123,9 @@ public final class OzoneMetadataManager {
   /**
    * Constructs OzoneMetadataManager.
    */
-  private OzoneMetadataManager() {
+  private OzoneMetadataManager(Configuration conf) {
 
     lock = new ReentrantReadWriteLock();
-    OzoneConfiguration conf = new OzoneConfiguration();
-
     String storageRoot =
         conf.getTrimmed(OzoneConfigKeys.DFS_STORAGE_LOCAL_ROOT,
             OzoneConfigKeys.DFS_STORAGE_LOCAL_ROOT_DEFAULT);
@@ -151,9 +149,10 @@ public final class OzoneMetadataManager {
    *
    * @return OzoneMetadataManager
    */
-  public static synchronized OzoneMetadataManager getOzoneMetadataManager() {
+  public static synchronized OzoneMetadataManager
+      getOzoneMetadataManager(Configuration conf) {
     if (bm == null) {
-      bm = new OzoneMetadataManager();
+      bm = new OzoneMetadataManager(conf);
     }
     return bm;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1244d8ff/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/messages/StringMessageBodyWriter.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/messages/StringMessageBodyWriter.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/messages/StringMessageBodyWriter.java
new file mode 100644
index 0000000..ad637af
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/messages/StringMessageBodyWriter.java
@@ -0,0 +1,62 @@
+/*
+ * 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.messages;
+
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.ozone.web.utils.OzoneUtils;
+
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.ext.MessageBodyWriter;
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Type;
+
+/**
+ * Writes outbound HTTP response strings.  We use this rather than the built-in
+ * writer so that we can determine content length from the string length instead
+ * of possibly falling back to a chunked response.
+ */
+public final class StringMessageBodyWriter implements
+    MessageBodyWriter<String> {
+  private static final int CHUNK_SIZE = 8192;
+
+  @Override
+  public long getSize(String str, Class<?> type, Type genericType,
+                      Annotation[] annotations, MediaType mediaType) {
+    return str.length();
+  }
+
+  @Override
+  public boolean isWriteable(Class<?> type, Type genericType,
+                             Annotation[] annotations, MediaType mediaType) {
+    return String.class.isAssignableFrom(type);
+  }
+
+  @Override
+  public void writeTo(String str, Class<?> type, Type genericType,
+                      Annotation[] annotations, MediaType mediaType,
+                      MultivaluedMap<String, Object> httpHeaders,
+                      OutputStream out) throws IOException {
+    IOUtils.copyBytes(new ByteArrayInputStream(
+        str.getBytes(OzoneUtils.ENCODING)), out, CHUNK_SIZE);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1244d8ff/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
index c6130c1..a806390 100644
--- 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
@@ -30,6 +30,7 @@ import javax.ws.rs.core.Response;
 import java.io.InputStream;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
+import java.nio.charset.Charset;
 import java.text.ParseException;
 import java.text.SimpleDateFormat;
 import java.util.Date;
@@ -44,6 +45,8 @@ import java.util.UUID;
 @InterfaceAudience.Private
 public final class OzoneUtils {
 
+  public static final Charset ENCODING = Charset.forName("UTF-8");
+
   private OzoneUtils() {
     // Never constructed
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1244d8ff/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneVolumes.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneVolumes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneVolumes.java
index aa07484..877a33d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneVolumes.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneVolumes.java
@@ -31,6 +31,8 @@ import org.apache.http.client.HttpClient;
 import org.apache.http.client.methods.HttpGet;
 import org.apache.http.client.methods.HttpPost;
 import org.apache.http.impl.client.DefaultHttpClient;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -64,13 +66,15 @@ public class TestOzoneVolumes {
     OzoneConfiguration conf = new OzoneConfiguration();
 
     URL p = conf.getClass().getResource("");
-    String path = p.getPath();
+    String path = p.getPath().concat(TestOzoneVolumes.class.getSimpleName());
     path += conf.getTrimmed(OzoneConfigKeys.DFS_STORAGE_LOCAL_ROOT,
         OzoneConfigKeys.DFS_STORAGE_LOCAL_ROOT_DEFAULT);
 
     conf.set(OzoneConfigKeys.DFS_STORAGE_LOCAL_ROOT, path);
     conf.setBoolean(OzoneConfigKeys.DFS_OBJECTSTORE_ENABLED_KEY, true);
     conf.set(OzoneConfigKeys.DFS_STORAGE_HANDLER_TYPE_KEY, "local");
+    conf.setBoolean(OzoneConfigKeys.DFS_OBJECTSTORE_TRACE_ENABLED_KEY, true);
+    Logger.getLogger("log4j.logger.org.apache.http").setLevel(Level.DEBUG);
 
     cluster = new MiniDFSCluster.Builder(conf).build();
     cluster.waitActive();
@@ -296,7 +300,8 @@ public class TestOzoneVolumes {
   public void testGetVolumesByUser() throws IOException {
     SimpleDateFormat format =
         new SimpleDateFormat("EEE, dd MMM yyyy HH:mm:ss ZZZ", Locale.US);
-
+    // We need to create a volume for this test to succeed.
+    testCreateVolumes();
     HttpClient client = new DefaultHttpClient();
     try {
       HttpGet httpget =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1244d8ff/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneWebAccess.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneWebAccess.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneWebAccess.java
index c1e8763..857ef34 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneWebAccess.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneWebAccess.java
@@ -23,6 +23,7 @@ import static org.apache.hadoop.ozone.web.utils.OzoneUtils.*;
 import static org.junit.Assert.assertEquals;
 
 import java.io.IOException;
+import java.net.URL;
 import java.text.SimpleDateFormat;
 import java.util.Date;
 import java.util.Locale;
@@ -69,9 +70,8 @@ public class TestOzoneWebAccess {
   public static void init() throws IOException {
     OzoneConfiguration conf = new OzoneConfiguration();
 
-    String path = new Path(
-        System.getProperty("test.build.data", "target/test/data"),
-        TestOzoneWebAccess.class.getSimpleName()).toUri().getPath();
+    URL p = conf.getClass().getResource("");
+    String path = p.getPath().concat(TestOzoneWebAccess.class.getSimpleName());
     conf.set(OzoneConfigKeys.DFS_STORAGE_LOCAL_ROOT, path);
     conf.setBoolean(OzoneConfigKeys.DFS_OBJECTSTORE_ENABLED_KEY, true);
     conf.set(OzoneConfigKeys.DFS_STORAGE_HANDLER_TYPE_KEY, "local");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1244d8ff/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java
new file mode 100644
index 0000000..45b8795
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java
@@ -0,0 +1,162 @@
+/*
+ * 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.client;
+
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.ozone.OzoneConfiguration;
+import org.apache.hadoop.ozone.web.exceptions.OzoneException;
+import org.apache.hadoop.ozone.web.request.OzoneQuota;
+import org.apache.hadoop.ozone.web.utils.OzoneConsts;
+import org.apache.hadoop.ozone.web.utils.OzoneUtils;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+public class TestVolume {
+  private static MiniDFSCluster cluster = null;
+  private static int port = 0;
+  private static OzoneClient client = null;
+
+  /**
+   * Create a MiniDFSCluster for testing.
+   * <p>
+   * Ozone is made active by setting DFS_OBJECTSTORE_ENABLED_KEY = true and
+   * DFS_STORAGE_HANDLER_TYPE_KEY = "local" , which uses a local directory to
+   * emulate Ozone backend.
+   *
+   * @throws IOException
+   */
+  @BeforeClass
+  public static void init() throws IOException, OzoneException,
+      URISyntaxException {
+    OzoneConfiguration conf = new OzoneConfiguration();
+
+    URL p = conf.getClass().getResource("");
+    String path = p.getPath().concat(TestVolume.class.getSimpleName());
+    path += conf.getTrimmed(OzoneConfigKeys.DFS_STORAGE_LOCAL_ROOT,
+        OzoneConfigKeys.DFS_STORAGE_LOCAL_ROOT_DEFAULT);
+
+    conf.set(OzoneConfigKeys.DFS_STORAGE_LOCAL_ROOT, path);
+    conf.setBoolean(OzoneConfigKeys.DFS_OBJECTSTORE_ENABLED_KEY, true);
+    conf.set(OzoneConfigKeys.DFS_STORAGE_HANDLER_TYPE_KEY, "local");
+    conf.setBoolean(OzoneConfigKeys.DFS_OBJECTSTORE_TRACE_ENABLED_KEY, true);
+    Logger.getLogger("log4j.logger.org.apache.http").setLevel(Level.DEBUG);
+
+    cluster = new MiniDFSCluster.Builder(conf).build();
+    cluster.waitActive();
+    DataNode dataNode = cluster.getDataNodes().get(0);
+    port = dataNode.getInfoPort();
+
+    client = new OzoneClient(String.format("http://localhost:%d", port));
+  }
+
+  /**
+   * shutdown MiniDFSCluster
+   */
+  @AfterClass
+  public static void shutdown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testCreateVolume() throws OzoneException {
+    String volumeName = OzoneUtils.getRequestID().toLowerCase();
+    client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
+    OzoneVolume vol = client.createVolume(volumeName, "bilbo", "100TB");
+
+    assertEquals(vol.getVolumeName(), volumeName);
+    assertEquals(vol.getCreatedby(), "hdfs");
+    assertEquals(vol.getOwnerName(), "bilbo");
+    assertEquals(vol.getQuota().getUnit(), OzoneQuota.Units.TB);
+    assertEquals(vol.getQuota().getSize(), 100);
+  }
+
+  @Test
+  public void testCreateDuplicateVolume() throws OzoneException {
+    try {
+      client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
+      client.createVolume("testvol", "bilbo", "100TB");
+      client.createVolume("testvol", "bilbo", "100TB");
+      assertFalse(true);
+    } catch (OzoneException ex) {
+      // OZone will throw saying volume already exists
+      assertEquals(ex.getShortMessage(),"volumeAlreadyExists");
+    }
+  }
+
+  @Test
+  public void testDeleteVolume() throws OzoneException {
+    String volumeName = OzoneUtils.getRequestID().toLowerCase();
+    client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
+    OzoneVolume vol = client.createVolume(volumeName, "bilbo", "100TB");
+    client.deleteVolume(vol.getVolumeName());
+  }
+
+  @Test
+  public void testChangeOwnerOnVolume() throws OzoneException {
+    String volumeName = OzoneUtils.getRequestID().toLowerCase();
+    client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
+    OzoneVolume vol = client.createVolume(volumeName, "bilbo", "100TB");
+    client.setVolumeOwner(volumeName, "frodo");
+    OzoneVolume newVol = client.getVolume(volumeName);
+    assertEquals(newVol.getOwnerName(), "frodo");
+  }
+
+  @Test
+  public void testChangeQuotaOnVolume() throws OzoneException, IOException {
+    String volumeName = OzoneUtils.getRequestID().toLowerCase();
+    client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
+    OzoneVolume vol = client.createVolume(volumeName, "bilbo", "100TB");
+    client.setVolumeQuota(volumeName, "1000MB");
+    OzoneVolume newVol = client.getVolume(volumeName);
+    assertEquals(newVol.getQuota().getSize(), 1000);
+    assertEquals(newVol.getQuota().getUnit(), OzoneQuota.Units.MB);
+  }
+
+  @Test
+  public void testListVolume() throws OzoneException, IOException {
+    client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
+    for (int x = 0; x < 10; x++) {
+      String volumeName = OzoneUtils.getRequestID().toLowerCase();
+      OzoneVolume vol = client.createVolume(volumeName, "frodo", "100TB");
+      assertNotNull(vol);
+    }
+
+    List<OzoneVolume> ovols = client.listVolumes("frodo");
+    assertTrue(ovols.size() >= 10);
+  }
+
+}