You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ozone.apache.org by xy...@apache.org on 2021/04/28 15:03:06 UTC

[ozone] branch master updated: HDDS-4585. Support bucket acl operation in S3g (#1701)

This is an automated email from the ASF dual-hosted git repository.

xyao pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ozone.git


The following commit(s) were added to refs/heads/master by this push:
     new 66a411b  HDDS-4585. Support bucket acl operation in S3g (#1701)
66a411b is described below

commit 66a411be58843eac7cbc7b32860b494707c9dbd6
Author: Sammi Chen <sa...@apache.org>
AuthorDate: Wed Apr 28 23:02:43 2021 +0800

    HDDS-4585. Support bucket acl operation in S3g (#1701)
---
 .../apache/hadoop/ozone/client/OzoneBucket.java    |  15 +-
 .../apache/hadoop/ozone/client/OzoneVolume.java    |  57 +++-
 .../hadoop/ozone/om/helpers/OzoneAclUtil.java      |  20 ++
 .../ozone/security/acl/IAccessAuthorizer.java      |   1 -
 .../client/rpc/TestOzoneRpcClientAbstract.java     |  12 +-
 .../hadoop/ozone/s3/endpoint/BucketEndpoint.java   | 283 +++++++++++++++--
 .../s3/endpoint/ListMultipartUploadsResult.java    |  53 +---
 .../endpoint/PutBucketAclRequestUnmarshaller.java  |  85 +++++
 .../org/apache/hadoop/ozone/s3/endpoint/S3Acl.java | 342 +++++++++++++++++++++
 .../hadoop/ozone/s3/endpoint/S3BucketAcl.java      | 235 ++++++++++++++
 .../apache/hadoop/ozone/s3/endpoint/S3Owner.java   |  72 +++++
 .../hadoop/ozone/s3/exception/S3ErrorTable.java    |   5 +
 .../hadoop/ozone/client/OzoneBucketStub.java       |  23 ++
 .../hadoop/ozone/client/OzoneVolumeStub.java       |  24 ++
 .../hadoop/ozone/s3/endpoint/TestBucketAcl.java    | 266 ++++++++++++++++
 .../{TestBucketGet.java => TestBucketList.java}    |  76 ++---
 .../hadoop/ozone/s3/endpoint/TestBucketPut.java    |   6 +-
 .../ozone/s3/endpoint/TestPermissionCheck.java     |  56 +++-
 .../src/test/resources/groupAccessControlList.xml  |  57 ++++
 .../src/test/resources/userAccessControlList.xml   |  42 +++
 20 files changed, 1605 insertions(+), 125 deletions(-)

diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
index f688a66..edcaec0 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
@@ -386,7 +386,7 @@ public class OzoneBucket extends WithMetadata {
    * for the bucket.
    * @throws IOException
    */
-  public boolean addAcls(OzoneAcl addAcl) throws IOException {
+  public boolean addAcl(OzoneAcl addAcl) throws IOException {
     return proxy.addAcl(ozoneObj, addAcl);
   }
 
@@ -396,11 +396,22 @@ public class OzoneBucket extends WithMetadata {
    * removed does not exist for the bucket.
    * @throws IOException
    */
-  public boolean removeAcls(OzoneAcl removeAcl) throws IOException {
+  public boolean removeAcl(OzoneAcl removeAcl) throws IOException {
     return proxy.removeAcl(ozoneObj, removeAcl);
   }
 
   /**
+   * Acls to be set for given Ozone object. This operations reset ACL for
+   * given object to list of ACLs provided in argument.
+   * @param acls List of acls.
+   *
+   * @throws IOException if there is error.
+   * */
+  public boolean setAcl(List<OzoneAcl> acls) throws IOException {
+    return proxy.setAcl(ozoneObj, acls);
+  }
+
+  /**
    * Sets/Changes the storage type of the bucket.
    * @param newStorageType Storage type to be set
    * @throws IOException
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneVolume.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneVolume.java
index 369500f..3847b12 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneVolume.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneVolume.java
@@ -26,6 +26,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.NoSuchElementException;
 
+import org.apache.commons.collections.ListUtils;
 import org.apache.hadoop.hdds.client.OzoneQuota;
 import org.apache.hadoop.hdds.conf.ConfigurationSource;
 import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
@@ -35,6 +36,8 @@ import org.apache.hadoop.ozone.om.helpers.WithMetadata;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.ozone.security.acl.OzoneObjInfo;
 
 import static org.apache.hadoop.ozone.OzoneConsts.QUOTA_RESET;
 
@@ -89,6 +92,8 @@ public class OzoneVolume extends WithMetadata {
 
   private int listCacheSize;
 
+  private OzoneObj ozoneObj;
+
   /**
    * Constructs OzoneVolume instance.
    * @param conf Configuration object.
@@ -122,6 +127,10 @@ public class OzoneVolume extends WithMetadata {
       modificationTime = Instant.ofEpochSecond(
           this.creationTime.getEpochSecond(), this.creationTime.getNano());
     }
+    this.ozoneObj = OzoneObjInfo.Builder.newBuilder()
+        .setVolumeName(name)
+        .setResType(OzoneObj.ResourceType.VOLUME)
+        .setStoreType(OzoneObj.StoreType.OZONE).build();
   }
 
   /**
@@ -261,7 +270,53 @@ public class OzoneVolume extends WithMetadata {
    * @return aclMap
    */
   public List<OzoneAcl> getAcls() {
-    return acls;
+    return ListUtils.unmodifiableList(acls);
+  }
+
+   /**
+   * Adds ACLs to the volume.
+   * @param addAcl ACL to be added
+   * @return true - if acl is successfully added, false if acl already exists
+   * for the bucket.
+   * @throws IOException
+   */
+  public boolean addAcl(OzoneAcl addAcl) throws IOException {
+    boolean added = proxy.addAcl(ozoneObj, addAcl);
+    if (added) {
+      acls.add(addAcl);
+    }
+    return added;
+  }
+
+  /**
+   * Remove acl for Ozone object. Return true if acl is removed successfully
+   * else false.
+   * @param acl Ozone acl to be removed.
+   *
+   * @throws IOException if there is error.
+   * */
+  public boolean removeAcl(OzoneAcl acl) throws IOException {
+    boolean removed = proxy.removeAcl(ozoneObj, acl);
+    if (removed) {
+      acls.remove(acl);
+    }
+    return removed;
+  }
+
+  /**
+   * Acls to be set for given Ozone object. This operations reset ACL for
+   * given object to list of ACLs provided in argument.
+   * @param aclList List of acls.
+   *
+   * @throws IOException if there is error.
+   * */
+  public boolean setAcl(List<OzoneAcl> aclList) throws IOException {
+    boolean reset = proxy.setAcl(ozoneObj, aclList);
+    if (reset) {
+      acls.clear();
+      acls.addAll(aclList);
+    }
+    return reset;
   }
 
   /**
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OzoneAclUtil.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OzoneAclUtil.java
index 0c15d79..9901ae7 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OzoneAclUtil.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OzoneAclUtil.java
@@ -69,6 +69,26 @@ public final class OzoneAclUtil {
   }
 
   /**
+   * Helper function to get acl list for one user/group.
+   *
+   * @param identityName
+   * @param type
+   * @param aclList
+   * @return list of OzoneAcls
+   * */
+  public static List<OzoneAcl> filterAclList(String identityName,
+      IAccessAuthorizer.ACLIdentityType type, List<OzoneAcl> aclList) {
+
+    if (aclList == null || aclList.isEmpty()) {
+      return new ArrayList<>();
+    }
+
+    List retList = aclList.stream().filter(acl -> acl.getType() == type
+        && acl.getName().equals(identityName)).collect(Collectors.toList());
+    return retList;
+  }
+
+  /**
    * Check if acl right requested for given RequestContext exist
    * in provided acl list.
    * Acl validation rules:
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/security/acl/IAccessAuthorizer.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/security/acl/IAccessAuthorizer.java
index 6e4581e..3523618 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/security/acl/IAccessAuthorizer.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/security/acl/IAccessAuthorizer.java
@@ -145,7 +145,6 @@ public interface IAccessAuthorizer {
         throw new IllegalArgumentException("ACL right is not recognized");
       }
     }
-
   }
 
   /**
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java
index b40d8a1..09b4145 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java
@@ -656,7 +656,7 @@ public abstract class TestOzoneRpcClientAbstract {
     acls.add(new OzoneAcl(USER, "test", ACLType.ALL, ACCESS));
     OzoneBucket bucket = volume.getBucket(bucketName);
     for (OzoneAcl acl : acls) {
-      assertTrue(bucket.addAcls(acl));
+      assertTrue(bucket.addAcl(acl));
     }
     OzoneBucket newBucket = volume.getBucket(bucketName);
     Assert.assertEquals(bucketName, newBucket.getName());
@@ -679,7 +679,7 @@ public abstract class TestOzoneRpcClientAbstract {
     volume.createBucket(bucketName, builder.build());
     OzoneBucket bucket = volume.getBucket(bucketName);
     for (OzoneAcl acl : acls) {
-      assertTrue(bucket.removeAcls(acl));
+      assertTrue(bucket.removeAcl(acl));
     }
     OzoneBucket newBucket = volume.getBucket(bucketName);
     Assert.assertEquals(bucketName, newBucket.getName());
@@ -2367,10 +2367,10 @@ public abstract class TestOzoneRpcClientAbstract {
     OzoneAcl acl2 = new OzoneAcl(USER, "Friday", ACLType.ALL, DEFAULT);
     OzoneAcl acl3 = new OzoneAcl(USER, "Jan", ACLType.ALL, ACCESS);
     OzoneAcl acl4 = new OzoneAcl(USER, "Feb", ACLType.ALL, ACCESS);
-    bucket.addAcls(acl1);
-    bucket.addAcls(acl2);
-    bucket.addAcls(acl3);
-    bucket.addAcls(acl4);
+    bucket.addAcl(acl1);
+    bucket.addAcl(acl2);
+    bucket.addAcl(acl3);
+    bucket.addAcl(acl4);
 
     doMultipartUpload(bucket, keyName, (byte)98);
     OzoneObj keyObj = OzoneObjInfo.Builder.newBuilder()
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java
index b8bed64..72a95ce 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java
@@ -17,6 +17,31 @@
  */
 package org.apache.hadoop.ozone.s3.endpoint;
 
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneKey;
+import org.apache.hadoop.ozone.client.OzoneMultipartUploadList;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
+import org.apache.hadoop.ozone.om.helpers.OzoneAclUtil;
+import org.apache.hadoop.ozone.s3.commontypes.KeyMetadata;
+import org.apache.hadoop.ozone.s3.endpoint.MultiDeleteRequest.DeleteObject;
+import org.apache.hadoop.ozone.s3.endpoint.MultiDeleteResponse.DeletedObject;
+import org.apache.hadoop.ozone.s3.endpoint.MultiDeleteResponse.Error;
+import org.apache.hadoop.ozone.s3.endpoint.S3BucketAcl.Grant;
+import org.apache.hadoop.ozone.s3.exception.OS3Exception;
+import org.apache.hadoop.ozone.s3.exception.S3ErrorTable;
+import org.apache.hadoop.ozone.s3.util.ContinueToken;
+import org.apache.hadoop.ozone.s3.util.S3StorageType;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.http.HttpStatus;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import javax.ws.rs.DELETE;
 import javax.ws.rs.DefaultValue;
 import javax.ws.rs.GET;
@@ -33,29 +58,16 @@ import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 import java.io.IOException;
 import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashSet;
 import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
 
-import org.apache.hadoop.hdds.client.ReplicationType;
-import org.apache.hadoop.ozone.client.OzoneBucket;
-import org.apache.hadoop.ozone.client.OzoneKey;
-import org.apache.hadoop.ozone.client.OzoneMultipartUploadList;
-import org.apache.hadoop.ozone.om.exceptions.OMException;
-import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
-import org.apache.hadoop.ozone.s3.commontypes.KeyMetadata;
-import org.apache.hadoop.ozone.s3.endpoint.MultiDeleteRequest.DeleteObject;
-import org.apache.hadoop.ozone.s3.endpoint.MultiDeleteResponse.DeletedObject;
-import org.apache.hadoop.ozone.s3.endpoint.MultiDeleteResponse.Error;
-import org.apache.hadoop.ozone.s3.exception.OS3Exception;
-import org.apache.hadoop.ozone.s3.exception.S3ErrorTable;
-import org.apache.hadoop.ozone.s3.util.ContinueToken;
-import org.apache.hadoop.ozone.s3.util.S3StorageType;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-import org.apache.commons.lang3.StringUtils;
+import static org.apache.hadoop.ozone.OzoneAcl.AclScope.ACCESS;
+import static org.apache.hadoop.ozone.s3.exception.S3ErrorTable.NOT_IMPLEMENTED;
 import static org.apache.hadoop.ozone.s3.util.S3Consts.ENCODING_TYPE;
-import org.apache.http.HttpStatus;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * Bucket level rest endpoints.
@@ -75,7 +87,7 @@ public class BucketEndpoint extends EndpointBase {
   @GET
   @SuppressFBWarnings
   @SuppressWarnings("parameternumber")
-  public Response list(
+  public Response get(
       @PathParam("bucket") String bucketName,
       @QueryParam("delimiter") String delimiter,
       @QueryParam("encoding-type") String encodingType,
@@ -86,8 +98,14 @@ public class BucketEndpoint extends EndpointBase {
       @QueryParam("continuation-token") String continueToken,
       @QueryParam("start-after") String startAfter,
       @QueryParam("uploads") String uploads,
+      @QueryParam("acl") String aclMarker,
       @Context HttpHeaders hh) throws OS3Exception, IOException {
 
+    if (aclMarker != null) {
+      S3BucketAcl result = getAcl(bucketName);
+      return Response.ok(result, MediaType.APPLICATION_XML_TYPE).build();
+    }
+
     if (browser != null) {
       InputStream browserPage = getClass()
           .getResourceAsStream("/browser.html");
@@ -210,9 +228,13 @@ public class BucketEndpoint extends EndpointBase {
   }
 
   @PUT
-  public Response put(@PathParam("bucket") String bucketName, @Context
-      HttpHeaders httpHeaders) throws IOException, OS3Exception {
-
+  public Response put(@PathParam("bucket") String bucketName,
+      @QueryParam("acl") String aclMarker,
+      @Context HttpHeaders httpHeaders,
+      InputStream body) throws IOException, OS3Exception {
+    if (aclMarker != null) {
+      return putAcl(bucketName, httpHeaders, body);
+    }
     try {
       String location = createS3Bucket(bucketName);
       LOG.info("Location is {}", location);
@@ -353,6 +375,219 @@ public class BucketEndpoint extends EndpointBase {
     return result;
   }
 
+  /**
+   * Implement acl get.
+   * <p>
+   * see: https://docs.aws.amazon.com/AmazonS3/latest/API/API_GetBucketAcl.html
+   */
+  public S3BucketAcl getAcl(String bucketName)
+      throws OS3Exception, IOException {
+    S3BucketAcl result = new S3BucketAcl();
+    try {
+      OzoneBucket bucket = getBucket(bucketName);
+      OzoneVolume volume = getVolume();
+      // TODO: use bucket owner instead of volume owner here once bucket owner
+      // TODO: is supported.
+      S3Owner owner = new S3Owner(volume.getOwner(), volume.getOwner());
+      result.setOwner(owner);
+
+      // TODO: remove this duplication avoid logic when ACCESS and DEFAULT scope
+      // TODO: are merged.
+      // Use set to remove ACLs with different scopes(ACCESS and DEFAULT)
+      Set<Grant> grantSet = new HashSet<>();
+      // Return ACL list
+      for (OzoneAcl acl : bucket.getAcls()) {
+        List<Grant> grants = S3Acl.ozoneNativeAclToS3Acl(acl);
+        grantSet.addAll(grants);
+      }
+      ArrayList<Grant> grantList = new ArrayList<>();
+      grantList.addAll(grantSet);
+      result.setAclList(
+          new S3BucketAcl.AccessControlList(grantList));
+      return result;
+    } catch (OMException ex) {
+      if (ex.getResult() == ResultCodes.BUCKET_NOT_FOUND) {
+        throw S3ErrorTable.newError(S3ErrorTable
+            .NO_SUCH_BUCKET, bucketName);
+      } else if (ex.getResult() == ResultCodes.PERMISSION_DENIED) {
+        throw S3ErrorTable.newError(S3ErrorTable
+            .ACCESS_DENIED, bucketName);
+      } else {
+        LOG.error("Failed to get acl of Bucket " + bucketName, ex);
+        throw S3ErrorTable.newError(S3ErrorTable.INTERNAL_ERROR, bucketName);
+      }
+    }
+  }
+
+  /**
+   * Implement acl put.
+   * <p>
+   * see: https://docs.aws.amazon.com/AmazonS3/latest/API/API_PutBucketAcl.html
+   */
+  public Response putAcl(String bucketName, HttpHeaders httpHeaders,
+      InputStream body) throws IOException, OS3Exception {
+    String grantReads = httpHeaders.getHeaderString(S3Acl.GRANT_READ);
+    String grantWrites = httpHeaders.getHeaderString(S3Acl.GRANT_WRITE);
+    String grantReadACP = httpHeaders.getHeaderString(S3Acl.GRANT_READ_CAP);
+    String grantWriteACP = httpHeaders.getHeaderString(S3Acl.GRANT_WRITE_CAP);
+    String grantFull = httpHeaders.getHeaderString(S3Acl.GRANT_FULL_CONTROL);
+
+    try {
+      OzoneBucket bucket = getBucket(bucketName);
+      OzoneVolume volume = getVolume();
+
+      List<OzoneAcl> ozoneAclListOnBucket = new ArrayList<>();
+      List<OzoneAcl> ozoneAclListOnVolume = new ArrayList<>();
+
+      if (grantReads == null && grantWrites == null && grantReadACP == null
+          && grantWriteACP == null && grantFull == null) {
+        S3BucketAcl putBucketAclRequest =
+            new PutBucketAclRequestUnmarshaller().readFrom(
+            null, null, null, null, null, body);
+        // Handle grants in body
+        ozoneAclListOnBucket.addAll(
+            S3Acl.s3AclToOzoneNativeAclOnBucket(putBucketAclRequest));
+        ozoneAclListOnVolume.addAll(
+            S3Acl.s3AclToOzoneNativeAclOnVolume(putBucketAclRequest));
+      } else {
+
+        // Handle grants in headers
+        if (grantReads != null) {
+          ozoneAclListOnBucket.addAll(getAndConvertAclOnBucket(grantReads,
+              S3Acl.ACLType.READ.getValue()));
+          ozoneAclListOnVolume.addAll(getAndConvertAclOnVolume(grantReads,
+              S3Acl.ACLType.READ.getValue()));
+        }
+        if (grantWrites != null) {
+          ozoneAclListOnBucket.addAll(getAndConvertAclOnBucket(grantWrites,
+              S3Acl.ACLType.WRITE.getValue()));
+          ozoneAclListOnVolume.addAll(getAndConvertAclOnVolume(grantWrites,
+              S3Acl.ACLType.WRITE.getValue()));
+        }
+        if (grantReadACP != null) {
+          ozoneAclListOnBucket.addAll(getAndConvertAclOnBucket(grantReadACP,
+              S3Acl.ACLType.READ_ACP.getValue()));
+          ozoneAclListOnVolume.addAll(getAndConvertAclOnVolume(grantReadACP,
+              S3Acl.ACLType.READ_ACP.getValue()));
+        }
+        if (grantWriteACP != null) {
+          ozoneAclListOnBucket.addAll(getAndConvertAclOnBucket(grantWriteACP,
+              S3Acl.ACLType.WRITE_ACP.getValue()));
+          ozoneAclListOnVolume.addAll(getAndConvertAclOnVolume(grantWriteACP,
+              S3Acl.ACLType.WRITE_ACP.getValue()));
+        }
+        if (grantFull != null) {
+          ozoneAclListOnBucket.addAll(getAndConvertAclOnBucket(grantFull,
+              S3Acl.ACLType.FULL_CONTROL.getValue()));
+          ozoneAclListOnVolume.addAll(getAndConvertAclOnVolume(grantFull,
+              S3Acl.ACLType.FULL_CONTROL.getValue()));
+        }
+      }
+
+      // A put request will reset all previous ACLs on bucket
+      bucket.setAcl(ozoneAclListOnBucket);
+      // A put request will reset input user/group's permission on volume
+      List<OzoneAcl> acls = bucket.getAcls();
+      List<OzoneAcl> aclsToRemoveOnVolume = new ArrayList<>();
+      List<OzoneAcl> currentAclsOnVolume = volume.getAcls();
+      // Remove input user/group's permission from Volume first
+      if (currentAclsOnVolume.size() > 0) {
+        for (OzoneAcl acl : acls) {
+          if (acl.getAclScope() == ACCESS) {
+            aclsToRemoveOnVolume.addAll(OzoneAclUtil.filterAclList(
+                acl.getName(), acl.getType(), currentAclsOnVolume));
+          }
+        }
+        for (OzoneAcl acl : aclsToRemoveOnVolume) {
+          volume.removeAcl(acl);
+        }
+      }
+      // Add new permission on Volume
+      for(OzoneAcl acl : ozoneAclListOnVolume) {
+        volume.addAcl(acl);
+      }
+    } catch (OMException exception) {
+      LOG.error("Error in set ACL Request for bucket: {}", bucketName,
+          exception);
+      if (exception.getResult() == ResultCodes.BUCKET_NOT_FOUND) {
+        throw S3ErrorTable.newError(S3ErrorTable.NO_SUCH_BUCKET,
+            bucketName);
+      } else if (exception.getResult() == ResultCodes.PERMISSION_DENIED) {
+        throw S3ErrorTable.newError(S3ErrorTable
+            .ACCESS_DENIED, bucketName);
+      }
+      throw exception;
+    }
+    return Response.status(HttpStatus.SC_OK).build();
+  }
+
+  /**
+   *  Example: x-amz-grant-write: \
+   *  uri="http://acs.amazonaws.com/groups/s3/LogDelivery", id="111122223333", \
+   *  id="555566667777".
+   */
+  private List<OzoneAcl> getAndConvertAclOnBucket(String value,
+      String permission) throws OS3Exception {
+    List<OzoneAcl> ozoneAclList = new ArrayList<>();
+    if (StringUtils.isEmpty(value)) {
+      return ozoneAclList;
+    }
+    String[] subValues = value.split(",");
+    for (String acl: subValues) {
+      String[] part = acl.split("=");
+      if (part.length != 2) {
+        throw S3ErrorTable.newError(S3ErrorTable.INVALID_ARGUMENT, acl);
+      }
+      S3Acl.ACLIdentityType type =
+          S3Acl.ACLIdentityType.getTypeFromHeaderType(part[0]);
+      if (type == null || !type.isSupported()) {
+        LOG.warn("S3 grantee {} is null or not supported", part[0]);
+        throw S3ErrorTable.newError(NOT_IMPLEMENTED, part[0]);
+      }
+      // Build ACL on Bucket
+      BitSet aclsOnBucket =
+          S3Acl.getOzoneAclOnBucketFromS3Permission(permission);
+      OzoneAcl defaultOzoneAcl = new OzoneAcl(
+          IAccessAuthorizer.ACLIdentityType.USER, part[1], aclsOnBucket,
+          OzoneAcl.AclScope.DEFAULT);
+      OzoneAcl accessOzoneAcl = new OzoneAcl(
+          IAccessAuthorizer.ACLIdentityType.USER, part[1], aclsOnBucket,
+          ACCESS);
+      ozoneAclList.add(defaultOzoneAcl);
+      ozoneAclList.add(accessOzoneAcl);
+    }
+    return ozoneAclList;
+  }
+
+  private List<OzoneAcl> getAndConvertAclOnVolume(String value,
+      String permission) throws OS3Exception {
+    List<OzoneAcl> ozoneAclList = new ArrayList<>();
+    if (StringUtils.isEmpty(value)) {
+      return ozoneAclList;
+    }
+    String[] subValues = value.split(",");
+    for (String acl: subValues) {
+      String[] part = acl.split("=");
+      if (part.length != 2) {
+        throw S3ErrorTable.newError(S3ErrorTable.INVALID_ARGUMENT, acl);
+      }
+      S3Acl.ACLIdentityType type =
+          S3Acl.ACLIdentityType.getTypeFromHeaderType(part[0]);
+      if (type == null || !type.isSupported()) {
+        LOG.warn("S3 grantee {} is null or not supported", part[0]);
+        throw S3ErrorTable.newError(NOT_IMPLEMENTED, part[0]);
+      }
+      // Build ACL on Volume
+      BitSet aclsOnVolume =
+          S3Acl.getOzoneAclOnVolumeFromS3Permission(permission);
+      OzoneAcl accessOzoneAcl = new OzoneAcl(
+          IAccessAuthorizer.ACLIdentityType.USER, part[1], aclsOnVolume,
+          ACCESS);
+      ozoneAclList.add(accessOzoneAcl);
+    }
+    return ozoneAclList;
+  }
+
   private void addKey(ListObjectResponse response, OzoneKey next) {
     KeyMetadata keyMetadata = new KeyMetadata();
     keyMetadata.setKey(next.getName());
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/ListMultipartUploadsResult.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/ListMultipartUploadsResult.java
index 20dc9cd..46297f7 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/ListMultipartUploadsResult.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/ListMultipartUploadsResult.java
@@ -38,9 +38,6 @@ import org.apache.hadoop.ozone.s3.util.S3StorageType;
     "http://s3.amazonaws.com/doc/2006-03-01/")
 public class ListMultipartUploadsResult {
 
-  public static final Owner
-      NOT_SUPPORTED_OWNER = new Owner("NOT-SUPPORTED", "Not Supported");
-
   @XmlElement(name = "Bucket")
   private String bucket;
 
@@ -148,10 +145,10 @@ public class ListMultipartUploadsResult {
     private String uploadId;
 
     @XmlElement(name = "Owner")
-    private Owner owner = NOT_SUPPORTED_OWNER;
+    private S3Owner owner = S3Owner.NOT_SUPPORTED_OWNER;
 
     @XmlElement(name = "Initiator")
-    private Owner initiator = NOT_SUPPORTED_OWNER;
+    private S3Owner initiator = S3Owner.NOT_SUPPORTED_OWNER;
 
     @XmlElement(name = "StorageClass")
     private String storageClass = "STANDARD";
@@ -193,21 +190,21 @@ public class ListMultipartUploadsResult {
       this.uploadId = uploadId;
     }
 
-    public Owner getOwner() {
+    public S3Owner getOwner() {
       return owner;
     }
 
     public void setOwner(
-        Owner owner) {
+        S3Owner owner) {
       this.owner = owner;
     }
 
-    public Owner getInitiator() {
+    public S3Owner getInitiator() {
       return initiator;
     }
 
     public void setInitiator(
-        Owner initiator) {
+        S3Owner initiator) {
       this.initiator = initiator;
     }
 
@@ -227,42 +224,4 @@ public class ListMultipartUploadsResult {
       this.initiated = initiated;
     }
   }
-
-  /**
-   * Upload information.
-   */
-  @XmlAccessorType(XmlAccessType.FIELD)
-  @XmlRootElement(name = "Owner")
-  public static class Owner {
-
-    @XmlElement(name = "ID")
-    private String id;
-
-    @XmlElement(name = "DisplayName")
-    private String displayName;
-
-    public Owner() {
-    }
-
-    public Owner(String id, String displayName) {
-      this.id = id;
-      this.displayName = displayName;
-    }
-
-    public String getId() {
-      return id;
-    }
-
-    public void setId(String id) {
-      this.id = id;
-    }
-
-    public String getDisplayName() {
-      return displayName;
-    }
-
-    public void setDisplayName(String displayName) {
-      this.displayName = displayName;
-    }
-  }
 }
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/PutBucketAclRequestUnmarshaller.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/PutBucketAclRequestUnmarshaller.java
new file mode 100644
index 0000000..3ca2e47
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/PutBucketAclRequestUnmarshaller.java
@@ -0,0 +1,85 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.s3.endpoint;
+
+import org.xml.sax.InputSource;
+import org.xml.sax.XMLReader;
+
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.ext.MessageBodyReader;
+import javax.ws.rs.ext.Provider;
+import javax.xml.XMLConstants;
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.UnmarshallerHandler;
+import javax.xml.parsers.SAXParserFactory;
+import java.io.IOException;
+import java.io.InputStream;
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Type;
+
+import static org.apache.hadoop.ozone.s3.util.S3Consts.S3_XML_NAMESPACE;
+
+/**
+ * Custom unmarshaller to read PutBucketAclRequest wo namespace.
+ */
+@Provider
+public class PutBucketAclRequestUnmarshaller
+    implements MessageBodyReader<S3BucketAcl> {
+
+  private final JAXBContext context;
+  private final XMLReader xmlReader;
+
+  public PutBucketAclRequestUnmarshaller() {
+    try {
+      context = JAXBContext.newInstance(S3BucketAcl.class);
+      SAXParserFactory saxParserFactory = SAXParserFactory.newInstance();
+      saxParserFactory.setFeature(XMLConstants.FEATURE_SECURE_PROCESSING, true);
+      xmlReader = saxParserFactory.newSAXParser().getXMLReader();
+    } catch (Exception ex) {
+      throw new AssertionError("Can not instantiate " +
+          "PutBucketAclRequest parser", ex);
+    }
+  }
+  @Override
+  public boolean isReadable(Class<?> aClass, Type type,
+      Annotation[] annotations, MediaType mediaType) {
+    return type.equals(S3BucketAcl.class);
+  }
+
+  @Override
+  public S3BucketAcl readFrom(
+      Class<S3BucketAcl> aClass, Type type,
+      Annotation[] annotations, MediaType mediaType,
+      MultivaluedMap<String, String> multivaluedMap,
+      InputStream inputStream) throws IOException, WebApplicationException {
+    try {
+      UnmarshallerHandler unmarshallerHandler =
+          context.createUnmarshaller().getUnmarshallerHandler();
+      XmlNamespaceFilter filter =
+          new XmlNamespaceFilter(S3_XML_NAMESPACE);
+      filter.setContentHandler(unmarshallerHandler);
+      filter.setParent(xmlReader);
+      filter.parse(new InputSource(inputStream));
+      return (S3BucketAcl)(unmarshallerHandler.getResult());
+    } catch (Exception e) {
+      throw new WebApplicationException("Can't parse request body to XML.", e);
+    }
+  }
+}
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/S3Acl.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/S3Acl.java
new file mode 100644
index 0000000..111f28c
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/S3Acl.java
@@ -0,0 +1,342 @@
+/*
+ * 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.s3.endpoint;
+
+import org.apache.hadoop.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.s3.endpoint.S3BucketAcl.Grant;
+import org.apache.hadoop.ozone.s3.endpoint.S3BucketAcl.Grantee;
+import org.apache.hadoop.ozone.s3.exception.OS3Exception;
+import org.apache.hadoop.ozone.s3.exception.S3ErrorTable;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.s3.exception.S3ErrorTable.INVALID_ARGUMENT;
+import static org.apache.hadoop.ozone.s3.exception.S3ErrorTable.NOT_IMPLEMENTED;
+
+public final class S3Acl {
+  private static final Logger LOG = LoggerFactory.getLogger(S3Acl.class);
+
+  // ACL put related headers
+  public static final String GRANT_READ = "x-amz-grant-read";
+  public static final String GRANT_WRITE = "x-amz-grant-write";
+  public static final String GRANT_READ_CAP = "x-amz-grant-read-acp";
+  public static final String GRANT_WRITE_CAP = "x-amz-grant-write-acp";
+  public static final String GRANT_FULL_CONTROL = "x-amz-grant-full-control";
+
+  // Not supported headers at current stage, may support it in future
+  public static final String CANNED_ACL_HEADER = "x-amz-acl";
+
+  /**
+   * https://docs.aws.amazon.com/AmazonS3/latest/dev/acl-overview.html.
+   */
+  enum ACLType {
+    // Allows grantee to list the objects in the bucket
+    READ("READ"),
+    // Allows grantee to create, overwrite, and delete any object in the bucket
+    WRITE("WRITE"),
+    // Allows grantee to write the bucket ACL
+    READ_ACP("READ_ACP"),
+    // Allows grantee to write the ACL for the applicable bucket
+    WRITE_ACP("WRITE_ACP"),
+    // Allows grantee above all permissions on the bucket
+    FULL_CONTROL("FULL_CONTROL");
+
+    public String getValue() {
+      return value;
+    }
+    /**
+     * String value for this Enum.
+     */
+    private final String value;
+
+    /**
+     * @param val String type for this enum.
+     */
+    ACLType(String val) {
+      value = val;
+    }
+
+
+    public static ACLType getType(String typeStr) {
+      for(ACLType type: ACLType.values()) {
+        if (type.getValue().equals(typeStr)) {
+          return type;
+        }
+      }
+      return null;
+    }
+  }
+
+  enum ACLIdentityType {
+    USER("CanonicalUser", true, "id"),
+    GROUP("Group", false, "url"),
+    USER_BY_EMAIL("AmazonCustomerByEmail", false, "emailAddress");
+
+    public String getGranteeType() {
+      return granteeType;
+    }
+
+    public String getHeaderType() {
+      return granteeInHeader;
+    }
+
+    /**
+     *  Grantee type in body XML.
+     */
+    private final String granteeType;
+
+    /**
+     * Is this type supported or not.
+     */
+    private final boolean supported;
+
+    /**
+     * Grantee type in header.
+     */
+    private final String granteeInHeader;
+
+    /**
+     * Init OzoneACLtypes enum.
+     *
+     * @param val String type for this enum.
+     */
+    ACLIdentityType(String val, boolean support, String headerType) {
+      granteeType = val;
+      supported = support;
+      granteeInHeader = headerType;
+    }
+
+    boolean isSupported() {
+      return supported;
+    }
+
+    public static ACLIdentityType getTypeFromGranteeType(String typeStr) {
+      for(ACLIdentityType type: ACLIdentityType.values()) {
+        if (type.getGranteeType().equals(typeStr)) {
+          return type;
+        }
+      }
+      return null;
+    }
+
+    public static ACLIdentityType getTypeFromHeaderType(String typeStr) {
+      for(ACLIdentityType type: ACLIdentityType.values()) {
+        if (type.getHeaderType().equals(typeStr)) {
+          return type;
+        }
+      }
+      return null;
+    }
+  }
+
+  private S3Acl() {
+  }
+
+  public static boolean isGranteeTypeSupported(String typeStr) {
+    ACLIdentityType type =  ACLIdentityType.getTypeFromGranteeType(typeStr);
+    return type == null ? false : type.isSupported();
+  }
+
+  public static boolean isHeaderTypeSupported(String typeStr) {
+    ACLIdentityType type =  ACLIdentityType.getTypeFromHeaderType(typeStr);
+    return type == null ? false : type.isSupported();
+  }
+
+  public static List<Grant> ozoneNativeAclToS3Acl(OzoneAcl ozoneAcl) {
+    // Since currently only "CanonicalUser" is supported, which maps to Ozone
+    // "USER"
+    List<Grant> grantList = new ArrayList<>();
+    if (ozoneAcl.getType() != IAccessAuthorizer.ACLIdentityType.USER) {
+      return grantList;
+    }
+
+    Grantee grantee = new Grantee();
+    grantee.setDisplayName(ozoneAcl.getName());
+    grantee.setId(ozoneAcl.getName());
+
+    List<IAccessAuthorizer.ACLType> acls = ozoneAcl.getAclList();
+    if (acls.contains(IAccessAuthorizer.ACLType.ALL)) {
+      Grant grant = new Grant();
+      grant.setGrantee(grantee);
+      grant.setPermission(ACLType.FULL_CONTROL.toString());
+      grantList.add(grant);
+    } else if (acls.contains(IAccessAuthorizer.ACLType.WRITE_ACL)) {
+      Grant grant = new Grant();
+      grant.setGrantee(grantee);
+      grant.setPermission(ACLType.WRITE_ACP.toString());
+      grantList.add(grant);
+    } else if (acls.contains(IAccessAuthorizer.ACLType.READ_ACL)) {
+      Grant grant = new Grant();
+      grant.setGrantee(grantee);
+      grant.setPermission(ACLType.READ_ACP.toString());
+      grantList.add(grant);
+    } else if (acls.contains(IAccessAuthorizer.ACLType.WRITE) &&
+        acls.contains(IAccessAuthorizer.ACLType.DELETE) &&
+        acls.contains(IAccessAuthorizer.ACLType.CREATE)) {
+      Grant grant = new Grant();
+      grant.setGrantee(grantee);
+      grant.setPermission(ACLType.WRITE.toString());
+      grantList.add(grant);
+    } else if (acls.contains(IAccessAuthorizer.ACLType.READ) &&
+        acls.contains(IAccessAuthorizer.ACLType.LIST)) {
+      Grant grant = new Grant();
+      grant.setGrantee(grantee);
+      grant.setPermission(ACLType.READ.toString());
+      grantList.add(grant);
+    } else {
+      LOG.error("Cannot find a good mapping for Ozone ACL {} to S3",
+          ozoneAcl.toString());
+    }
+    return grantList;
+  }
+
+  public static List<OzoneAcl> s3AclToOzoneNativeAclOnBucket(
+      S3BucketAcl bucketAcl) throws OS3Exception {
+    List<OzoneAcl> ozoneAclList = new ArrayList<>();
+    List<Grant> grantList = bucketAcl.getAclList().getGrantList();
+    for (Grant grant : grantList) {
+      //  Only "CanonicalUser" is supported, which maps to Ozone "USER"
+      ACLIdentityType identityType = ACLIdentityType.getTypeFromGranteeType(
+          grant.getGrantee().getXsiType());
+      if (identityType != null && identityType.isSupported()) {
+        String permission = grant.getPermission();
+        BitSet acls = getOzoneAclOnBucketFromS3Permission(permission);
+        OzoneAcl defaultOzoneAcl = new OzoneAcl(
+            IAccessAuthorizer.ACLIdentityType.USER,
+            grant.getGrantee().getId(), acls,
+            OzoneAcl.AclScope.DEFAULT);
+        OzoneAcl accessOzoneAcl = new OzoneAcl(
+            IAccessAuthorizer.ACLIdentityType.USER,
+            grant.getGrantee().getId(), acls,
+            OzoneAcl.AclScope.ACCESS);
+        ozoneAclList.add(defaultOzoneAcl);
+        ozoneAclList.add(accessOzoneAcl);
+      } else {
+        LOG.error("Grantee type {} is not supported",
+            grant.getGrantee().getXsiType());
+        throw S3ErrorTable.newError(NOT_IMPLEMENTED,
+            grant.getGrantee().getXsiType());
+      }
+    }
+    return ozoneAclList;
+  }
+
+  public static BitSet getOzoneAclOnBucketFromS3Permission(String permission)
+      throws OS3Exception {
+    ACLType permissionType = ACLType.getType(permission);
+    if (permissionType == null) {
+      throw S3ErrorTable.newError(S3ErrorTable.INVALID_ARGUMENT, permission);
+    }
+    BitSet acls = new BitSet(IAccessAuthorizer.ACLType.getNoOfAcls());
+    switch (permissionType) {
+    case FULL_CONTROL:
+      acls.set(IAccessAuthorizer.ACLType.ALL.ordinal());
+      break;
+    case WRITE_ACP:
+      acls.set(IAccessAuthorizer.ACLType.WRITE_ACL.ordinal());
+      break;
+    case READ_ACP:
+      acls.set(IAccessAuthorizer.ACLType.READ_ACL.ordinal());
+      break;
+    case WRITE:
+      acls.set(IAccessAuthorizer.ACLType.WRITE.ordinal());
+      acls.set(IAccessAuthorizer.ACLType.DELETE.ordinal());
+      acls.set(IAccessAuthorizer.ACLType.CREATE.ordinal());
+      break;
+    case READ:
+      acls.set(IAccessAuthorizer.ACLType.READ.ordinal());
+      acls.set(IAccessAuthorizer.ACLType.LIST.ordinal());
+      break;
+    default:
+      LOG.error("Failed to recognize S3 permission {}", permission);
+      throw S3ErrorTable.newError(INVALID_ARGUMENT, permission);
+    }
+    return acls;
+  }
+
+  public static List<OzoneAcl> s3AclToOzoneNativeAclOnVolume(
+      S3BucketAcl bucketAcl) throws OS3Exception {
+    List<OzoneAcl> ozoneAclList = new ArrayList<>();
+    List<Grant> grantList = bucketAcl.getAclList().getGrantList();
+    for (Grant grant : grantList) {
+      //  Only "CanonicalUser" is supported, which maps to Ozone "USER"
+      ACLIdentityType identityType = ACLIdentityType.getTypeFromGranteeType(
+          grant.getGrantee().getXsiType());
+      if (identityType != null && identityType.isSupported()) {
+        String permission = grant.getPermission();
+        BitSet acls = getOzoneAclOnVolumeFromS3Permission(permission);
+        OzoneAcl accessOzoneAcl = new OzoneAcl(
+            IAccessAuthorizer.ACLIdentityType.USER,
+            grant.getGrantee().getId(), acls,
+            OzoneAcl.AclScope.ACCESS);
+        ozoneAclList.add(accessOzoneAcl);
+      } else {
+        LOG.error("Grantee type {} is not supported",
+            grant.getGrantee().getXsiType());
+        throw S3ErrorTable.newError(NOT_IMPLEMENTED,
+            grant.getGrantee().getXsiType());
+      }
+    }
+    return ozoneAclList;
+  }
+
+  // User privilege on volume follows the "lest privilege" principle.
+  public static BitSet getOzoneAclOnVolumeFromS3Permission(String permission)
+      throws OS3Exception {
+    BitSet acls = new BitSet(IAccessAuthorizer.ACLType.getNoOfAcls());
+    ACLType permissionType = ACLType.getType(permission);
+    if (permissionType == null) {
+      throw S3ErrorTable.newError(S3ErrorTable.INVALID_ARGUMENT, permission);
+    }
+    switch (permissionType) {
+    case FULL_CONTROL:
+      acls.set(IAccessAuthorizer.ACLType.READ.ordinal());
+      acls.set(IAccessAuthorizer.ACLType.WRITE.ordinal());
+      acls.set(IAccessAuthorizer.ACLType.READ_ACL.ordinal());
+      acls.set(IAccessAuthorizer.ACLType.WRITE_ACL.ordinal());
+      break;
+    case WRITE_ACP:
+      acls.set(IAccessAuthorizer.ACLType.READ.ordinal());
+      acls.set(IAccessAuthorizer.ACLType.READ_ACL.ordinal());
+      acls.set(IAccessAuthorizer.ACLType.WRITE_ACL.ordinal());
+      break;
+    case READ_ACP:
+      acls.set(IAccessAuthorizer.ACLType.READ.ordinal());
+      acls.set(IAccessAuthorizer.ACLType.READ_ACL.ordinal());
+      break;
+    case WRITE:
+      acls.set(IAccessAuthorizer.ACLType.READ.ordinal());
+      acls.set(IAccessAuthorizer.ACLType.WRITE.ordinal());
+      break;
+    case READ:
+      acls.set(IAccessAuthorizer.ACLType.READ.ordinal());
+      break;
+    default:
+      LOG.error("Failed to recognize S3 permission {}", permission);
+      throw S3ErrorTable.newError(INVALID_ARGUMENT, permission);
+    }
+    return acls;
+  }
+}
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/S3BucketAcl.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/S3BucketAcl.java
new file mode 100644
index 0000000..7a699e4
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/S3BucketAcl.java
@@ -0,0 +1,235 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.s3.endpoint;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlAttribute;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+
+/**
+ * Bucket ACL.
+ */
+@XmlAccessorType(XmlAccessType.FIELD)
+@XmlRootElement(name = "AccessControlPolicy",
+    namespace = "http://s3.amazonaws.com/doc/2006-03-01/")
+public class S3BucketAcl {
+
+  @XmlElement(name = "Owner")
+  private S3Owner owner;
+
+  @XmlElement(name = "AccessControlList")
+  private AccessControlList aclList;
+
+  public S3Owner getOwner() {
+    return owner;
+  }
+
+  public void setOwner(S3Owner owner) {
+    this.owner = owner;
+  }
+
+  public AccessControlList getAclList() {
+    return aclList;
+  }
+
+  public void setAclList(AccessControlList aclList) {
+    this.aclList = aclList;
+  }
+
+  @Override
+  public String toString() {
+    return "GetBucketAclResponse{" +
+        "owner=" + owner +
+        ", aclList=" + aclList +
+        '}';
+  }
+
+  @XmlAccessorType(XmlAccessType.FIELD)
+  @XmlRootElement(name = "AccessControlList")
+  public static class AccessControlList {
+
+    @XmlElement(name = "Grant")
+    private List<Grant> grantList = new ArrayList<>();
+
+    public void addGrant(Grant grant) {
+      grantList.add(grant);
+    }
+
+    public List<Grant> getGrantList() {
+      return grantList;
+    }
+
+    public AccessControlList(List<Grant> grants) {
+      this.grantList = grants;
+    }
+
+    public AccessControlList() {
+
+    }
+
+    @Override
+    public String toString() {
+      return "AccessControlList{" +
+          "grantList=" + grantList +
+          '}';
+    }
+  }
+
+  @XmlAccessorType(XmlAccessType.FIELD)
+  @XmlRootElement(name = "Grant")
+  public static class Grant {
+
+    @XmlElement(name = "Grantee")
+    private Grantee grantee;
+
+    @XmlElement(name = "Permission")
+    private String permission;
+
+    public String getPermission() {
+      return permission;
+    }
+
+    public void setPermission(String permission) {
+      this.permission = permission;
+    }
+
+    public Grantee getGrantee() {
+      return grantee;
+    }
+
+    public void setGrantee(Grantee grantee) {
+      this.grantee = grantee;
+    }
+
+    @Override
+    public String toString() {
+      return "Grant{" +
+          "grantee=" + grantee +
+          ", permission='" + permission + '\'' +
+          '}';
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      Grant grant = (Grant) o;
+      return Objects.equals(grantee, grant.grantee) &&
+          Objects.equals(permission, grant.permission);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(grantee, permission);
+    }
+  }
+
+  /**
+   * A grantee can be an AWS account or one of the predefined Amazon S3 groups.
+   * https://docs.aws.amazon.com/AmazonS3/latest/dev/acl-overview.html
+   */
+  @XmlAccessorType(XmlAccessType.FIELD)
+  @XmlRootElement(name = "Grantee")
+  public static class Grantee {
+
+    @XmlElement(name = "DisplayName")
+    private String displayName;
+
+    @XmlElement(name = "ID")
+    private String id;
+
+    @XmlAttribute(name="xsi:type")
+    private String xsiType = "CanonicalUser";
+
+    @XmlAttribute(name="xmlns:xsi")
+    private String xsiNs = "http://www.w3.org/2001/XMLSchema-instance";
+
+    public String getXsiNs() {
+      return xsiNs;
+    }
+
+    public String getXsiType() {
+      return xsiType;
+    }
+
+    public String getId() {
+      return id;
+    }
+
+    public void setId(String id) {
+      this.id = id;
+    }
+
+    public String getDisplayName() {
+      return displayName;
+    }
+
+    public void setDisplayName(String name) {
+      this.displayName = name;
+    }
+
+    public void setXsiType(String type) {
+      this.xsiType = type;
+    }
+
+    public void setXsiNs(String ns) {
+      this.xsiNs = ns;
+    }
+
+    @Override
+    public String toString() {
+      return "Grantee{" +
+          "displayName='" + displayName + '\'' +
+          ", id='" + id + '\'' +
+          ", xsiType='" + xsiType + '\'' +
+          ", xsiNs='" + xsiNs + '\'' +
+          '}';
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+
+      Grantee grantee = (Grantee) o;
+      return Objects.equals(displayName, grantee.displayName) &&
+          Objects.equals(id, grantee.id) &&
+          Objects.equals(xsiType, grantee.xsiType) &&
+          Objects.equals(xsiNs, grantee.xsiNs);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(displayName, id, xsiType, xsiNs);
+    }
+  }
+}
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/S3Owner.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/S3Owner.java
new file mode 100644
index 0000000..48f154e
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/S3Owner.java
@@ -0,0 +1,72 @@
+/*
+ * 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.s3.endpoint;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+
+@XmlAccessorType(XmlAccessType.FIELD)
+@XmlRootElement(name = "Owner")
+public class S3Owner {
+
+  public static final S3Owner
+      NOT_SUPPORTED_OWNER = new S3Owner("NOT-SUPPORTED", "Not Supported");
+
+  @XmlElement(name = "DisplayName")
+  private String displayName;
+
+  @XmlElement(name = "ID")
+  private String id;
+
+  public S3Owner() {
+
+  }
+
+  public S3Owner(String id, String displayName) {
+    this.id = id;
+    this.displayName = displayName;
+  }
+
+  public String getDisplayName() {
+    return displayName;
+  }
+
+  public void setDisplayName(String name) {
+    this.displayName = name;
+  }
+
+  public String getId() {
+    return id;
+  }
+
+  public void setId(String id) {
+    this.id = id;
+  }
+
+  @Override
+  public String toString() {
+    return "S3Owner{" +
+        "displayName='" + displayName + '\'' +
+        ", id='" + id + '\'' +
+        '}';
+  }
+}
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/exception/S3ErrorTable.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/exception/S3ErrorTable.java
index 7d9b55f..86d9fc0 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/exception/S3ErrorTable.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/exception/S3ErrorTable.java
@@ -25,6 +25,7 @@ import static java.net.HttpURLConnection.HTTP_CONFLICT;
 import static java.net.HttpURLConnection.HTTP_FORBIDDEN;
 import static java.net.HttpURLConnection.HTTP_NOT_FOUND;
 import static java.net.HttpURLConnection.HTTP_PRECON_FAILED;
+import static java.net.HttpURLConnection.HTTP_NOT_IMPLEMENTED;
 import static java.net.HttpURLConnection.HTTP_SERVER_ERROR;
 import static org.apache.hadoop.ozone.s3.util.S3Consts.RANGE_NOT_SATISFIABLE;
 
@@ -114,6 +115,10 @@ public final class S3ErrorTable {
   public static final OS3Exception PRECOND_FAILED = new OS3Exception(
       "PreconditionFailed", "At least one of the pre-conditions you " +
       "specified did not hold", HTTP_PRECON_FAILED);
+  
+  public static final OS3Exception NOT_IMPLEMENTED = new OS3Exception(
+      "NotImplemented", "This part of feature is not implemented yet.",
+      HTTP_NOT_IMPLEMENTED);
 
   /**
    * Create a new instance of Error.
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneBucketStub.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneBucketStub.java
index 7861bd4..30cd412 100644
--- a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneBucketStub.java
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneBucketStub.java
@@ -35,6 +35,7 @@ import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.hadoop.hdds.client.ReplicationFactor;
 import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.hdds.protocol.StorageType;
+import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.client.io.OzoneInputStream;
 import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
 import org.apache.hadoop.ozone.client.OzoneMultipartUploadPartListParts.PartInfo;
@@ -57,6 +58,8 @@ public class OzoneBucketStub extends OzoneBucket {
 
   private Map<String, Map<Integer, Part>> partList = new HashMap<>();
 
+  private ArrayList<OzoneAcl> aclList = new ArrayList<>();
+
   /**
    * Constructs OzoneBucket instance.
    *
@@ -291,7 +294,27 @@ public class OzoneBucketStub extends OzoneBucket {
       return ozoneMultipartUploadPartListParts;
 
     }
+  }
+
+  @Override
+  public List<OzoneAcl> getAcls() throws IOException {
+    return (List<OzoneAcl>)aclList.clone();
+  }
+
+  @Override
+  public boolean removeAcl(OzoneAcl removeAcl) throws IOException {
+    return aclList.remove(removeAcl);
+  }
+
+  @Override
+  public boolean addAcl(OzoneAcl addAcl) throws IOException {
+    return aclList.add(addAcl);
+  }
 
+  @Override
+  public boolean setAcl(List<OzoneAcl> acls) throws IOException {
+    aclList.clear();
+    return aclList.addAll(acls);
   }
 
   /**
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneVolumeStub.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneVolumeStub.java
index f5e0603..55a7191 100644
--- a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneVolumeStub.java
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneVolumeStub.java
@@ -20,6 +20,7 @@
 package org.apache.hadoop.ozone.client;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
@@ -38,6 +39,8 @@ public class OzoneVolumeStub extends OzoneVolume {
 
   private Map<String, OzoneBucketStub> buckets = new HashMap<>();
 
+  private ArrayList<OzoneAcl> aclList = new ArrayList<>();
+
   public OzoneVolumeStub(String name, String admin, String owner,
       long quotaInBytes, long quotaInNamespace, long creationTime,
       List<OzoneAcl> acls) {
@@ -107,4 +110,25 @@ public class OzoneVolumeStub extends OzoneVolume {
       throw new OMException("", OMException.ResultCodes.BUCKET_NOT_FOUND);
     }
   }
+
+  @Override
+  public List<OzoneAcl> getAcls() {
+    return (List<OzoneAcl>)aclList.clone();
+  }
+
+  @Override
+  public boolean addAcl(OzoneAcl addAcl) throws IOException {
+    return aclList.add(addAcl);
+  }
+
+  @Override
+  public boolean removeAcl(OzoneAcl acl) throws IOException {
+    return aclList.remove(acl);
+  }
+
+  @Override
+  public boolean setAcl(List<OzoneAcl> acls) throws IOException {
+    aclList.clear();
+    return aclList.addAll(acls);
+  }
 }
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketAcl.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketAcl.java
new file mode 100644
index 0000000..703ed98
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketAcl.java
@@ -0,0 +1,266 @@
+/*
+ * 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.s3.endpoint;
+
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientStub;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.s3.exception.OS3Exception;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.core.HttpHeaders;
+import javax.ws.rs.core.Response;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Map;
+
+import static java.net.HttpURLConnection.HTTP_NOT_FOUND;
+import static java.net.HttpURLConnection.HTTP_NOT_IMPLEMENTED;
+import static java.net.HttpURLConnection.HTTP_OK;
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.when;
+
+/**
+ * This class tests Bucket ACL get/set request.
+ */
+public class TestBucketAcl {
+
+  private static final String BUCKET_NAME = OzoneConsts.S3_BUCKET;
+  private OzoneClient client;
+
+  private HttpServletRequest servletRequest;
+  private Map<String, String[]> parameterMap;
+  private HttpHeaders headers;
+  private BucketEndpoint bucketEndpoint;
+  private static final String ACL_MARKER = "acl";
+
+  @Before
+  public void setup() throws IOException {
+    client = new OzoneClientStub();
+    client.getObjectStore().createS3Bucket(BUCKET_NAME);
+
+    servletRequest = Mockito.mock(HttpServletRequest.class);
+    parameterMap = Mockito.mock(Map.class);
+    headers = Mockito.mock(HttpHeaders.class);
+    when(servletRequest.getParameterMap()).thenReturn(parameterMap);
+
+    bucketEndpoint = new BucketEndpoint();
+    bucketEndpoint.setClient(client);
+  }
+
+  @After
+  public void clean() throws IOException {
+    if (client != null) {
+      client.close();
+    }
+  }
+
+  @Test
+  public void testGetAcl() throws Exception {
+    when(parameterMap.containsKey(ACL_MARKER)).thenReturn(true);
+    Response response =
+        bucketEndpoint.get(BUCKET_NAME, null, null, null, 0, null, null,
+            null, null, null, ACL_MARKER, headers);
+    assertEquals(HTTP_OK, response.getStatus());
+    System.out.println(response.getEntity());
+  }
+
+  @Test
+  public void testSetAclWithNotSupportedGranteeType() throws Exception {
+    when(headers.getHeaderString(S3Acl.GRANT_READ))
+        .thenReturn(S3Acl.ACLIdentityType.GROUP.getHeaderType() + "=root");
+    when(parameterMap.containsKey(ACL_MARKER)).thenReturn(true);
+    try {
+      bucketEndpoint.put(BUCKET_NAME, ACL_MARKER, headers, null);
+    } catch (Exception e) {
+      Assert.assertTrue(e instanceof OS3Exception &&
+          ((OS3Exception) e).getHttpCode() == HTTP_NOT_IMPLEMENTED);
+    }
+  }
+
+  @Test
+  public void testRead() throws Exception {
+    when(parameterMap.containsKey(ACL_MARKER)).thenReturn(true);
+    when(headers.getHeaderString(S3Acl.GRANT_READ))
+        .thenReturn(S3Acl.ACLIdentityType.USER.getHeaderType() + "=root");
+    Response response =
+        bucketEndpoint.put(BUCKET_NAME, ACL_MARKER, headers, null);
+    assertEquals(HTTP_OK, response.getStatus());
+    S3BucketAcl getResponse = bucketEndpoint.getAcl(BUCKET_NAME);
+    assertEquals(1, getResponse.getAclList().getGrantList().size());
+    assertEquals(S3Acl.ACLType.READ.getValue(),
+        getResponse.getAclList().getGrantList().get(0).getPermission());
+  }
+
+  @Test
+  public void testWrite() throws Exception {
+    when(parameterMap.containsKey(ACL_MARKER)).thenReturn(true);
+    when(headers.getHeaderString(S3Acl.GRANT_WRITE))
+        .thenReturn(S3Acl.ACLIdentityType.USER.getHeaderType() + "=root");
+    Response response =
+        bucketEndpoint.put(BUCKET_NAME, ACL_MARKER, headers, null);
+    assertEquals(HTTP_OK, response.getStatus());
+    S3BucketAcl getResponse = bucketEndpoint.getAcl(BUCKET_NAME);
+    assertEquals(1, getResponse.getAclList().getGrantList().size());
+    assertEquals(S3Acl.ACLType.WRITE.getValue(),
+        getResponse.getAclList().getGrantList().get(0).getPermission());
+  }
+
+  @Test
+  public void testReadACP() throws Exception {
+    when(parameterMap.containsKey(ACL_MARKER)).thenReturn(true);
+    when(headers.getHeaderString(S3Acl.GRANT_READ_CAP))
+        .thenReturn(S3Acl.ACLIdentityType.USER.getHeaderType() + "=root");
+    Response response =
+        bucketEndpoint.put(BUCKET_NAME, ACL_MARKER, headers, null);
+    assertEquals(HTTP_OK, response.getStatus());
+    S3BucketAcl getResponse =
+        bucketEndpoint.getAcl(BUCKET_NAME);
+    assertEquals(1, getResponse.getAclList().getGrantList().size());
+    assertEquals(S3Acl.ACLType.READ_ACP.getValue(),
+        getResponse.getAclList().getGrantList().get(0).getPermission());
+  }
+
+  @Test
+  public void testWriteACP() throws Exception {
+    when(parameterMap.containsKey(ACL_MARKER)).thenReturn(true);
+    when(headers.getHeaderString(S3Acl.GRANT_WRITE_CAP))
+        .thenReturn(S3Acl.ACLIdentityType.USER.getHeaderType() + "=root");
+    Response response =
+        bucketEndpoint.put(BUCKET_NAME, ACL_MARKER, headers, null);
+    assertEquals(HTTP_OK, response.getStatus());
+    S3BucketAcl getResponse = bucketEndpoint.getAcl(BUCKET_NAME);
+    assertEquals(1, getResponse.getAclList().getGrantList().size());
+    assertEquals(S3Acl.ACLType.WRITE_ACP.getValue(),
+        getResponse.getAclList().getGrantList().get(0).getPermission());
+  }
+
+  @Test
+  public void testFullControl() throws Exception {
+    when(parameterMap.containsKey(ACL_MARKER)).thenReturn(true);
+    when(headers.getHeaderString(S3Acl.GRANT_FULL_CONTROL))
+        .thenReturn(S3Acl.ACLIdentityType.USER.getHeaderType() + "=root");
+    Response response =
+        bucketEndpoint.put(BUCKET_NAME, ACL_MARKER, headers, null);
+    assertEquals(HTTP_OK, response.getStatus());
+    S3BucketAcl getResponse = bucketEndpoint.getAcl(BUCKET_NAME);
+    assertEquals(1, getResponse.getAclList().getGrantList().size());
+    assertEquals(S3Acl.ACLType.FULL_CONTROL.getValue(),
+        getResponse.getAclList().getGrantList().get(0).getPermission());
+  }
+
+  @Test
+  public void testCombination() throws Exception {
+    when(parameterMap.containsKey(ACL_MARKER)).thenReturn(true);
+    when(headers.getHeaderString(S3Acl.GRANT_READ))
+        .thenReturn(S3Acl.ACLIdentityType.USER.getHeaderType() + "=root");
+    when(headers.getHeaderString(S3Acl.GRANT_WRITE))
+        .thenReturn(S3Acl.ACLIdentityType.USER.getHeaderType() + "=root");
+    when(headers.getHeaderString(S3Acl.GRANT_READ_CAP))
+        .thenReturn(S3Acl.ACLIdentityType.USER.getHeaderType() + "=root");
+    when(headers.getHeaderString(S3Acl.GRANT_WRITE_CAP))
+        .thenReturn(S3Acl.ACLIdentityType.USER.getHeaderType() + "=root");
+    when(headers.getHeaderString(S3Acl.GRANT_FULL_CONTROL))
+        .thenReturn(S3Acl.ACLIdentityType.USER.getHeaderType() + "=root");
+    Response response =
+        bucketEndpoint.put(BUCKET_NAME, ACL_MARKER, headers, null);
+    assertEquals(HTTP_OK, response.getStatus());
+    S3BucketAcl getResponse = bucketEndpoint.getAcl(BUCKET_NAME);
+    assertEquals(5, getResponse.getAclList().getGrantList().size());
+  }
+
+  @Test
+  public void testPutClearOldAcls() throws Exception {
+    when(parameterMap.containsKey(ACL_MARKER)).thenReturn(true);
+    when(headers.getHeaderString(S3Acl.GRANT_READ))
+        .thenReturn(S3Acl.ACLIdentityType.USER.getHeaderType() + "=root");
+    // Put READ
+    Response response =
+        bucketEndpoint.put(BUCKET_NAME, ACL_MARKER, headers, null);
+    assertEquals(HTTP_OK, response.getStatus());
+    S3BucketAcl getResponse = bucketEndpoint.getAcl(BUCKET_NAME);
+    assertEquals(1, getResponse.getAclList().getGrantList().size());
+    assertEquals(S3Acl.ACLType.READ.getValue(),
+        getResponse.getAclList().getGrantList().get(0).getPermission());
+    OzoneVolume volume = bucketEndpoint.getVolume();
+    assertEquals(1, volume.getAcls().size());
+    assertEquals(IAccessAuthorizer.ACLType.READ,
+        volume.getAcls().get(0).getAclList().get(0));
+
+    when(headers.getHeaderString(S3Acl.GRANT_READ))
+        .thenReturn(null);
+    when(headers.getHeaderString(S3Acl.GRANT_WRITE))
+        .thenReturn(S3Acl.ACLIdentityType.USER.getHeaderType() + "=root");
+    //Put WRITE
+    response =
+        bucketEndpoint.put(BUCKET_NAME, ACL_MARKER, headers, null);
+    assertEquals(HTTP_OK, response.getStatus());
+    getResponse = bucketEndpoint.getAcl(BUCKET_NAME);
+    assertEquals(1, getResponse.getAclList().getGrantList().size());
+    assertEquals(S3Acl.ACLType.WRITE.getValue(),
+        getResponse.getAclList().getGrantList().get(0).getPermission());
+    volume = bucketEndpoint.getVolume();
+    assertEquals(1, volume.getAcls().size());
+    assertEquals(IAccessAuthorizer.ACLType.READ,
+        volume.getAcls().get(0).getAclList().get(0));
+  }
+
+  @Test(expected = OS3Exception.class)
+  public void testAclInBodyWithGroupUser() throws Exception {
+    InputStream inputBody = TestBucketAcl.class.getClassLoader()
+        .getResourceAsStream("groupAccessControlList.xml");
+    when(parameterMap.containsKey(ACL_MARKER)).thenReturn(true);
+    bucketEndpoint.put(BUCKET_NAME, ACL_MARKER, headers, inputBody);
+  }
+
+  @Test
+  public void testAclInBody() throws Exception {
+    InputStream inputBody = TestBucketAcl.class.getClassLoader()
+        .getResourceAsStream("userAccessControlList.xml");
+    when(parameterMap.containsKey(ACL_MARKER)).thenReturn(true);
+    Response response =
+        bucketEndpoint.put(BUCKET_NAME, ACL_MARKER, headers, inputBody);
+    assertEquals(HTTP_OK, response.getStatus());
+    S3BucketAcl getResponse = bucketEndpoint.getAcl(BUCKET_NAME);
+    assertEquals(2, getResponse.getAclList().getGrantList().size());
+  }
+
+  @Test
+  public void testBucketNotExist() throws Exception {
+    when(parameterMap.containsKey(ACL_MARKER)).thenReturn(true);
+    when(headers.getHeaderString(S3Acl.GRANT_READ))
+        .thenReturn(S3Acl.ACLIdentityType.USER.getHeaderType() + "=root");
+    try {
+      bucketEndpoint.getAcl("bucket-not-exist");
+    } catch (Exception e) {
+      Assert.assertTrue(e instanceof OS3Exception &&
+          ((OS3Exception)e).getHttpCode() == HTTP_NOT_FOUND);
+    }
+  }
+}
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketGet.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketList.java
similarity index 80%
rename from hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketGet.java
rename to hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketList.java
index 9bafae5..84e7555 100644
--- a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketGet.java
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketList.java
@@ -33,7 +33,7 @@ import org.junit.Test;
 /**
  * Testing basic object list browsing.
  */
-public class TestBucketGet {
+public class TestBucketList {
 
   @Test
   public void listRoot() throws OS3Exception, IOException {
@@ -45,8 +45,8 @@ public class TestBucketGet {
     getBucket.setClient(client);
 
     ListObjectResponse getBucketResponse =
-        (ListObjectResponse) getBucket
-            .list("b1", "/", null, null, 100, "", null, null, null, null, null)
+        (ListObjectResponse) getBucket.get("b1", "/", null, null, 100, "",
+            null, null, null, null, null, null)
             .getEntity();
 
     Assert.assertEquals(1, getBucketResponse.getCommonPrefixes().size());
@@ -69,8 +69,8 @@ public class TestBucketGet {
     getBucket.setClient(client);
 
     ListObjectResponse getBucketResponse =
-        (ListObjectResponse) getBucket.list("b1", "/", null, null, 100,
-            "dir1", null, null, null, null, null).getEntity();
+        (ListObjectResponse) getBucket.get("b1", "/", null, null, 100,
+            "dir1", null, null, null, null, null, null).getEntity();
 
     Assert.assertEquals(1, getBucketResponse.getCommonPrefixes().size());
     Assert.assertEquals("dir1/",
@@ -93,8 +93,8 @@ public class TestBucketGet {
 
     ListObjectResponse getBucketResponse =
         (ListObjectResponse) getBucket
-            .list("b1", "/", null, null, 100, "dir1/", null, null,
-                null, null, null)
+            .get("b1", "/", null, null, 100, "dir1/", null, null,
+                null, null, null, null)
             .getEntity();
 
     Assert.assertEquals(1, getBucketResponse.getCommonPrefixes().size());
@@ -120,8 +120,8 @@ public class TestBucketGet {
     getBucket.setClient(ozoneClient);
 
     ListObjectResponse getBucketResponse =
-        (ListObjectResponse) getBucket.list("b1", "/", null, null, 100,
-            "dir1", null, null, null, null, null).getEntity();
+        (ListObjectResponse) getBucket.get("b1", "/", null, null, 100,
+            "dir1", null, null, null, null, null, null).getEntity();
 
     Assert.assertEquals(3, getBucketResponse.getCommonPrefixes().size());
 
@@ -139,8 +139,8 @@ public class TestBucketGet {
     getBucket.setClient(ozoneClient);
 
     ListObjectResponse getBucketResponse =
-        (ListObjectResponse) getBucket.list("b1", "/", null, null, 100,
-            "", null, null, null, null, null).getEntity();
+        (ListObjectResponse) getBucket.get("b1", "/", null, null, 100,
+            "", null, null, null, null, null, null).getEntity();
 
     Assert.assertEquals(3, getBucketResponse.getCommonPrefixes().size());
     Assert.assertEquals("file2", getBucketResponse.getContents().get(0)
@@ -160,8 +160,8 @@ public class TestBucketGet {
     getBucket.setClient(ozoneClient);
 
     ListObjectResponse getBucketResponse =
-        (ListObjectResponse) getBucket.list("b1", "/", null, null, 100,
-            "dir1bh", null, null, "dir1/dir2/file2", null, null).getEntity();
+        (ListObjectResponse) getBucket.get("b1", "/", null, null, 100, "dir1bh",
+            null, null, "dir1/dir2/file2", null, null, null).getEntity();
 
     Assert.assertEquals(2, getBucketResponse.getCommonPrefixes().size());
 
@@ -183,8 +183,8 @@ public class TestBucketGet {
 
     // First time
     ListObjectResponse getBucketResponse =
-        (ListObjectResponse) getBucket.list("b1", null, null, null, maxKeys,
-            "", null, null, null, null, null).getEntity();
+        (ListObjectResponse) getBucket.get("b1", null, null, null, maxKeys,
+            "", null, null, null, null, null, null).getEntity();
 
     Assert.assertTrue(getBucketResponse.isTruncated());
     Assert.assertEquals(2, getBucketResponse.getContents().size());
@@ -192,8 +192,8 @@ public class TestBucketGet {
     // 2nd time
     String continueToken = getBucketResponse.getNextToken();
     getBucketResponse =
-        (ListObjectResponse) getBucket.list("b1", null, null, null, maxKeys,
-            "", null, continueToken, null, null, null).getEntity();
+        (ListObjectResponse) getBucket.get("b1", null, null, null, maxKeys,
+            "", null, continueToken, null, null, null, null).getEntity();
     Assert.assertTrue(getBucketResponse.isTruncated());
     Assert.assertEquals(2, getBucketResponse.getContents().size());
 
@@ -202,8 +202,8 @@ public class TestBucketGet {
 
     //3rd time
     getBucketResponse =
-        (ListObjectResponse) getBucket.list("b1", null, null, null, maxKeys,
-            "", null, continueToken, null, null, null).getEntity();
+        (ListObjectResponse) getBucket.get("b1", null, null, null, maxKeys,
+            "", null, continueToken, null, null, null, null).getEntity();
 
     Assert.assertFalse(getBucketResponse.isTruncated());
     Assert.assertEquals(1, getBucketResponse.getContents().size());
@@ -234,8 +234,8 @@ public class TestBucketGet {
     ListObjectResponse getBucketResponse;
 
     getBucketResponse =
-        (ListObjectResponse) getBucket.list("b1", "/", null, null, maxKeys,
-            "test/", null, null, null, null, null).getEntity();
+        (ListObjectResponse) getBucket.get("b1", "/", null, null, maxKeys,
+            "test/", null, null, null, null, null, null).getEntity();
 
     Assert.assertEquals(0, getBucketResponse.getContents().size());
     Assert.assertEquals(2, getBucketResponse.getCommonPrefixes().size());
@@ -245,9 +245,9 @@ public class TestBucketGet {
         getBucketResponse.getCommonPrefixes().get(1).getPrefix());
 
     getBucketResponse =
-        (ListObjectResponse) getBucket.list("b1", "/", null, null, maxKeys,
-            "test/", null, getBucketResponse.getNextToken(), null, null, null)
-            .getEntity();
+        (ListObjectResponse) getBucket.get("b1", "/", null, null, maxKeys,
+            "test/", null, getBucketResponse.getNextToken(), null, null, null,
+            null).getEntity();
     Assert.assertEquals(1, getBucketResponse.getContents().size());
     Assert.assertEquals(1, getBucketResponse.getCommonPrefixes().size());
     Assert.assertEquals("test/dir3/",
@@ -277,8 +277,8 @@ public class TestBucketGet {
 
     // First time
     ListObjectResponse getBucketResponse =
-        (ListObjectResponse) getBucket.list("b1", "/", null, null, maxKeys,
-            "dir", null, null, null, null, null).getEntity();
+        (ListObjectResponse) getBucket.get("b1", "/", null, null, maxKeys,
+            "dir", null, null, null, null, null, null).getEntity();
 
     Assert.assertTrue(getBucketResponse.isTruncated());
     Assert.assertEquals(2, getBucketResponse.getCommonPrefixes().size());
@@ -286,16 +286,16 @@ public class TestBucketGet {
     // 2nd time
     String continueToken = getBucketResponse.getNextToken();
     getBucketResponse =
-        (ListObjectResponse) getBucket.list("b1", "/", null, null, maxKeys,
-            "dir", null, continueToken, null, null, null).getEntity();
+        (ListObjectResponse) getBucket.get("b1", "/", null, null, maxKeys,
+            "dir", null, continueToken, null, null, null, null).getEntity();
     Assert.assertTrue(getBucketResponse.isTruncated());
     Assert.assertEquals(2, getBucketResponse.getCommonPrefixes().size());
 
     //3rd time
     continueToken = getBucketResponse.getNextToken();
     getBucketResponse =
-        (ListObjectResponse) getBucket.list("b1", "/", null, null, maxKeys,
-            "dir", null, continueToken, null, null, null).getEntity();
+        (ListObjectResponse) getBucket.get("b1", "/", null, null, maxKeys,
+            "dir", null, continueToken, null, null, null, null).getEntity();
 
     Assert.assertFalse(getBucketResponse.isTruncated());
     Assert.assertEquals(1, getBucketResponse.getCommonPrefixes().size());
@@ -314,8 +314,8 @@ public class TestBucketGet {
     getBucket.setClient(ozoneClient);
 
     try {
-      getBucket.list("b1", "/", null, null, 2, "dir", null, "random", null,
-          null, null).getEntity();
+      getBucket.get("b1", "/", null, null, 2, "dir", null, "random", null,
+          null, null, null).getEntity();
       fail("listWithContinuationTokenFail");
     } catch (OS3Exception ex) {
       Assert.assertEquals("random", ex.getResource());
@@ -336,8 +336,8 @@ public class TestBucketGet {
     getBucket.setClient(ozoneClient);
 
     ListObjectResponse getBucketResponse =
-        (ListObjectResponse) getBucket.list("b1", null, null, null, 1000,
-            null, null, null, null, null, null).getEntity();
+        (ListObjectResponse) getBucket.get("b1", null, null, null, 1000,
+            null, null, null, null, null, null, null).getEntity();
 
     Assert.assertFalse(getBucketResponse.isTruncated());
     Assert.assertEquals(5, getBucketResponse.getContents().size());
@@ -347,15 +347,15 @@ public class TestBucketGet {
     String startAfter = "dir0/file1";
 
     getBucketResponse =
-        (ListObjectResponse) getBucket.list("b1", null, null, null,
-            1000, null, null, null, startAfter, null, null).getEntity();
+        (ListObjectResponse) getBucket.get("b1", null, null, null,
+            1000, null, null, null, startAfter, null, null, null).getEntity();
 
     Assert.assertFalse(getBucketResponse.isTruncated());
     Assert.assertEquals(4, getBucketResponse.getContents().size());
 
     getBucketResponse =
-        (ListObjectResponse) getBucket.list("b1", null, null, null,
-            1000, null, null, null, "random", null, null).getEntity();
+        (ListObjectResponse) getBucket.get("b1", null, null, null,
+            1000, null, null, null, "random", null, null, null).getEntity();
 
     Assert.assertFalse(getBucketResponse.isTruncated());
     Assert.assertEquals(0, getBucketResponse.getContents().size());
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketPut.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketPut.java
index 738bb1d..dd15518 100644
--- a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketPut.java
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketPut.java
@@ -61,7 +61,7 @@ public class TestBucketPut {
   public void testBucketFailWithAuthHeaderMissing() throws Exception {
 
     try {
-      bucketEndpoint.put(bucketName, null);
+      bucketEndpoint.put(bucketName, null, null, null);
     } catch (OS3Exception ex) {
       Assert.assertEquals(HTTP_NOT_FOUND, ex.getHttpCode());
       Assert.assertEquals(MALFORMED_HEADER.getCode(), ex.getCode());
@@ -70,7 +70,7 @@ public class TestBucketPut {
 
   @Test
   public void testBucketPut() throws Exception {
-    Response response = bucketEndpoint.put(bucketName, null);
+    Response response = bucketEndpoint.put(bucketName, null, null, null);
     assertEquals(200, response.getStatus());
     assertNotNull(response.getLocation());
   }
@@ -78,7 +78,7 @@ public class TestBucketPut {
   @Test
   public void testBucketFailWithInvalidHeader() throws Exception {
     try {
-      bucketEndpoint.put(bucketName, null);
+      bucketEndpoint.put(bucketName, null, null, null);
     } catch (OS3Exception ex) {
       Assert.assertEquals(HTTP_NOT_FOUND, ex.getHttpCode());
       Assert.assertEquals(MALFORMED_HEADER.getCode(), ex.getCode());
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestPermissionCheck.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestPermissionCheck.java
index 1c5622e..81b0718 100644
--- a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestPermissionCheck.java
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestPermissionCheck.java
@@ -32,16 +32,19 @@ import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
 
+import javax.servlet.http.HttpServletRequest;
 import javax.ws.rs.core.HttpHeaders;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 
 import static java.net.HttpURLConnection.HTTP_FORBIDDEN;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.anyLong;
 import static org.mockito.ArgumentMatchers.anyString;
 import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.when;
 
 /**
  * Test operation permission check result.
@@ -115,7 +118,7 @@ public class TestPermissionCheck {
     bucketEndpoint.setClient(client);
 
     try {
-      bucketEndpoint.put("bucketName", null);
+      bucketEndpoint.put("bucketName", null, null, null);
       Assert.fail("Should fail");
     } catch (Exception e) {
       Assert.assertTrue(e instanceof OS3Exception);
@@ -162,8 +165,8 @@ public class TestPermissionCheck {
     bucketEndpoint.setClient(client);
 
     try {
-      bucketEndpoint.list("bucketName", null, null, null, 1000,
-          null, null, null, null, null, null);
+      bucketEndpoint.get("bucketName", null, null, null, 1000,
+          null, null, null, null, null, null, null);
       Assert.fail("Should fail");
     } catch (Exception e) {
       Assert.assertTrue(e instanceof OS3Exception);
@@ -191,6 +194,53 @@ public class TestPermissionCheck {
         response.getErrors().get(0).getCode().equals("PermissionDenied"));
   }
 
+  @Test
+  public void testGetAcl() throws Exception {
+    Mockito.when(objectStore.getVolume(anyString())).thenReturn(volume);
+    Mockito.when(objectStore.getS3Bucket(anyString())).thenReturn(bucket);
+    doThrow(exception).when(bucket).getAcls();
+
+    HttpServletRequest servletRequest = Mockito.mock(HttpServletRequest.class);
+    Map<String, String[]> parameterMap = Mockito.mock(Map.class);
+    when(servletRequest.getParameterMap()).thenReturn(parameterMap);
+
+    when(parameterMap.containsKey("acl")).thenReturn(true);
+    when(headers.getHeaderString(S3Acl.GRANT_READ))
+        .thenReturn(S3Acl.ACLIdentityType.USER.getHeaderType() + "=root");
+    BucketEndpoint bucketEndpoint = new BucketEndpoint();
+    bucketEndpoint.setClient(client);
+    try {
+      bucketEndpoint.get("bucketName", null, null, null, 1000,
+          null, null, null, null, null, "acl", null);
+    } catch (Exception e) {
+      Assert.assertTrue(e instanceof OS3Exception &&
+          ((OS3Exception)e).getHttpCode() == HTTP_FORBIDDEN);
+    }
+  }
+
+  @Test
+  public void testSetAcl() throws Exception {
+    Mockito.when(objectStore.getVolume(anyString())).thenReturn(volume);
+    Mockito.when(objectStore.getS3Bucket(anyString())).thenReturn(bucket);
+    doThrow(exception).when(bucket).addAcl(any());
+
+    HttpServletRequest servletRequest = Mockito.mock(HttpServletRequest.class);
+    Map<String, String[]> parameterMap = Mockito.mock(Map.class);
+    when(servletRequest.getParameterMap()).thenReturn(parameterMap);
+
+    when(parameterMap.containsKey("acl")).thenReturn(true);
+    when(headers.getHeaderString(S3Acl.GRANT_READ))
+        .thenReturn(S3Acl.ACLIdentityType.USER.getHeaderType() + "=root");
+    BucketEndpoint bucketEndpoint = new BucketEndpoint();
+    bucketEndpoint.setClient(client);
+    try {
+      bucketEndpoint.put("bucketName", "acl", headers, null);
+    } catch (Exception e) {
+      Assert.assertTrue(e instanceof OS3Exception &&
+          ((OS3Exception)e).getHttpCode() == HTTP_FORBIDDEN);
+    }
+  }
+
   /**
    *  Object Endpoint.
    */
diff --git a/hadoop-ozone/s3gateway/src/test/resources/groupAccessControlList.xml b/hadoop-ozone/s3gateway/src/test/resources/groupAccessControlList.xml
new file mode 100644
index 0000000..a774583
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/test/resources/groupAccessControlList.xml
@@ -0,0 +1,57 @@
+<?xml version="1.0"?>
+<!--
+ 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.
+-->
+<AccessControlPolicy xmlns="http://s3.amazonaws.com/doc/2006-03-01/">
+<Owner>
+    <ID>852b113e7a2f25102679df27bb0ae12b3f85be6BucketOwnerCanonicalUserID</ID>
+    <DisplayName>owner</DisplayName>
+</Owner>
+<AccessControlList>
+    <Grant>
+        <Grantee xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:type="CanonicalUser">
+            <ID>852b113e7a2f25102679df27bb0ae12b3f85be6BucketOwnerCanonicalUserID</ID>
+            <DisplayName>owner</DisplayName>
+        </Grantee>
+        <Permission>FULL_CONTROL</Permission>
+    </Grant>
+    <Grant>
+        <Grantee xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:type="Group">
+            <URI xmlns="">http://acs.amazonaws.com/groups/global/AllUsers</URI>
+        </Grantee>
+        <Permission xmlns="">READ</Permission>
+    </Grant>
+    <Grant>
+        <Grantee xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:type="Group">
+            <URI xmlns="">http://acs.amazonaws.com/groups/s3/LogDelivery</URI>
+        </Grantee>
+        <Permission xmlns="">WRITE</Permission>
+    </Grant>
+    <Grant>
+        <Grantee xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:type="AmazonCustomerByEmail">
+            <EmailAddress xmlns="">xyz@amazon.com</EmailAddress>
+        </Grantee>
+        <Permission xmlns="">WRITE_ACP</Permission>
+    </Grant>
+    <Grant>
+        <Grantee xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:type="CanonicalUser">
+            <ID xmlns="">f30716ab7115dcb44a5ef76e9d74b8e20567f63TestAccountCanonicalUserID</ID>
+        </Grantee>
+        <Permission xmlns="">READ_ACP</Permission>
+    </Grant>
+</AccessControlList>
+</AccessControlPolicy>
\ No newline at end of file
diff --git a/hadoop-ozone/s3gateway/src/test/resources/userAccessControlList.xml b/hadoop-ozone/s3gateway/src/test/resources/userAccessControlList.xml
new file mode 100644
index 0000000..45ce76e
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/test/resources/userAccessControlList.xml
@@ -0,0 +1,42 @@
+<?xml version="1.0"?>
+<!--
+ 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.
+-->
+<AccessControlPolicy xmlns="http://s3.amazonaws.com/doc/2006-03-01/">
+ <Owner>
+   <ID>852b113e7a2f25102679df27bb0ae12b3f85be6BucketOwnerCanonicalUserID</ID>
+   <DisplayName>owner</DisplayName>
+ </Owner>
+ <AccessControlList>
+   <Grant>
+     <Grantee xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+             xsi:type="CanonicalUser">
+       <ID>852b113e7a2f25102679df27bb0ae12b3f85be6BucketOwnerCanonicalUserID
+       </ID>
+       <DisplayName>owner</DisplayName>
+     </Grantee>
+     <Permission>FULL_CONTROL</Permission>
+    </Grant>
+    <Grant>
+      <Grantee xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+               xsi:type="CanonicalUser">
+        <ID xmlns="">f30716ab7115dcb44a5ef76e9d74b8e20567f63TestAccountCanonicalUserID</ID>
+      </Grantee>
+      <Permission xmlns="">READ_ACP</Permission>
+     </Grant>
+ </AccessControlList>
+</AccessControlPolicy>
\ No newline at end of file

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@ozone.apache.org
For additional commands, e-mail: commits-help@ozone.apache.org