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/11 00:23:46 UTC

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

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-7240 28f770d8b -> e4fb9bdd4


HDFS-9925. Ozone: Add Ozone Client lib for bucket 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/e4fb9bdd
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e4fb9bdd
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e4fb9bdd

Branch: refs/heads/HDFS-7240
Commit: e4fb9bdd49fbf16aec9bae74e3dd8699f9ea1ea6
Parents: 28f770d
Author: Chris Nauroth <cn...@apache.org>
Authored: Thu Mar 10 15:23:41 2016 -0800
Committer: Chris Nauroth <cn...@apache.org>
Committed: Thu Mar 10 15:23:41 2016 -0800

----------------------------------------------------------------------
 .../hadoop/ozone/web/client/OzoneBucket.java    | 128 ++++++
 .../hadoop/ozone/web/client/OzoneVolume.java    | 416 +++++++++++++++++++
 .../ozone/web/handlers/VolumeHandler.java       |   4 +-
 .../web/handlers/VolumeProcessTemplate.java     |  26 ++
 .../hadoop/ozone/web/request/OzoneAcl.java      |   6 +
 .../hadoop/ozone/web/response/ListBuckets.java  |   2 +-
 .../hadoop/ozone/web/client/TestBuckets.java    | 174 ++++++++
 7 files changed, 752 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e4fb9bdd/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneBucket.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneBucket.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneBucket.java
new file mode 100644
index 0000000..cde8a69
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneBucket.java
@@ -0,0 +1,128 @@
+/*
+ * 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.commons.httpclient.HttpException;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.ozone.web.request.OzoneAcl;
+import org.apache.hadoop.ozone.web.response.BucketInfo;
+import org.apache.hadoop.ozone.web.utils.OzoneConsts;
+import org.apache.http.HttpRequest;
+import org.apache.http.HttpRequestInterceptor;
+import org.apache.http.protocol.HTTP;
+import org.apache.http.protocol.HttpContext;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * A Bucket class the represents an Ozone Bucket.
+ */
+public class OzoneBucket {
+
+  private BucketInfo bucketInfo;
+  private OzoneVolume volume;
+
+  /**
+   * Constructor for bucket.
+   *
+   * @param info   - BucketInfo
+   * @param volume - OzoneVolume Object that contains this bucket
+   */
+  public OzoneBucket(BucketInfo info, OzoneVolume volume) {
+    this.bucketInfo = info;
+    this.volume = volume;
+  }
+
+  /**
+   * Gets bucket Info.
+   *
+   * @return BucketInfo
+   */
+  public BucketInfo getBucketInfo() {
+    return bucketInfo;
+  }
+
+  /**
+   * Sets Bucket Info.
+   *
+   * @param bucketInfo BucketInfo
+   */
+  public void setBucketInfo(BucketInfo bucketInfo) {
+    this.bucketInfo = bucketInfo;
+  }
+
+  /**
+   * Returns the parent volume class.
+   *
+   * @return - OzoneVolume
+   */
+  OzoneVolume getVolume() {
+    return volume;
+  }
+
+  /**
+   * Returns bucket name.
+   *
+   * @return Bucket Name
+   */
+  public String getBucketName() {
+    return bucketInfo.getBucketName();
+  }
+
+  /**
+   * Returns the Acls on the bucket.
+   *
+   * @return - Acls
+   */
+  public List<OzoneAcl> getAcls() {
+    return bucketInfo.getAcls();
+  }
+
+  /**
+   * Return versioning info on the bucket - Enabled or disabled.
+   *
+   * @return - Version Enum
+   */
+  public OzoneConsts.Versioning getVersioning() {
+    return bucketInfo.getVersioning();
+  }
+
+  /**
+   * Gets the Storage class for the bucket.
+   *
+   * @return Storage Class Enum
+   */
+  public StorageType getStorageClass() {
+    return bucketInfo.getStorageType();
+  }
+
+  private static class ContentLengthHeaderRemover implements
+      HttpRequestInterceptor {
+    @Override
+    public void process(HttpRequest request, HttpContext context)
+        throws HttpException, IOException {
+
+      // fighting org.apache.http.protocol
+      // .RequestContent's ProtocolException("Content-Length header
+      // already present");
+      request.removeHeaders(HTTP.CONTENT_LEN);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e4fb9bdd/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
index 46033cc..0de1eb2 100644
--- 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
@@ -18,13 +18,35 @@
 
 package org.apache.hadoop.ozone.web.client;
 
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.ozone.web.exceptions.OzoneException;
+import org.apache.hadoop.ozone.web.headers.Header;
 import org.apache.hadoop.ozone.web.request.OzoneQuota;
+import org.apache.hadoop.ozone.web.response.BucketInfo;
+import org.apache.hadoop.ozone.web.response.ListBuckets;
 import org.apache.hadoop.ozone.web.response.VolumeInfo;
+import org.apache.hadoop.ozone.web.utils.OzoneConsts;
+import org.apache.hadoop.ozone.web.utils.OzoneUtils;
+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.utils.URIBuilder;
+import org.apache.http.impl.client.DefaultHttpClient;
+import org.apache.http.util.EntityUtils;
 
 import java.io.IOException;
+import java.net.URISyntaxException;
 import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
 import java.util.Map;
 
+import static java.net.HttpURLConnection.HTTP_CREATED;
+import static java.net.HttpURLConnection.HTTP_OK;
+
 /**
  * Ozone Volume Class.
  */
@@ -52,6 +74,11 @@ public class OzoneVolume {
     this.client = client;
   }
 
+  /**
+   * Returns a Json String of this class.
+   * @return String
+   * @throws IOException
+   */
   public String getJsonString() throws IOException {
     return volumeInfo.toJsonString();
   }
@@ -119,4 +146,393 @@ public class OzoneVolume {
   OzoneClient getClient() {
     return client;
   }
+
+  /**
+   * Create Bucket - Creates a bucket under a given volume.
+   *
+   * @param bucketName - Bucket Name
+   * @param acls - Acls - User Acls
+   * @param storageType - Storage Class
+   * @param versioning - enable versioning support on a bucket.
+   *
+   *
+   * @return - a Ozone Bucket Object
+   */
+  public OzoneBucket createBucket(String bucketName, String[] acls,
+                                  StorageType storageType,
+                                  OzoneConsts.Versioning versioning)
+      throws OzoneException {
+
+    try {
+      OzoneUtils.verifyBucketName(bucketName);
+      DefaultHttpClient httpClient = new DefaultHttpClient();
+      URIBuilder builder = new URIBuilder(getClient().getEndPointURI());
+      builder.setPath("/" + getVolumeName() + "/" + bucketName).build();
+
+      HttpPost httppost = client.getHttpPost(null, builder.toString());
+      if (acls != null) {
+        for (String acl : acls) {
+          httppost
+              .addHeader(Header.OZONE_ACLS, Header.OZONE_ACL_ADD + " " + acl);
+        }
+      }
+
+      httppost.addHeader(Header.OZONE_STORAGE_TYPE, storageType.toString());
+      httppost.addHeader(Header.OZONE_BUCKET_VERSIONING, versioning.toString());
+      executeCreateBucket(httppost, httpClient);
+      return getBucket(bucketName);
+    } catch (IOException | URISyntaxException | IllegalArgumentException ex) {
+      throw new OzoneClientException(ex.getMessage());
+    }
+  }
+
+  /**
+   * Create Bucket.
+   *
+   * @param bucketName - bucket name
+   * @param acls - acls
+   * @param storageType - storage class
+   *
+   * @throws OzoneException
+   */
+  public OzoneBucket createBucket(String bucketName, String[] acls,
+                                  StorageType storageType)
+      throws OzoneException {
+    return createBucket(bucketName, acls, storageType,
+        OzoneConsts.Versioning.DISABLED);
+  }
+
+  /**
+   * Create Bucket.
+   *
+   * @param bucketName - bucket name
+   * @param acls - acls
+   *
+   * @throws OzoneException
+   */
+  public OzoneBucket createBucket(String bucketName, String[] acls)
+      throws OzoneException {
+    return createBucket(bucketName, acls, StorageType.DEFAULT,
+        OzoneConsts.Versioning.DISABLED);
+  }
+
+
+  /**
+   * Create Bucket.
+   *
+   * @param bucketName - bucket name
+   *
+   * @throws OzoneException
+   */
+  public OzoneBucket createBucket(String bucketName) throws OzoneException {
+
+    return createBucket(bucketName, null,  StorageType.DEFAULT,
+        OzoneConsts.Versioning.DISABLED);
+  }
+
+
+  /**
+   * execute a Create Bucket Request against Ozone server.
+   *
+   * @param httppost - httpPost
+   *
+   * @throws IOException
+   * @throws OzoneException
+   */
+  private void executeCreateBucket(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.consumeQuietly(entity);
+      }
+    }
+  }
+
+  /**
+   * Adds Acls to an existing bucket.
+   *
+   * @param bucketName - Name of the bucket
+   * @param acls - Acls
+   *
+   * @throws OzoneException
+   */
+  public void addAcls(String bucketName, String[] acls) throws OzoneException {
+
+    try {
+      OzoneUtils.verifyBucketName(bucketName);
+      DefaultHttpClient httpClient = new DefaultHttpClient();
+      URIBuilder builder = new URIBuilder(getClient().getEndPointURI());
+      builder.setPath("/" + getVolumeName() + "/" + bucketName).build();
+      HttpPut putRequest = client.getHttpPut(builder.toString());
+
+      for (String acl : acls) {
+        putRequest
+            .addHeader(Header.OZONE_ACLS, Header.OZONE_ACL_ADD + " " + acl);
+      }
+      executePutBucket(putRequest, httpClient);
+    } catch (URISyntaxException | IOException ex) {
+      throw new OzoneClientException(ex.getMessage());
+    }
+  }
+
+  /**
+   * Removes ACLs from a bucket.
+   *
+   * @param bucketName - Bucket Name
+   * @param acls - Acls to be removed
+   *
+   * @throws OzoneException
+   */
+  public void removeAcls(String bucketName, String[] acls)
+      throws OzoneException {
+    try {
+      OzoneUtils.verifyBucketName(bucketName);
+      DefaultHttpClient httpClient = new DefaultHttpClient();
+      URIBuilder builder = new URIBuilder(getClient().getEndPointURI());
+      builder.setPath("/" + getVolumeName() + "/" + bucketName).build();
+      HttpPut putRequest = client.getHttpPut(builder.toString());
+
+      for (String acl : acls) {
+        putRequest
+            .addHeader(Header.OZONE_ACLS, Header.OZONE_ACL_REMOVE + " " + acl);
+      }
+      executePutBucket(putRequest, httpClient);
+    } catch (URISyntaxException | IOException ex) {
+      throw new OzoneClientException(ex.getMessage());
+    }
+  }
+
+  /**
+   * Returns information about an existing bucket.
+   *
+   * @param bucketName - BucketName
+   *
+   * @return OZoneBucket
+   */
+  public OzoneBucket getBucket(String bucketName) throws OzoneException {
+    try {
+      OzoneUtils.verifyBucketName(bucketName);
+      DefaultHttpClient httpClient = new DefaultHttpClient();
+      URIBuilder builder = new URIBuilder(getClient().getEndPointURI());
+      builder.setPath("/" + getVolumeName() + "/" + bucketName)
+        .setParameter(Header.OZONE_LIST_QUERY_TAG,
+            Header.OZONE_LIST_QUERY_BUCKET).build();
+      HttpGet getRequest = client.getHttpGet(builder.toString());
+      return executeInfoBucket(getRequest, httpClient);
+
+    } catch (IOException | URISyntaxException | IllegalArgumentException ex) {
+      throw new OzoneClientException(ex.getMessage());
+    }
+  }
+
+
+  /**
+   * Execute the info bucket call.
+   *
+   * @param getRequest - httpGet Request
+   * @param httpClient - Http Client
+   *
+   * @return OzoneBucket
+   *
+   * @throws IOException
+   * @throws OzoneException
+   */
+  private OzoneBucket executeInfoBucket(HttpGet getRequest,
+                                        DefaultHttpClient httpClient)
+      throws IOException, OzoneException {
+    HttpEntity entity = null;
+    try {
+      HttpResponse response = httpClient.execute(getRequest);
+      int errorCode = response.getStatusLine().getStatusCode();
+      entity = response.getEntity();
+      if (entity == null) {
+        throw new OzoneClientException("Unexpected null in http payload");
+      }
+      if ((errorCode == HTTP_OK) || (errorCode == HTTP_CREATED)) {
+        OzoneBucket bucket =
+            new OzoneBucket(BucketInfo.parse(EntityUtils.toString(entity)),
+                this);
+        return bucket;
+      }
+      throw OzoneException.parse(EntityUtils.toString(entity));
+    } finally {
+      if (entity != null) {
+        EntityUtils.consumeQuietly(entity);
+      }
+    }
+  }
+
+  /**
+   * Execute the put bucket call.
+   *
+   * @param putRequest - http put request
+   * @param httpClient - Http Client
+   *
+   * @return OzoneBucket
+   *
+   * @throws IOException
+   * @throws OzoneException
+   */
+  private void executePutBucket(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) {
+        return;
+      }
+
+      if (entity != null) {
+        throw OzoneException.parse(EntityUtils.toString(entity));
+      }
+
+      throw new OzoneClientException("Unexpected null in http result");
+    } finally {
+      if (entity != null) {
+        EntityUtils.consumeQuietly(entity);
+      }
+    }
+  }
+
+  /**
+   * Gets a list of buckets on this volume.
+   *
+   * @return - List of buckets
+   *
+   * @throws OzoneException
+   */
+  public List<OzoneBucket> listBuckets() throws OzoneException {
+    try {
+      DefaultHttpClient httpClient = new DefaultHttpClient();
+
+      URIBuilder builder = new URIBuilder(getClient().getEndPointURI());
+      builder.setPath("/" + getVolumeName()).build();
+
+      HttpGet getRequest = client.getHttpGet(builder.toString());
+      return executeListBuckets(getRequest, httpClient);
+
+    } catch (IOException | URISyntaxException e) {
+      throw new OzoneClientException(e.getMessage());
+    }
+  }
+
+  /**
+   * executes the List Bucket Call.
+   *
+   * @param getRequest - http Request
+   * @param httpClient - http Client
+   *
+   * @return List of OzoneBuckets
+   *
+   * @throws IOException
+   * @throws OzoneException
+   */
+  private List<OzoneBucket> executeListBuckets(HttpGet getRequest,
+                                               DefaultHttpClient httpClient)
+      throws IOException, OzoneException {
+    HttpEntity entity = null;
+    List<OzoneBucket> ozoneBucketList = new LinkedList<OzoneBucket>();
+    try {
+      HttpResponse response = httpClient.execute(getRequest);
+      int errorCode = response.getStatusLine().getStatusCode();
+
+      entity = response.getEntity();
+
+      if (entity == null) {
+        throw new OzoneClientException("Unexpected null in http payload");
+      }
+      if (errorCode == HTTP_OK) {
+        ListBuckets bucketList =
+            ListBuckets.parse(EntityUtils.toString(entity));
+
+        for (BucketInfo info : bucketList.getBuckets()) {
+          ozoneBucketList.add(new OzoneBucket(info, this));
+        }
+        return ozoneBucketList;
+
+      } else {
+        throw OzoneException.parse(EntityUtils.toString(entity));
+      }
+    } finally {
+      if (entity != null) {
+        EntityUtils.consumeQuietly(entity);
+      }
+    }
+  }
+
+  /**
+   * Delete an empty bucket.
+   *
+   * @param bucketName - Name of the bucket to delete
+   *
+   * @throws OzoneException
+   */
+  public void deleteBucket(String bucketName) throws OzoneException {
+    try {
+      OzoneUtils.verifyBucketName(bucketName);
+      DefaultHttpClient httpClient = new DefaultHttpClient();
+      URIBuilder builder = new URIBuilder(getClient().getEndPointURI());
+      builder.setPath("/" + getVolumeName() + "/" + bucketName).build();
+
+      HttpDelete delRequest = client.getHttpDelete(builder.toString());
+      executeDeleteBucket(delRequest, httpClient);
+
+    } catch (IOException | URISyntaxException | IllegalArgumentException ex) {
+      throw new OzoneClientException(ex.getMessage());
+    }
+  }
+
+  /**
+   * Executes delete bucket call.
+   *
+   * @param delRequest - Delete Request
+   * @param httpClient - Http Client
+7   *
+   * @throws IOException
+   * @throws OzoneException
+   */
+  private void executeDeleteBucket(HttpDelete delRequest,
+                                   DefaultHttpClient httpClient)
+      throws IOException, OzoneException {
+    HttpEntity entity = null;
+    try {
+      HttpResponse response = httpClient.execute(delRequest);
+      int errorCode = response.getStatusLine().getStatusCode();
+      entity = response.getEntity();
+
+      if (errorCode == HTTP_OK) {
+        return;
+      }
+
+      if (entity == null) {
+        throw new OzoneClientException("Unexpected null in http payload.");
+      }
+
+      throw OzoneException.parse(EntityUtils.toString(entity));
+
+    } finally {
+      if (entity != null) {
+        EntityUtils.consumeQuietly(entity);
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e4fb9bdd/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
index 73a7a07..09a021b 100644
--- 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
@@ -209,9 +209,7 @@ public class VolumeHandler implements Volume {
 
         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
+            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:

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e4fb9bdd/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
index 8f4d4a6..d357da7 100644
--- 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
@@ -23,6 +23,7 @@ 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.ListBuckets;
 import org.apache.hadoop.ozone.web.response.ListVolumes;
 import org.apache.hadoop.ozone.web.response.VolumeInfo;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;
@@ -232,4 +233,29 @@ public abstract class VolumeProcessTemplate {
     }
   }
 
+
+  /**
+   * Returns a list of Buckets in a Volume.
+   *
+   * @return List of Buckets
+   *
+   * @throws OzoneException
+   */
+  Response getBucketsInVolume(VolumeArgs args) throws OzoneException {
+    String requestID = OzoneUtils.getRequestID();
+    String hostName = OzoneUtils.getHostName();
+    try {
+      UserAuth auth = UserHandlerBuilder.getAuthHandler();
+      // TODO : Check for ACLS access.
+      StorageHandler fs = StorageHandlerBuilder.getStorageHandler();
+      ListBuckets bucketList = fs.listBuckets(args);
+      return OzoneUtils.getResponse(args, HTTP_OK, bucketList.toJsonString());
+    } catch (IOException ex) {
+      OzoneException exp =
+          ErrorTable.newError(ErrorTable.SERVER_ERROR, requestID, "", hostName);
+      exp.setMessage("unable to get the bucket list for the specified volume.");
+      throw exp;
+
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e4fb9bdd/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/request/OzoneAcl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/request/OzoneAcl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/request/OzoneAcl.java
index 521c277..cdeb6c6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/request/OzoneAcl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/request/OzoneAcl.java
@@ -38,6 +38,12 @@ public class OzoneAcl {
 
   /**
    * Constructor for OzoneAcl.
+   */
+  public OzoneAcl() {
+  }
+
+  /**
+   * Constructor for OzoneAcl.
    *
    * @param type - Type
    * @param name - Name of user

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e4fb9bdd/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/response/ListBuckets.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/response/ListBuckets.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/response/ListBuckets.java
index 18b166c..62bd491 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/response/ListBuckets.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/response/ListBuckets.java
@@ -75,7 +75,7 @@ public class ListBuckets {
    * @return Bucket list
    */
   public List<BucketInfo> getBuckets() {
-    return Collections.unmodifiableList(buckets);
+    return buckets;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e4fb9bdd/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java
new file mode 100644
index 0000000..3cab268
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java
@@ -0,0 +1,174 @@
+/*
+ * 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.fs.StorageType;
+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.OzoneUtils;
+import org.junit.AfterClass;
+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.assertNotNull;
+import static org.junit.Assert.fail;
+
+public class TestBuckets {
+
+  static MiniDFSCluster cluster = null;
+  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,
+      URISyntaxException, OzoneException {
+    OzoneConfiguration conf = new OzoneConfiguration();
+
+    URL p = conf.getClass().getResource("");
+    String path = p.getPath();
+    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");
+
+    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 testCreateBucket() throws OzoneException, IOException {
+    String volumeName = OzoneUtils.getRequestID().toLowerCase();
+    client.setUserAuth("hdfs");
+    OzoneVolume vol = client.createVolume(volumeName, "bilbo", "100TB");
+    String[] acls = {"user:frodo:rw", "user:samwise:rw"};
+
+    // create 10 buckets under same volume
+    for (int x = 0; x < 10; x++) {
+      String bucketName = OzoneUtils.getRequestID().toLowerCase();
+      OzoneBucket bucket =
+          vol.createBucket(bucketName, acls, StorageType.DEFAULT);
+      assertEquals(bucket.getBucketName(), bucketName);
+    }
+    client.close();
+
+    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 testAddBucketAcls() throws OzoneException, IOException {
+    String volumeName = OzoneUtils.getRequestID().toLowerCase();
+    client.setUserAuth("hdfs");
+    OzoneVolume vol = client.createVolume(volumeName, "bilbo", "100TB");
+    String[] acls = {"user:frodo:rw", "user:samwise:rw"};
+    String bucketName = OzoneUtils.getRequestID().toLowerCase();
+    vol.createBucket(bucketName);
+    vol.addAcls(bucketName, acls);
+    OzoneBucket updatedBucket = vol.getBucket(bucketName);
+    assertEquals(updatedBucket.getAcls().size(), 2);
+    client.close();
+  }
+
+  @Test
+  public void testRemoveBucketAcls() throws OzoneException, IOException {
+    String volumeName = OzoneUtils.getRequestID().toLowerCase();
+    client.setUserAuth("hdfs");
+    OzoneVolume vol = client.createVolume(volumeName, "bilbo", "100TB");
+    String[] acls = {"user:frodo:rw", "user:samwise:rw"};
+    String bucketName = OzoneUtils.getRequestID().toLowerCase();
+    OzoneBucket bucket = vol.createBucket(bucketName, acls);
+    assertEquals(bucket.getAcls().size(), 2);
+    vol.removeAcls(bucketName, acls);
+    OzoneBucket updatedBucket = vol.getBucket(bucketName);
+
+    // We removed all acls
+    assertEquals(updatedBucket.getAcls().size(), 0);
+    client.close();
+  }
+
+  @Test
+  public void testDeleteBucket() throws OzoneException, IOException {
+    String volumeName = OzoneUtils.getRequestID().toLowerCase();
+    client.setUserAuth("hdfs");
+    OzoneVolume vol = client.createVolume(volumeName, "bilbo", "100TB");
+    String[] acls = {"user:frodo:rw", "user:samwise:rw"};
+    String bucketName = OzoneUtils.getRequestID().toLowerCase();
+    vol.createBucket(bucketName, acls);
+    vol.deleteBucket(bucketName);
+    try {
+      OzoneBucket updatedBucket = vol.getBucket(bucketName);
+      fail("Fetching deleted bucket, Should not reach here.");
+    } catch (Exception ex) {
+      // must throw
+      assertNotNull(ex);
+    }
+    client.close();
+  }
+
+  @Test
+  public void testListBucket() throws OzoneException, IOException {
+    String volumeName = OzoneUtils.getRequestID().toLowerCase();
+    client.setUserAuth("hdfs");
+    OzoneVolume vol = client.createVolume(volumeName, "bilbo", "100TB");
+    String[] acls = {"user:frodo:rw", "user:samwise:rw"};
+    for (int x = 0; x < 10; x++) {
+      String bucketName = OzoneUtils.getRequestID().toLowerCase();
+      vol.createBucket(bucketName, acls);
+    }
+    List<OzoneBucket> bucketList = vol.listBuckets();
+    assertEquals(bucketList.size(), 10);
+    client.close();
+  }
+}