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 bh...@apache.org on 2019/08/09 04:29:11 UTC

[hadoop] branch trunk updated: HDDS-1884. Support Bucket ACL operations for OM HA. (#1202)

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

bharat pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 91f41b7  HDDS-1884. Support Bucket ACL operations for OM HA. (#1202)
91f41b7 is described below

commit 91f41b7d885d7b0f3abf132a5c8e8812fb179330
Author: Bharat Viswanadham <bh...@apache.org>
AuthorDate: Thu Aug 8 21:29:00 2019 -0700

    HDDS-1884. Support Bucket ACL operations for OM HA. (#1202)
---
 .../hadoop/ozone/om/exceptions/OMException.java    |   5 +-
 .../hadoop/ozone/om/helpers/OmBucketInfo.java      |  95 ++++++++++-
 .../hadoop/ozone/util/BooleanBiFunction.java       |  11 ++
 .../src/main/proto/OzoneManagerProtocol.proto      |   2 +
 .../apache/hadoop/ozone/om/TestOzoneManagerHA.java | 142 ++++++++++++++++
 .../om/ratis/utils/OzoneManagerRatisUtils.java     |  15 +-
 .../om/request/bucket/acl/OMBucketAclRequest.java  | 186 +++++++++++++++++++++
 .../request/bucket/acl/OMBucketAddAclRequest.java  | 122 ++++++++++++++
 .../bucket/acl/OMBucketRemoveAclRequest.java       | 101 +++++++++++
 .../request/bucket/acl/OMBucketSetAclRequest.java  | 100 +++++++++++
 .../ozone/om/request/bucket/acl/package-info.java  |  23 +++
 .../hadoop/ozone/om/request/util/ObjectParser.java |  74 ++++++++
 .../hadoop/ozone/om/request/util/package-info.java |  23 +++
 .../response/bucket/acl/OMBucketAclResponse.java   |  62 +++++++
 .../ozone/om/response/bucket/acl/package-info.java |  22 +++
 15 files changed, 978 insertions(+), 5 deletions(-)

diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java
index 78bdb21..1e291ed 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java
@@ -205,6 +205,9 @@ public class OMException extends IOException {
 
     S3_BUCKET_INVALID_LENGTH,
 
-    RATIS_ERROR // Error in Ratis server
+    RATIS_ERROR, // Error in Ratis server
+
+    INVALID_PATH_IN_ACL_REQUEST // Error code when path name is invalid during
+    // acl requests.
   }
 }
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java
index 51cabe6..4d764a5 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.ozone.om.helpers;
 
 
+import java.util.BitSet;
 import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.LinkedList;
@@ -30,11 +31,14 @@ import org.apache.hadoop.hdds.protocol.StorageType;
 import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.audit.Auditable;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.BucketInfo;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .BucketInfo;
 import org.apache.hadoop.ozone.protocolPB.OMPBHelper;
 
 import com.google.common.base.Preconditions;
 
+import static org.apache.hadoop.ozone.OzoneAcl.ZERO_BITSET;
+
 /**
  * A class that encapsulates Bucket Info.
  */
@@ -125,6 +129,95 @@ public final class OmBucketInfo extends WithMetadata implements Auditable {
   }
 
   /**
+   * Add an ozoneAcl to list of existing Acl set.
+   * @param ozoneAcl
+   * @return true - if successfully added, false if not added or acl is
+   * already existing in the acl list.
+   */
+  public boolean addAcl(OzoneAcl ozoneAcl) {
+    // Case 1: When we are adding more rights to existing user/group.
+    boolean addToExistingAcl = false;
+    for(OzoneAcl existingAcl: getAcls()) {
+      if(existingAcl.getName().equals(ozoneAcl.getName()) &&
+          existingAcl.getType().equals(ozoneAcl.getType())) {
+
+        BitSet bits = (BitSet) ozoneAcl.getAclBitSet().clone();
+
+        // We need to do "or" before comparision because think of a case like
+        // existing acl is 777 and newly added acl is 444, we have already
+        // that acl set. In this case if we do direct check they will not
+        // be equal, but if we do or and then check, we shall know it
+        // has acl's already set or not.
+        bits.or(existingAcl.getAclBitSet());
+
+        if (bits.equals(existingAcl.getAclBitSet())) {
+          return false;
+        } else {
+          existingAcl.getAclBitSet().or(ozoneAcl.getAclBitSet());
+          addToExistingAcl = true;
+          break;
+        }
+      }
+    }
+
+    // Case 2: When a completely new acl is added.
+    if(!addToExistingAcl) {
+      getAcls().add(ozoneAcl);
+    }
+    return true;
+  }
+
+  /**
+   * Remove acl from existing acl list.
+   * @param ozoneAcl
+   * @return true - if successfully removed, false if not able to remove due
+   * to that acl is not in the existing acl list.
+   */
+  public boolean removeAcl(OzoneAcl ozoneAcl) {
+    boolean removed = false;
+
+    // When we are removing subset of rights from existing acl.
+    for(OzoneAcl existingAcl: getAcls()) {
+      if (existingAcl.getName().equals(ozoneAcl.getName()) &&
+          existingAcl.getType().equals(ozoneAcl.getType())) {
+        BitSet bits = (BitSet) ozoneAcl.getAclBitSet().clone();
+        bits.and(existingAcl.getAclBitSet());
+
+        // This happens when the acl bitset is not existing for current name
+        // and type.
+        // Like a case we have 444 permission, 333 is asked to removed.
+        if (bits.equals(ZERO_BITSET)) {
+          return false;
+        }
+
+        // We have some matching. Remove them.
+        existingAcl.getAclBitSet().xor(bits);
+
+        // If existing acl has same bitset as passed acl bitset, remove that
+        // acl from the list
+        if (existingAcl.getAclBitSet().equals(ZERO_BITSET)) {
+          getAcls().remove(existingAcl);
+        }
+        removed = true;
+        break;
+      }
+    }
+
+    return removed;
+  }
+
+  /**
+   * Reset the existing acl list.
+   * @param ozoneAcls
+   * @return true - if successfully able to reset.
+   */
+  public boolean setAcls(List<OzoneAcl> ozoneAcls) {
+    this.acls.clear();
+    this.acls = ozoneAcls;
+    return true;
+  }
+
+  /**
    * Returns true if bucket version is enabled, else false.
    * @return isVersionEnabled
    */
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/util/BooleanBiFunction.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/util/BooleanBiFunction.java
new file mode 100644
index 0000000..a70f4b0
--- /dev/null
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/util/BooleanBiFunction.java
@@ -0,0 +1,11 @@
+package org.apache.hadoop.ozone.util;
+
+/**
+ * Defines a functional interface having two inputs and returns boolean as
+ * output.
+ */
+@FunctionalInterface
+public interface BooleanBiFunction<LEFT, RIGHT> {
+  boolean apply(LEFT left, RIGHT right);
+}
+
diff --git a/hadoop-ozone/common/src/main/proto/OzoneManagerProtocol.proto b/hadoop-ozone/common/src/main/proto/OzoneManagerProtocol.proto
index 43e2548..27b9b11 100644
--- a/hadoop-ozone/common/src/main/proto/OzoneManagerProtocol.proto
+++ b/hadoop-ozone/common/src/main/proto/OzoneManagerProtocol.proto
@@ -283,6 +283,8 @@ enum Status {
     S3_BUCKET_INVALID_LENGTH = 51; // s3 bucket invalid length.
 
     RATIS_ERROR = 52;
+
+    INVALID_PATH_IN_ACL_REQUEST = 53; // Invalid path name in acl request.
 }
 
 
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerHA.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerHA.java
index 92fc263..29b6368 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerHA.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerHA.java
@@ -19,11 +19,16 @@ package org.apache.hadoop.ozone.om;
 import java.io.IOException;
 import java.net.ConnectException;
 import java.net.InetSocketAddress;
+import java.util.BitSet;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.UUID;
 
+import org.apache.hadoop.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.ozone.security.acl.OzoneObjInfo;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -65,6 +70,7 @@ import org.apache.hadoop.util.Time;
 
 import static org.apache.hadoop.ozone.MiniOzoneHAClusterImpl
     .NODE_FAILURE_TIMEOUT;
+import static org.apache.hadoop.ozone.OzoneAcl.AclScope.DEFAULT;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ACL_ENABLED;
 import static org.apache.hadoop.ozone.OzoneConfigKeys
     .OZONE_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY;
@@ -76,6 +82,9 @@ import static org.apache.hadoop.ozone.OzoneConfigKeys
     .OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS;
 import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.FILE_ALREADY_EXISTS;
 import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.NOT_A_FILE;
+import static org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLIdentityType.USER;
+import static org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLType.READ;
+import static org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLType.WRITE;
 import static org.junit.Assert.fail;
 
 /**
@@ -760,6 +769,139 @@ public class TestOzoneManagerHA {
   }
 
   @Test
+  public void testAddBucketAcl() throws Exception {
+    OzoneBucket ozoneBucket = setupBucket();
+    String remoteUserName = "remoteUser";
+    OzoneAcl defaultUserAcl = new OzoneAcl(USER, remoteUserName,
+        READ, DEFAULT);
+
+    OzoneObj ozoneObj = OzoneObjInfo.Builder.newBuilder()
+        .setResType(OzoneObj.ResourceType.BUCKET)
+        .setStoreType(OzoneObj.StoreType.OZONE)
+        .setVolumeName(ozoneBucket.getVolumeName())
+        .setBucketName(ozoneBucket.getName()).build();
+
+    boolean addAcl = objectStore.addAcl(ozoneObj, defaultUserAcl);
+    Assert.assertTrue(addAcl);
+
+    List<OzoneAcl> acls = objectStore.getAcl(ozoneObj);
+
+    Assert.assertTrue(containsAcl(defaultUserAcl, acls));
+
+    // Add an already existing acl.
+    addAcl = objectStore.addAcl(ozoneObj, defaultUserAcl);
+    Assert.assertFalse(addAcl);
+
+    // Add an acl by changing acl type with same type, name and scope.
+    defaultUserAcl = new OzoneAcl(USER, remoteUserName,
+        WRITE, DEFAULT);
+    addAcl = objectStore.addAcl(ozoneObj, defaultUserAcl);
+    Assert.assertTrue(addAcl);
+  }
+
+  @Test
+  public void testRemoveBucketAcl() throws Exception {
+    OzoneBucket ozoneBucket = setupBucket();
+    String remoteUserName = "remoteUser";
+    OzoneAcl defaultUserAcl = new OzoneAcl(USER, remoteUserName,
+        READ, DEFAULT);
+
+    OzoneObj ozoneObj = OzoneObjInfo.Builder.newBuilder()
+        .setResType(OzoneObj.ResourceType.BUCKET)
+        .setStoreType(OzoneObj.StoreType.OZONE)
+        .setVolumeName(ozoneBucket.getVolumeName())
+        .setBucketName(ozoneBucket.getName()).build();
+
+    // As by default create bucket we add some default acls in RpcClient.
+    List<OzoneAcl> acls = objectStore.getAcl(ozoneObj);
+
+    Assert.assertTrue(acls.size() > 0);
+
+    // Remove an existing acl.
+    boolean removeAcl = objectStore.removeAcl(ozoneObj, acls.get(0));
+    Assert.assertTrue(removeAcl);
+
+    // Trying to remove an already removed acl.
+    removeAcl = objectStore.removeAcl(ozoneObj, acls.get(0));
+    Assert.assertFalse(removeAcl);
+
+    boolean addAcl = objectStore.addAcl(ozoneObj, defaultUserAcl);
+    Assert.assertTrue(addAcl);
+
+    // Just changed acl type here to write, rest all is same as defaultUserAcl.
+    OzoneAcl modifiedUserAcl = new OzoneAcl(USER, remoteUserName,
+        WRITE, DEFAULT);
+    addAcl = objectStore.addAcl(ozoneObj, modifiedUserAcl);
+    Assert.assertTrue(addAcl);
+
+    removeAcl = objectStore.removeAcl(ozoneObj, modifiedUserAcl);
+    Assert.assertTrue(removeAcl);
+
+    removeAcl = objectStore.removeAcl(ozoneObj, defaultUserAcl);
+    Assert.assertTrue(removeAcl);
+
+  }
+
+  @Test
+  public void testSetBucketAcl() throws Exception {
+    OzoneBucket ozoneBucket = setupBucket();
+    String remoteUserName = "remoteUser";
+    OzoneAcl defaultUserAcl = new OzoneAcl(USER, remoteUserName,
+        READ, DEFAULT);
+
+    OzoneObj ozoneObj = OzoneObjInfo.Builder.newBuilder()
+        .setResType(OzoneObj.ResourceType.BUCKET)
+        .setStoreType(OzoneObj.StoreType.OZONE)
+        .setVolumeName(ozoneBucket.getVolumeName())
+        .setBucketName(ozoneBucket.getName()).build();
+
+    // As by default create bucket we add some default acls in RpcClient.
+    List<OzoneAcl> acls = objectStore.getAcl(ozoneObj);
+
+    Assert.assertTrue(acls.size() > 0);
+
+    OzoneAcl modifiedUserAcl = new OzoneAcl(USER, remoteUserName,
+        WRITE, DEFAULT);
+
+    List<OzoneAcl> newAcls = Collections.singletonList(modifiedUserAcl);
+    boolean setAcl = objectStore.setAcl(ozoneObj, newAcls);
+    Assert.assertTrue(setAcl);
+
+    // Get acls and check whether they are reset or not.
+    List<OzoneAcl> getAcls = objectStore.getAcl(ozoneObj);
+
+    Assert.assertTrue(newAcls.size() == getAcls.size());
+    int i = 0;
+    for (OzoneAcl ozoneAcl : newAcls) {
+      Assert.assertTrue(compareAcls(getAcls.get(i++), ozoneAcl));
+    }
+  }
+
+  private boolean containsAcl(OzoneAcl ozoneAcl, List<OzoneAcl> ozoneAcls) {
+    for (OzoneAcl acl : ozoneAcls) {
+      boolean result = compareAcls(ozoneAcl, acl);
+      if (result) {
+        // We found a match, return.
+        return result;
+      }
+    }
+    return false;
+  }
+
+  private boolean compareAcls(OzoneAcl givenAcl, OzoneAcl existingAcl) {
+    if (givenAcl.getType().equals(existingAcl.getType())
+        && givenAcl.getName().equals(existingAcl.getName())
+        && givenAcl.getAclScope().equals(existingAcl.getAclScope())) {
+      BitSet bitSet = (BitSet) givenAcl.getAclBitSet().clone();
+      bitSet.and(existingAcl.getAclBitSet());
+      if (bitSet.equals(existingAcl.getAclBitSet())) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  @Test
   public void testOMRatisSnapshot() throws Exception {
     String userName = "user" + RandomStringUtils.randomNumeric(5);
     String adminName = "admin" + RandomStringUtils.randomNumeric(5);
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/utils/OzoneManagerRatisUtils.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/utils/OzoneManagerRatisUtils.java
index 460daaa..d0dd640 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/utils/OzoneManagerRatisUtils.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/utils/OzoneManagerRatisUtils.java
@@ -23,6 +23,9 @@ import org.apache.hadoop.ozone.om.request.bucket.OMBucketCreateRequest;
 import org.apache.hadoop.ozone.om.request.bucket.OMBucketDeleteRequest;
 import org.apache.hadoop.ozone.om.request.bucket.OMBucketSetPropertyRequest;
 import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.bucket.acl.OMBucketAddAclRequest;
+import org.apache.hadoop.ozone.om.request.bucket.acl.OMBucketRemoveAclRequest;
+import org.apache.hadoop.ozone.om.request.bucket.acl.OMBucketSetAclRequest;
 import org.apache.hadoop.ozone.om.request.file.OMDirectoryCreateRequest;
 import org.apache.hadoop.ozone.om.request.file.OMFileCreateRequest;
 import org.apache.hadoop.ozone.om.request.key.OMAllocateBlockRequest;
@@ -136,19 +139,25 @@ public final class OzoneManagerRatisUtils {
       ObjectType type = omRequest.getAddAclRequest().getObj().getResType();
       if (ObjectType.VOLUME == type) {
         return new OMVolumeAddAclRequest(omRequest);
+      } else if (ObjectType.BUCKET == type) {
+        return new OMBucketAddAclRequest(omRequest);
       }
     } else if (Type.RemoveAcl == cmdType) {
-      ObjectType type = omRequest.getAddAclRequest().getObj().getResType();
+      ObjectType type = omRequest.getRemoveAclRequest().getObj().getResType();
       if (ObjectType.VOLUME == type) {
         return new OMVolumeRemoveAclRequest(omRequest);
+      } else if (ObjectType.BUCKET == type) {
+        return new OMBucketRemoveAclRequest(omRequest);
       }
     } else if (Type.SetAcl == cmdType) {
-      ObjectType type = omRequest.getAddAclRequest().getObj().getResType();
+      ObjectType type = omRequest.getSetAclRequest().getObj().getResType();
       if (ObjectType.VOLUME == type) {
         return new OMVolumeSetAclRequest(omRequest);
+      } else if (ObjectType.BUCKET == type) {
+        return new OMBucketSetAclRequest(omRequest);
       }
     }
-    //TODO: handle bucket, key and prefix AddAcl
+    //TODO: handle key and prefix AddAcl
     return null;
   }
 
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/acl/OMBucketAclRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/acl/OMBucketAclRequest.java
new file mode 100644
index 0000000..9c47419
--- /dev/null
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/acl/OMBucketAclRequest.java
@@ -0,0 +1,186 @@
+/**
+ * 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.om.request.bucket.acl;
+
+import java.io.IOException;
+import java.util.List;
+
+import com.google.common.base.Optional;
+import org.apache.hadoop.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMetrics;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.util.BooleanBiFunction;
+import org.apache.hadoop.ozone.om.request.util.ObjectParser;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OzoneObj.ObjectType;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.utils.db.cache.CacheKey;
+import org.apache.hadoop.utils.db.cache.CacheValue;
+
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * Base class for Bucket acl request.
+ */
+public abstract class OMBucketAclRequest extends OMClientRequest {
+
+  private BooleanBiFunction<List<OzoneAcl>, OmBucketInfo> omBucketAclOp;
+
+  public OMBucketAclRequest(OMRequest omRequest,
+      BooleanBiFunction<List<OzoneAcl>, OmBucketInfo> aclOp) {
+    super(omRequest);
+    omBucketAclOp = aclOp;
+  }
+
+  @Override
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long transactionLogIndex,
+      OzoneManagerDoubleBufferHelper ozoneManagerDoubleBufferHelper) {
+
+    // protobuf guarantees acls are non-null.
+    List<OzoneAcl> ozoneAcls = getAcls();
+
+    OMMetrics omMetrics = ozoneManager.getMetrics();
+    omMetrics.incNumBucketUpdates();
+    OmBucketInfo omBucketInfo = null;
+
+    OMResponse.Builder omResponse = onInit();
+    OMClientResponse omClientResponse = null;
+    IOException exception = null;
+
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    boolean lockAcquired = false;
+    String volume = null;
+    String bucket = null;
+    boolean operationResult = false;
+    try {
+      ObjectParser objectParser = new ObjectParser(getPath(),
+          ObjectType.BUCKET);
+
+      volume = objectParser.getVolume();
+      bucket = objectParser.getBucket();
+
+      // check Acl
+      if (ozoneManager.getAclsEnabled()) {
+        checkAcls(ozoneManager, OzoneObj.ResourceType.VOLUME,
+            OzoneObj.StoreType.OZONE, IAccessAuthorizer.ACLType.WRITE_ACL,
+            volume, null, null);
+      }
+      lockAcquired =
+          omMetadataManager.getLock().acquireLock(BUCKET_LOCK, volume, bucket);
+
+      String dbBucketKey = omMetadataManager.getBucketKey(volume, bucket);
+      omBucketInfo = omMetadataManager.getBucketTable().get(dbBucketKey);
+      if (omBucketInfo == null) {
+        throw new OMException(OMException.ResultCodes.BUCKET_NOT_FOUND);
+      }
+
+      operationResult = omBucketAclOp.apply(ozoneAcls, omBucketInfo);
+
+      if (operationResult) {
+        // update cache.
+        omMetadataManager.getBucketTable().addCacheEntry(
+            new CacheKey<>(dbBucketKey),
+            new CacheValue<>(Optional.of(omBucketInfo), transactionLogIndex));
+      }
+
+      omClientResponse = onSuccess(omResponse, omBucketInfo, operationResult);
+
+    } catch (IOException ex) {
+      exception = ex;
+      omClientResponse = onFailure(omResponse, ex);
+    } finally {
+      if (omClientResponse != null) {
+        omClientResponse.setFlushFuture(
+            ozoneManagerDoubleBufferHelper.add(omClientResponse,
+                transactionLogIndex));
+      }
+      if (lockAcquired) {
+        omMetadataManager.getLock().releaseLock(BUCKET_LOCK, volume, bucket);
+      }
+    }
+
+
+    onComplete(operationResult, exception, ozoneManager.getMetrics());
+
+    return omClientResponse;
+  }
+
+  /**
+   * Get the Acls from the request.
+   * @return List of OzoneAcls, for add/remove it is a single element list
+   * for set it can be non-single element list.
+   */
+  abstract List<OzoneAcl> getAcls();
+
+  /**
+   * Get the path name from the request.
+   * @return path name
+   */
+  abstract String getPath();
+
+  // TODO: Finer grain metrics can be moved to these callbacks. They can also
+  // be abstracted into separate interfaces in future.
+  /**
+   * Get the initial om response builder with lock.
+   * @return om response builder.
+   */
+  abstract OMResponse.Builder onInit();
+
+  /**
+   * Get the om client response on success case with lock.
+   * @param omResponse
+   * @param omBucketInfo
+   * @param operationResult
+   * @return OMClientResponse
+   */
+  abstract OMClientResponse onSuccess(
+      OMResponse.Builder omResponse, OmBucketInfo omBucketInfo,
+      boolean operationResult);
+
+  /**
+   * Get the om client response on failure case with lock.
+   * @param omResponse
+   * @param exception
+   * @return OMClientResponse
+   */
+  abstract OMClientResponse onFailure(OMResponse.Builder omResponse,
+      IOException exception);
+
+  /**
+   * Completion hook for final processing before return without lock.
+   * Usually used for logging without lock and metric update.
+   * @param operationResult
+   * @param exception
+   * @param omMetrics
+   */
+  abstract void onComplete(boolean operationResult, IOException exception,
+      OMMetrics omMetrics);
+
+
+}
+
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/acl/OMBucketAddAclRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/acl/OMBucketAddAclRequest.java
new file mode 100644
index 0000000..41aef6d
--- /dev/null
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/acl/OMBucketAddAclRequest.java
@@ -0,0 +1,122 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <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.om.request.bucket.acl;
+
+import java.io.IOException;
+import java.util.List;
+
+import com.google.common.collect.Lists;
+import org.apache.hadoop.ozone.om.OMMetrics;
+import org.apache.hadoop.ozone.util.BooleanBiFunction;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.bucket.acl.OMBucketAclResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .AddAclResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .OMResponse;
+
+/**
+ * Handle add Acl request for bucket.
+ */
+public class OMBucketAddAclRequest extends OMBucketAclRequest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OMBucketAddAclRequest.class);
+
+  private static BooleanBiFunction<List<OzoneAcl>, OmBucketInfo> bucketAddAclOp;
+  private String path;
+  private List<OzoneAcl> ozoneAcls;
+
+  static {
+    bucketAddAclOp = (ozoneAcls, omBucketInfo) -> {
+      return omBucketInfo.addAcl(ozoneAcls.get(0));
+    };
+  }
+
+  public OMBucketAddAclRequest(OMRequest omRequest) {
+    super(omRequest, bucketAddAclOp);
+    OzoneManagerProtocolProtos.AddAclRequest addAclRequest =
+        getOmRequest().getAddAclRequest();
+    path = addAclRequest.getObj().getPath();
+    ozoneAcls = Lists.newArrayList(
+        OzoneAcl.fromProtobuf(addAclRequest.getAcl()));
+  }
+
+  @Override
+  List<OzoneAcl> getAcls() {
+    return ozoneAcls;
+  }
+
+  @Override
+  String getPath() {
+    return path;
+  }
+
+  @Override
+  OMResponse.Builder onInit() {
+    return OMResponse.newBuilder().setCmdType(
+        OzoneManagerProtocolProtos.Type.AddAcl).setStatus(
+        OzoneManagerProtocolProtos.Status.OK).setSuccess(true);
+
+  }
+
+  @Override
+  OMClientResponse onSuccess(OMResponse.Builder omResponse,
+      OmBucketInfo omBucketInfo, boolean operationResult) {
+    omResponse.setSuccess(operationResult);
+    omResponse.setAddAclResponse(AddAclResponse.newBuilder()
+         .setResponse(operationResult));
+    return new OMBucketAclResponse(omBucketInfo,
+        omResponse.build());
+  }
+
+  @Override
+  OMClientResponse onFailure(OMResponse.Builder omResponse,
+      IOException exception) {
+    return new OMBucketAclResponse(null,
+        createErrorOMResponse(omResponse, exception));
+  }
+
+  @Override
+  void onComplete(boolean operationResult, IOException exception,
+      OMMetrics omMetrics) {
+    if (operationResult) {
+      LOG.debug("Add acl: {} to path: {} success!", getAcls(), getPath());
+    } else {
+      omMetrics.incNumBucketUpdateFails();
+      if (exception == null) {
+        LOG.error("Add acl {} to path {} failed, because acl already exist",
+            getAcls(), getPath());
+      } else {
+        LOG.error("Add acl {} to path {} failed!", getAcls(), getPath(),
+            exception);
+      }
+    }
+  }
+
+}
+
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/acl/OMBucketRemoveAclRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/acl/OMBucketRemoveAclRequest.java
new file mode 100644
index 0000000..cc60897
--- /dev/null
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/acl/OMBucketRemoveAclRequest.java
@@ -0,0 +1,101 @@
+package org.apache.hadoop.ozone.om.request.bucket.acl;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Lists;
+import org.apache.hadoop.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.om.OMMetrics;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.util.BooleanBiFunction;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.bucket.acl.OMBucketAclResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.RemoveAclResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+
+
+/**
+ * Handle removeAcl request for bucket.
+ */
+public class OMBucketRemoveAclRequest extends OMBucketAclRequest {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OMBucketAddAclRequest.class);
+
+  private static BooleanBiFunction<List<OzoneAcl>, OmBucketInfo> bucketAddAclOp;
+  private String path;
+  private List<OzoneAcl> ozoneAcls;
+
+  static {
+    bucketAddAclOp = (ozoneAcls, omBucketInfo) -> {
+      return omBucketInfo.removeAcl(ozoneAcls.get(0));
+    };
+  }
+
+  public OMBucketRemoveAclRequest(OMRequest omRequest) {
+    super(omRequest, bucketAddAclOp);
+    OzoneManagerProtocolProtos.RemoveAclRequest removeAclRequest =
+        getOmRequest().getRemoveAclRequest();
+    path = removeAclRequest.getObj().getPath();
+    ozoneAcls = Lists.newArrayList(
+        OzoneAcl.fromProtobuf(removeAclRequest.getAcl()));
+  }
+
+  @Override
+  List<OzoneAcl> getAcls() {
+    return ozoneAcls;
+  }
+
+  @Override
+  String getPath() {
+    return path;
+  }
+
+  @Override
+  OMResponse.Builder onInit() {
+    return OMResponse.newBuilder().setCmdType(
+        OzoneManagerProtocolProtos.Type.RemoveAcl).setStatus(
+        OzoneManagerProtocolProtos.Status.OK).setSuccess(true);
+
+  }
+
+  @Override
+  OMClientResponse onSuccess(OMResponse.Builder omResponse,
+      OmBucketInfo omBucketInfo, boolean operationResult) {
+    omResponse.setSuccess(operationResult);
+    omResponse.setRemoveAclResponse(RemoveAclResponse.newBuilder()
+        .setResponse(operationResult));
+    return new OMBucketAclResponse(omBucketInfo,
+        omResponse.build());
+  }
+
+  @Override
+  OMClientResponse onFailure(OMResponse.Builder omResponse,
+      IOException exception) {
+    return new OMBucketAclResponse(null,
+        createErrorOMResponse(omResponse, exception));
+  }
+
+  @Override
+  void onComplete(boolean operationResult, IOException exception,
+      OMMetrics omMetrics) {
+    if (operationResult) {
+      LOG.debug("Remove acl: {} for path: {} success!", getAcls(), getPath());
+    } else {
+      omMetrics.incNumBucketUpdateFails();
+      if (exception == null) {
+        LOG.error("Remove acl {} for path {} failed, because acl does not " +
+                "exist",
+            getAcls(), getPath());
+      } else {
+        LOG.error("Remove acl {} for path {} failed!", getAcls(), getPath(),
+            exception);
+      }
+    }
+  }
+}
+
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/acl/OMBucketSetAclRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/acl/OMBucketSetAclRequest.java
new file mode 100644
index 0000000..7767898
--- /dev/null
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/acl/OMBucketSetAclRequest.java
@@ -0,0 +1,100 @@
+package org.apache.hadoop.ozone.om.request.bucket.acl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.om.OMMetrics;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.util.BooleanBiFunction;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.bucket.acl.OMBucketAclResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SetAclResponse;
+
+/**
+ * Handle setAcl request for bucket.
+ */
+public class OMBucketSetAclRequest extends OMBucketAclRequest {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OMBucketAddAclRequest.class);
+
+  private static BooleanBiFunction< List<OzoneAcl>,
+        OmBucketInfo > bucketAddAclOp;
+  private String path;
+  private List<OzoneAcl> ozoneAcls;
+
+  static {
+    bucketAddAclOp = (ozoneAcls, omBucketInfo) -> {
+      return omBucketInfo.setAcls(ozoneAcls);
+    };
+  }
+
+  public OMBucketSetAclRequest(OMRequest omRequest) {
+    super(omRequest, bucketAddAclOp);
+    OzoneManagerProtocolProtos.SetAclRequest setAclRequest =
+        getOmRequest().getSetAclRequest();
+    path = setAclRequest.getObj().getPath();
+    ozoneAcls = new ArrayList<>();
+    setAclRequest.getAclList().forEach(aclInfo ->
+        ozoneAcls.add(OzoneAcl.fromProtobuf(aclInfo)));
+  }
+
+  @Override
+  List<OzoneAcl> getAcls() {
+    return ozoneAcls;
+  }
+
+  @Override
+  String getPath() {
+    return path;
+  }
+
+  @Override
+  OMResponse.Builder onInit() {
+    return OMResponse.newBuilder().setCmdType(
+        OzoneManagerProtocolProtos.Type.SetAcl).setStatus(
+        OzoneManagerProtocolProtos.Status.OK).setSuccess(true);
+
+  }
+
+  @Override
+  OMClientResponse onSuccess(OMResponse.Builder omResponse,
+      OmBucketInfo omBucketInfo, boolean operationResult) {
+    omResponse.setSuccess(operationResult);
+    omResponse.setSetAclResponse(SetAclResponse.newBuilder()
+        .setResponse(operationResult));
+    return new OMBucketAclResponse(omBucketInfo,
+        omResponse.build());
+  }
+
+  @Override
+  OMClientResponse onFailure(OMResponse.Builder omResponse,
+      IOException exception) {
+    return new OMBucketAclResponse(null,
+        createErrorOMResponse(omResponse, exception));
+  }
+
+  @Override
+  void onComplete(boolean operationResult, IOException exception,
+      OMMetrics omMetrics) {
+    if (operationResult) {
+      LOG.debug("Set acl: {} for path: {} success!", getAcls(), getPath());
+    } else {
+      omMetrics.incNumBucketUpdateFails();
+      if (exception == null) {
+        LOG.error("Set acl {} for path {} failed", getAcls(), getPath());
+      } else {
+        LOG.error("Set acl {} for path {} failed!", getAcls(), getPath(),
+            exception);
+      }
+    }
+  }
+}
+
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/acl/package-info.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/acl/package-info.java
new file mode 100644
index 0000000..7b3b43d5
--- /dev/null
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/acl/package-info.java
@@ -0,0 +1,23 @@
+/**
+ * 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.
+ */
+
+/**
+ * This package contains classes for handling acl requests for bucket.
+ */
+package org.apache.hadoop.ozone.om.request.bucket.acl;
+
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/util/ObjectParser.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/util/ObjectParser.java
new file mode 100644
index 0000000..7b258a0
--- /dev/null
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/util/ObjectParser.java
@@ -0,0 +1,74 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <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.om.request.util;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .OzoneObj.ObjectType;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_DELIMITER;
+
+/**
+ * Utility class to parse {@link OzoneObj#getPath()}.
+ */
+public class ObjectParser {
+
+  private String volume;
+  private String bucket;
+  private String key;
+
+  /**
+   * Parse the path and extract volume, bucket and key names.
+   * @param path
+   */
+  public ObjectParser(String path, ObjectType objectType) throws OMException {
+    Preconditions.checkNotNull(path);
+    String[] tokens = StringUtils.split(path, OZONE_URI_DELIMITER, 3);
+
+    if (objectType == ObjectType.VOLUME && tokens.length == 1) {
+      volume = tokens[0];
+    } else if (objectType == ObjectType.BUCKET && tokens.length == 2) {
+      volume = tokens[0];
+      bucket = tokens[1];
+    } else if (objectType == ObjectType.KEY && tokens.length == 3) {
+      volume = tokens[0];
+      bucket = tokens[1];
+      key = tokens[3];
+    } else {
+      throw new OMException("Illegal path " + path,
+          OMException.ResultCodes.INVALID_PATH_IN_ACL_REQUEST);
+    }
+  }
+
+  public String getVolume() {
+    return volume;
+  }
+
+  public String getBucket() {
+    return bucket;
+  }
+
+  public String getKey() {
+    return key;
+  }
+}
+
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/util/package-info.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/util/package-info.java
new file mode 100644
index 0000000..72fc09a
--- /dev/null
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/util/package-info.java
@@ -0,0 +1,23 @@
+/**
+ * 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 contains helper/utility classes for requests.
+ */
+package org.apache.hadoop.ozone.om.request.util;
+
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/bucket/acl/OMBucketAclResponse.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/bucket/acl/OMBucketAclResponse.java
new file mode 100644
index 0000000..a8c5b86
--- /dev/null
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/bucket/acl/OMBucketAclResponse.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <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.om.response.bucket.acl;
+
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .OMResponse;
+import org.apache.hadoop.utils.db.BatchOperation;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import java.io.IOException;
+
+/**
+ * Response for Bucket acl request.
+ */
+public class OMBucketAclResponse extends OMClientResponse {
+
+  private final OmBucketInfo omBucketInfo;
+
+  public OMBucketAclResponse(@Nullable OmBucketInfo omBucketInfo,
+      @Nonnull OMResponse omResponse) {
+    super(omResponse);
+    this.omBucketInfo = omBucketInfo;
+  }
+
+  @Override
+  public void addToDBBatch(OMMetadataManager omMetadataManager,
+      BatchOperation batchOperation) throws IOException {
+
+    // If response status is OK and success is true, add to DB batch.
+    if (getOMResponse().getStatus() == OzoneManagerProtocolProtos.Status.OK &&
+        getOMResponse().getSuccess()) {
+      String dbBucketKey =
+          omMetadataManager.getBucketKey(omBucketInfo.getVolumeName(),
+              omBucketInfo.getBucketName());
+      omMetadataManager.getBucketTable().putWithBatch(batchOperation,
+          dbBucketKey, omBucketInfo);
+    }
+  }
+
+}
+
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/bucket/acl/package-info.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/bucket/acl/package-info.java
new file mode 100644
index 0000000..dd26272
--- /dev/null
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/bucket/acl/package-info.java
@@ -0,0 +1,22 @@
+/**
+ * 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.
+ */
+
+/**
+ * This package contains classes for handling bucket acl responses.
+ */
+package org.apache.hadoop.ozone.om.response.bucket.acl;
\ No newline at end of file


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