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 xy...@apache.org on 2017/06/07 22:00:41 UTC

hadoop git commit: HDFS-11880. Ozone: KSM: Remove protobuf formats from KSM wrappers. Contributed by Nandakumar Vadivelu.

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-7240 18706f433 -> da8bc385a


HDFS-11880. Ozone: KSM: Remove protobuf formats from KSM wrappers. Contributed by Nandakumar Vadivelu.


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

Branch: refs/heads/HDFS-7240
Commit: da8bc385a754519ab2d7c76f6c3540054778d90c
Parents: 18706f4
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Wed Jun 7 14:49:44 2017 -0700
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Wed Jun 7 14:49:44 2017 -0700

----------------------------------------------------------------------
 .../hadoop/ksm/helpers/KsmBucketArgs.java       |  59 +++---
 .../hadoop/ksm/helpers/KsmBucketInfo.java       |  47 +++--
 .../java/org/apache/hadoop/ozone/OzoneAcl.java  | 206 ++++++++++++++++++
 .../org/apache/hadoop/ozone/OzoneConsts.java    | 112 ++++++++++
 .../org/apache/hadoop/ozone/package-info.java   |  22 ++
 .../hadoop/ozone/protocolPB/KSMPBHelper.java    | 113 ++++++++++
 .../hadoop/ozone/protocolPB/package-info.java   |  24 +++
 .../org/apache/hadoop/ozone/TestOzoneAcls.java  | 138 ++++++++++++
 .../org/apache/hadoop/ozone/OzoneConsts.java    | 112 ----------
 .../hadoop/ozone/ksm/BucketManagerImpl.java     |  47 ++---
 .../apache/hadoop/ozone/ksm/KSMConfigKeys.java  |   2 +-
 .../hadoop/ozone/protocolPB/KSMPBHelper.java    | 113 ----------
 .../hadoop/ozone/web/client/OzoneBucket.java    |   2 +-
 .../hadoop/ozone/web/handlers/BucketArgs.java   |   2 +-
 .../ozone/web/interfaces/StorageHandler.java    |   2 +-
 .../web/localstorage/LocalStorageHandler.java   |   2 +-
 .../web/localstorage/OzoneMetadataManager.java  |   2 +-
 .../hadoop/ozone/web/request/OzoneAcl.java      | 208 -------------------
 .../hadoop/ozone/web/response/BucketInfo.java   |   2 +-
 .../web/storage/DistributedStorageHandler.java  |  25 +--
 .../hadoop/ozone/ksm/TestBucketManagerImpl.java |  47 ++---
 .../hadoop/ozone/ksm/TestKeySpaceManager.java   |   2 +-
 .../apache/hadoop/ozone/web/TestBucketInfo.java |   2 +-
 .../apache/hadoop/ozone/web/TestOzoneAcls.java  | 139 -------------
 24 files changed, 729 insertions(+), 701 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/da8bc385/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/KsmBucketArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/KsmBucketArgs.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/KsmBucketArgs.java
index f867628..a682f5f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/KsmBucketArgs.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/KsmBucketArgs.java
@@ -18,14 +18,15 @@
 package org.apache.hadoop.ksm.helpers;
 
 import java.util.List;
+import java.util.stream.Collectors;
 
 import com.google.common.base.Preconditions;
-import org.apache.hadoop.hdfs.protocol.proto
-    .HdfsProtos.StorageTypeProto;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
+import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.protocol.proto
     .KeySpaceManagerProtocolProtos.BucketArgs;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.OzoneAclInfo;
+import org.apache.hadoop.ozone.protocolPB.KSMPBHelper;
 
 /**
  * A class that encapsulates Bucket Arguments.
@@ -42,11 +43,11 @@ public final class KsmBucketArgs {
   /**
    * ACL's that are to be added for the bucket.
    */
-  private List<OzoneAclInfo> addAcls;
+  private List<OzoneAcl> addAcls;
   /**
    * ACL's that are to be removed from the bucket.
    */
-  private List<OzoneAclInfo> removeAcls;
+  private List<OzoneAcl> removeAcls;
   /**
    * Bucket Version flag.
    */
@@ -55,7 +56,7 @@ public final class KsmBucketArgs {
    * Type of storage to be used for this bucket.
    * [RAM_DISK, SSD, DISK, ARCHIVE]
    */
-  private StorageTypeProto storageType;
+  private StorageType storageType;
 
   /**
    * Private constructor, constructed via builder.
@@ -67,8 +68,8 @@ public final class KsmBucketArgs {
    * @param storageType - Storage type to be used.
    */
   private KsmBucketArgs(String volumeName, String bucketName,
-      List<OzoneAclInfo> addAcls, List<OzoneAclInfo> removeAcls,
-      Boolean isVersionEnabled, StorageTypeProto storageType) {
+      List<OzoneAcl> addAcls, List<OzoneAcl> removeAcls,
+      Boolean isVersionEnabled, StorageType storageType) {
     this.volumeName = volumeName;
     this.bucketName = bucketName;
     this.addAcls = addAcls;
@@ -97,7 +98,7 @@ public final class KsmBucketArgs {
    * Returns the ACL's that are to be added.
    * @return List<OzoneAclInfo>
    */
-  public List<OzoneAclInfo> getAddAcls() {
+  public List<OzoneAcl> getAddAcls() {
     return addAcls;
   }
 
@@ -105,7 +106,7 @@ public final class KsmBucketArgs {
    * Returns the ACL's that are to be removed.
    * @return List<OzoneAclInfo>
    */
-  public List<OzoneAclInfo> getRemoveAcls() {
+  public List<OzoneAcl> getRemoveAcls() {
     return removeAcls;
   }
 
@@ -121,7 +122,7 @@ public final class KsmBucketArgs {
    * Returns the type of storage to be used.
    * @return StorageType
    */
-  public StorageTypeProto getStorageType() {
+  public StorageType getStorageType() {
     return storageType;
   }
 
@@ -140,10 +141,10 @@ public final class KsmBucketArgs {
   public static class Builder {
     private String volumeName;
     private String bucketName;
-    private List<OzoneAclInfo> addAcls;
-    private List<OzoneAclInfo> removeAcls;
+    private List<OzoneAcl> addAcls;
+    private List<OzoneAcl> removeAcls;
     private Boolean isVersionEnabled;
-    private StorageTypeProto storageType;
+    private StorageType storageType;
 
     public Builder setVolumeName(String volume) {
       this.volumeName = volume;
@@ -155,12 +156,12 @@ public final class KsmBucketArgs {
       return this;
     }
 
-    public Builder setAddAcls(List<OzoneAclInfo> acls) {
+    public Builder setAddAcls(List<OzoneAcl> acls) {
       this.addAcls = acls;
       return this;
     }
 
-    public Builder setRemoveAcls(List<OzoneAclInfo> acls) {
+    public Builder setRemoveAcls(List<OzoneAcl> acls) {
       this.removeAcls = acls;
       return this;
     }
@@ -170,7 +171,7 @@ public final class KsmBucketArgs {
       return this;
     }
 
-    public Builder setStorageType(StorageTypeProto storage) {
+    public Builder setStorageType(StorageType storage) {
       this.storageType = storage;
       return this;
     }
@@ -195,16 +196,19 @@ public final class KsmBucketArgs {
     builder.setVolumeName(volumeName)
         .setBucketName(bucketName);
     if(addAcls != null && !addAcls.isEmpty()) {
-      builder.addAllAddAcls(addAcls);
+      builder.addAllAddAcls(addAcls.stream().map(
+          KSMPBHelper::convertOzoneAcl).collect(Collectors.toList()));
     }
     if(removeAcls != null && !removeAcls.isEmpty()) {
-      builder.addAllRemoveAcls(removeAcls);
+      builder.addAllRemoveAcls(removeAcls.stream().map(
+          KSMPBHelper::convertOzoneAcl).collect(Collectors.toList()));
     }
     if(isVersionEnabled != null) {
       builder.setIsVersionEnabled(isVersionEnabled);
     }
     if(storageType != null) {
-      builder.setStorageType(storageType);
+      builder.setStorageType(
+          PBHelperClient.convertStorageType(storageType));
     }
     return builder.build();
   }
@@ -217,10 +221,13 @@ public final class KsmBucketArgs {
   public static KsmBucketArgs getFromProtobuf(BucketArgs bucketArgs) {
     return new KsmBucketArgs(bucketArgs.getVolumeName(),
         bucketArgs.getBucketName(),
-        bucketArgs.getAddAclsList(),
-        bucketArgs.getRemoveAclsList(),
-        bucketArgs.hasIsVersionEnabled() ? bucketArgs.getIsVersionEnabled() :
-        null,
-        bucketArgs.hasStorageType() ? bucketArgs.getStorageType() : null);
+        bucketArgs.getAddAclsList().stream().map(
+            KSMPBHelper::convertOzoneAcl).collect(Collectors.toList()),
+        bucketArgs.getRemoveAclsList().stream().map(
+            KSMPBHelper::convertOzoneAcl).collect(Collectors.toList()),
+        bucketArgs.hasIsVersionEnabled() ?
+            bucketArgs.getIsVersionEnabled() : null,
+        bucketArgs.hasStorageType() ? PBHelperClient.convertStorageType(
+            bucketArgs.getStorageType()) : null);
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da8bc385/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/KsmBucketInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/KsmBucketInfo.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/KsmBucketInfo.java
index ba4c019..effbb11 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/KsmBucketInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/KsmBucketInfo.java
@@ -18,15 +18,16 @@
 package org.apache.hadoop.ksm.helpers;
 
 import com.google.common.base.Preconditions;
-import org.apache.hadoop.hdfs.protocol.proto
-    .HdfsProtos.StorageTypeProto;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
+import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.protocol.proto
     .KeySpaceManagerProtocolProtos.BucketInfo;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.OzoneAclInfo;
+import org.apache.hadoop.ozone.protocolPB.KSMPBHelper;
 
 import java.util.LinkedList;
 import java.util.List;
+import java.util.stream.Collectors;
 
 /**
  * A class that encapsulates Bucket Info.
@@ -43,7 +44,7 @@ public final class KsmBucketInfo {
   /**
    * ACL Information.
    */
-  private List<OzoneAclInfo> acls;
+  private List<OzoneAcl> acls;
   /**
    * Bucket Version flag.
    */
@@ -52,7 +53,7 @@ public final class KsmBucketInfo {
    * Type of storage to be used for this bucket.
    * [RAM_DISK, SSD, DISK, ARCHIVE]
    */
-  private StorageTypeProto storageType;
+  private StorageType storageType;
 
   /**
    * Private constructor, constructed via builder.
@@ -63,8 +64,8 @@ public final class KsmBucketInfo {
    * @param storageType - Storage type to be used.
    */
   private KsmBucketInfo(String volumeName, String bucketName,
-                        List<OzoneAclInfo> acls, boolean isVersionEnabled,
-                        StorageTypeProto storageType) {
+                        List<OzoneAcl> acls, boolean isVersionEnabled,
+                        StorageType storageType) {
     this.volumeName = volumeName;
     this.bucketName = bucketName;
     this.acls = acls;
@@ -90,9 +91,9 @@ public final class KsmBucketInfo {
 
   /**
    * Returns the ACL's associated with this bucket.
-   * @return List<OzoneAclInfo>
+   * @return List<OzoneAcl>
    */
-  public List<OzoneAclInfo> getAcls() {
+  public List<OzoneAcl> getAcls() {
     return acls;
   }
 
@@ -106,9 +107,9 @@ public final class KsmBucketInfo {
 
   /**
    * Returns the type of storage to be used.
-   * @return StorageTypeProto
+   * @return StorageType
    */
-  public StorageTypeProto getStorageType() {
+  public StorageType getStorageType() {
     return storageType;
   }
 
@@ -127,15 +128,15 @@ public final class KsmBucketInfo {
   public static class Builder {
     private String volumeName;
     private String bucketName;
-    private List<OzoneAclInfo> acls;
+    private List<OzoneAcl> acls;
     private Boolean isVersionEnabled;
-    private StorageTypeProto storageType;
+    private StorageType storageType;
 
     Builder() {
       //Default values
       this.acls = new LinkedList<>();
       this.isVersionEnabled = false;
-      this.storageType = StorageTypeProto.DISK;
+      this.storageType = StorageType.DISK;
     }
 
     public Builder setVolumeName(String volume) {
@@ -148,7 +149,7 @@ public final class KsmBucketInfo {
       return this;
     }
 
-    public Builder setAcls(List<OzoneAclInfo> listOfAcls) {
+    public Builder setAcls(List<OzoneAcl> listOfAcls) {
       this.acls = listOfAcls;
       return this;
     }
@@ -158,7 +159,7 @@ public final class KsmBucketInfo {
       return this;
     }
 
-    public Builder setStorageType(StorageTypeProto storage) {
+    public Builder setStorageType(StorageType storage) {
       this.storageType = storage;
       return this;
     }
@@ -185,9 +186,11 @@ public final class KsmBucketInfo {
     return BucketInfo.newBuilder()
         .setVolumeName(volumeName)
         .setBucketName(bucketName)
-        .addAllAcls(acls)
+        .addAllAcls(acls.stream().map(
+            KSMPBHelper::convertOzoneAcl).collect(Collectors.toList()))
         .setIsVersionEnabled(isVersionEnabled)
-        .setStorageType(storageType)
+        .setStorageType(PBHelperClient.convertStorageType(
+            storageType))
         .build();
   }
 
@@ -200,8 +203,10 @@ public final class KsmBucketInfo {
     return new KsmBucketInfo(
         bucketInfo.getVolumeName(),
         bucketInfo.getBucketName(),
-        bucketInfo.getAclsList(),
+        bucketInfo.getAclsList().stream().map(
+            KSMPBHelper::convertOzoneAcl).collect(Collectors.toList()),
         bucketInfo.getIsVersionEnabled(),
-        bucketInfo.getStorageType());
+        PBHelperClient.convertStorageType(
+            bucketInfo.getStorageType()));
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da8bc385/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/OzoneAcl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/OzoneAcl.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/OzoneAcl.java
new file mode 100644
index 0000000..0aca0ad
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/OzoneAcl.java
@@ -0,0 +1,206 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+
+package org.apache.hadoop.ozone;
+
+import java.util.Objects;
+
+/**
+ * OzoneACL classes define bucket ACLs used in OZONE.
+ *
+ * ACLs in Ozone follow this pattern.
+ * • user:name:rw
+ * • group:name:rw
+ * • world::rw
+ */
+public class OzoneAcl {
+  private OzoneACLType type;
+  private String name;
+  private OzoneACLRights rights;
+
+  /**
+   * Constructor for OzoneAcl.
+   */
+  public OzoneAcl() {
+  }
+
+  /**
+   * Constructor for OzoneAcl.
+   *
+   * @param type - Type
+   * @param name - Name of user
+   * @param rights - Rights
+   */
+  public OzoneAcl(OzoneACLType type, String name, OzoneACLRights rights) {
+    this.name = name;
+    this.rights = rights;
+    this.type = type;
+    if (type == OzoneACLType.WORLD && name.length() != 0) {
+      throw new IllegalArgumentException("Unexpected name part in world type");
+    }
+    if (((type == OzoneACLType.USER) || (type == OzoneACLType.GROUP))
+        && (name.length() == 0)) {
+      throw new IllegalArgumentException("User or group name is required");
+    }
+  }
+
+  /**
+   * Parses an ACL string and returns the ACL object.
+   *
+   * @param acl - Acl String , Ex. user:anu:rw
+   *
+   * @return - Ozone ACLs
+   */
+  public static OzoneAcl parseAcl(String acl) throws IllegalArgumentException {
+    if ((acl == null) || acl.isEmpty()) {
+      throw new IllegalArgumentException("ACLs cannot be null or empty");
+    }
+    String[] parts = acl.trim().split(":");
+    if (parts.length < 3) {
+      throw new IllegalArgumentException("ACLs are not in expected format");
+    }
+
+    OzoneACLType aclType = OzoneACLType.valueOf(parts[0].toUpperCase());
+    OzoneACLRights rights = OzoneACLRights.getACLRight(parts[2].toLowerCase());
+
+    // TODO : Support sanitation of these user names by calling into
+    // userAuth Interface.
+    return new OzoneAcl(aclType, parts[1], rights);
+  }
+
+  /**
+   * Returns a hash code value for the object. This method is
+   * supported for the benefit of hash tables.
+   *
+   * @return a hash code value for this object.
+   *
+   * @see Object#equals(Object)
+   * @see System#identityHashCode
+   */
+  @Override
+  public int hashCode() {
+    return Objects.hash(this.getName(), this.getRights().toString(),
+                        this.getType().toString());
+  }
+
+  /**
+   * Returns name.
+   *
+   * @return name
+   */
+  public String getName() {
+    return name;
+  }
+
+  /**
+   * Returns Rights.
+   *
+   * @return - Rights
+   */
+  public OzoneACLRights getRights() {
+    return rights;
+  }
+
+  /**
+   * Returns Type.
+   *
+   * @return type
+   */
+  public OzoneACLType getType() {
+    return type;
+  }
+
+  /**
+   * Indicates whether some other object is "equal to" this one.
+   *
+   * @param obj the reference object with which to compare.
+   *
+   * @return {@code true} if this object is the same as the obj
+   * argument; {@code false} otherwise.
+   */
+  @Override
+  public boolean equals(Object obj) {
+    if (obj == null) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    OzoneAcl otherAcl = (OzoneAcl) obj;
+    return otherAcl.getName().equals(this.getName()) &&
+        otherAcl.getRights() == this.getRights() &&
+        otherAcl.getType() == this.getType();
+  }
+
+  /**
+   * ACL types.
+   */
+  public enum OzoneACLType {
+    USER(OzoneConsts.OZONE_ACL_USER_TYPE),
+    GROUP(OzoneConsts.OZONE_ACL_GROUP_TYPE),
+    WORLD(OzoneConsts.OZONE_ACL_WORLD_TYPE);
+
+    /**
+     * String value for this Enum.
+     */
+    private final String value;
+
+    /**
+     * Init OzoneACLtypes enum.
+     *
+     * @param val String type for this enum.
+     */
+    OzoneACLType(String val) {
+      value = val;
+    }
+  }
+
+  /**
+   * ACL rights.
+   */
+  public enum OzoneACLRights {
+    READ, WRITE, READ_WRITE;
+
+    /**
+     * Returns the ACL rights based on passed in String.
+     *
+     * @param type ACL right string
+     *
+     * @return OzoneACLRights
+     */
+    public static OzoneACLRights getACLRight(String type) {
+      if (type == null || type.isEmpty()) {
+        throw new IllegalArgumentException("ACL right cannot be empty");
+      }
+
+      switch (type) {
+      case OzoneConsts.OZONE_ACL_READ:
+        return OzoneACLRights.READ;
+      case OzoneConsts.OZONE_ACL_WRITE:
+        return OzoneACLRights.WRITE;
+      case OzoneConsts.OZONE_ACL_READ_WRITE:
+      case OzoneConsts.OZONE_ACL_WRITE_READ:
+        return OzoneACLRights.READ_WRITE;
+      default:
+        throw new IllegalArgumentException("ACL right is not recognized");
+      }
+
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da8bc385/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
new file mode 100644
index 0000000..a783249
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * Set of constants used in Ozone implementation.
+ */
+@InterfaceAudience.Private
+public final class OzoneConsts {
+  public static final String OZONE_SIMPLE_ROOT_USER = "root";
+  public static final String OZONE_SIMPLE_HDFS_USER = "hdfs";
+
+  /*
+   * BucketName length is used for both buckets and volume lengths
+   */
+  public static final int OZONE_MIN_BUCKET_NAME_LENGTH = 3;
+  public static final int OZONE_MAX_BUCKET_NAME_LENGTH = 63;
+
+  public static final String OZONE_ACL_USER_TYPE = "user";
+  public static final String OZONE_ACL_GROUP_TYPE = "group";
+  public static final String OZONE_ACL_WORLD_TYPE = "world";
+
+  public static final String OZONE_ACL_READ = "r";
+  public static final String OZONE_ACL_WRITE = "w";
+  public static final String OZONE_ACL_READ_WRITE = "rw";
+  public static final String OZONE_ACL_WRITE_READ = "wr";
+
+  public static final String OZONE_DATE_FORMAT =
+      "EEE, dd MMM yyyy HH:mm:ss zzz";
+  public static final String OZONE_TIME_ZONE = "GMT";
+
+  public static final String OZONE_COMPONENT = "component";
+  public static final String OZONE_FUNCTION  = "function";
+  public static final String OZONE_RESOURCE = "resource";
+  public static final String OZONE_USER = "user";
+  public static final String OZONE_REQUEST = "request";
+
+  public static final String CONTAINER_EXTENSION = ".container";
+  public static final String CONTAINER_META = ".meta";
+
+  //  container storage is in the following format.
+  //  Data Volume basePath/containers/<containerName>/metadata and
+  //  Data Volume basePath/containers/<containerName>/data/...
+  public static final String CONTAINER_PREFIX  = "containers";
+  public static final String CONTAINER_META_PATH = "metadata";
+  public static final String CONTAINER_DATA_PATH = "data";
+  public static final String CONTAINER_ROOT_PREFIX = "repository";
+
+  public static final String FILE_HASH = "SHA-256";
+  public final static String CHUNK_OVERWRITE = "OverWriteRequested";
+
+  public static final int CHUNK_SIZE = 1 * 1024 * 1024; // 1 MB
+  public static final long KB = 1024L;
+  public static final long MB = KB * 1024L;
+  public static final long GB = MB * 1024L;
+  public static final long TB = GB * 1024L;
+
+  /**
+   * level DB names used by SCM and data nodes.
+   */
+  public static final String CONTAINER_DB = "container.db";
+  public static final String BLOCK_DB = "block.db";
+  public static final String NODEPOOL_DB = "nodepool.db";
+  public static final String OPEN_CONTAINERS_DB = "openContainers.db";
+  public static final String KSM_DB_NAME = "ksm.db";
+
+  /**
+   * Supports Bucket Versioning.
+   */
+  public enum Versioning {NOT_DEFINED, ENABLED, DISABLED}
+
+  /**
+   * Ozone handler types.
+   */
+  public static final String OZONE_HANDLER_DISTRIBUTED = "distributed";
+  public static final String OZONE_HANDLER_LOCAL = "local";
+
+  /**
+   * KSM LevelDB prefixes.
+   */
+  public static final String KSM_VOLUME_PREFIX = "/";
+  public static final String KSM_BUCKET_PREFIX = KSM_VOLUME_PREFIX;
+  public static final String KSM_KEY_PREFIX = KSM_VOLUME_PREFIX;
+  public static final String KSM_USER_PREFIX = "$";
+
+  /**
+   * Max KSM Quota size of 1024 PB.
+   */
+  public static final long MAX_QUOTA_IN_BYTES = 1024L * 1024 * TB;
+
+  private OzoneConsts() {
+    // Never Constructed
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da8bc385/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/package-info.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/package-info.java
new file mode 100644
index 0000000..764ff3c
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/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
+ *
+ *     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;
+
+/**
+ This package contains ozone client side libraries.
+ */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da8bc385/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/protocolPB/KSMPBHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/protocolPB/KSMPBHelper.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/protocolPB/KSMPBHelper.java
new file mode 100644
index 0000000..fdc3ce7
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/protocolPB/KSMPBHelper.java
@@ -0,0 +1,113 @@
+/**
+ * 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.protocolPB;
+
+import org.apache.hadoop.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.OzoneAclInfo;
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.OzoneAclInfo.OzoneAclType;
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.OzoneAclInfo.OzoneAclRights;
+
+/**
+ * Utilities for converting protobuf classes.
+ */
+public final class KSMPBHelper {
+
+  private KSMPBHelper() {
+    /** Hidden constructor */
+  }
+
+  /**
+   * Converts OzoneAcl into protobuf's OzoneAclInfo.
+   * @return OzoneAclInfo
+   */
+  public static OzoneAclInfo convertOzoneAcl(OzoneAcl acl) {
+    OzoneAclInfo.OzoneAclType aclType;
+    switch(acl.getType()) {
+    case USER:
+      aclType = OzoneAclType.USER;
+      break;
+    case GROUP:
+      aclType = OzoneAclType.GROUP;
+      break;
+    case WORLD:
+      aclType = OzoneAclType.WORLD;
+      break;
+    default:
+      throw new IllegalArgumentException("ACL type is not recognized");
+    }
+    OzoneAclInfo.OzoneAclRights aclRights;
+    switch(acl.getRights()) {
+    case READ:
+      aclRights = OzoneAclRights.READ;
+      break;
+    case WRITE:
+      aclRights = OzoneAclRights.WRITE;
+      break;
+    case READ_WRITE:
+      aclRights = OzoneAclRights.READ_WRITE;
+      break;
+    default:
+      throw new IllegalArgumentException("ACL right is not recognized");
+    }
+
+    return OzoneAclInfo.newBuilder().setType(aclType)
+        .setName(acl.getName())
+        .setRights(aclRights)
+        .build();
+  }
+
+  /**
+   * Converts protobuf's OzoneAclInfo into OzoneAcl.
+   * @return OzoneAcl
+   */
+  public static OzoneAcl convertOzoneAcl(OzoneAclInfo aclInfo) {
+    OzoneAcl.OzoneACLType aclType;
+    switch(aclInfo.getType()) {
+    case USER:
+      aclType = OzoneAcl.OzoneACLType.USER;
+      break;
+    case GROUP:
+      aclType = OzoneAcl.OzoneACLType.GROUP;
+      break;
+    case WORLD:
+      aclType = OzoneAcl.OzoneACLType.WORLD;
+      break;
+    default:
+      throw new IllegalArgumentException("ACL type is not recognized");
+    }
+    OzoneAcl.OzoneACLRights aclRights;
+    switch(aclInfo.getRights()) {
+    case READ:
+      aclRights = OzoneAcl.OzoneACLRights.READ;
+      break;
+    case WRITE:
+      aclRights = OzoneAcl.OzoneACLRights.WRITE;
+      break;
+    case READ_WRITE:
+      aclRights = OzoneAcl.OzoneACLRights.READ_WRITE;
+      break;
+    default:
+      throw new IllegalArgumentException("ACL right is not recognized");
+    }
+
+    return new OzoneAcl(aclType, aclInfo.getName(), aclRights);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da8bc385/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/protocolPB/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/protocolPB/package-info.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/protocolPB/package-info.java
new file mode 100644
index 0000000..860386d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/protocolPB/package-info.java
@@ -0,0 +1,24 @@
+/**
+ * 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.protocolPB;
+
+/**
+ * This package contains classes for the Protocol Buffers binding of Ozone
+ * protocols.
+ */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da8bc385/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/ozone/TestOzoneAcls.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/ozone/TestOzoneAcls.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/ozone/TestOzoneAcls.java
new file mode 100644
index 0000000..e8fd4d3
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/ozone/TestOzoneAcls.java
@@ -0,0 +1,138 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone;
+
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Set;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+public class TestOzoneAcls {
+
+  @Test
+  public void TestACLParse() {
+    HashMap<String, Boolean> testMatrix;
+    testMatrix = new HashMap<>();
+
+    testMatrix.put("user:bilbo:r", Boolean.TRUE);
+    testMatrix.put("user:bilbo:w", Boolean.TRUE);
+    testMatrix.put("user:bilbo:rw", Boolean.TRUE);
+    testMatrix.put("user:bilbo:wr", Boolean.TRUE);
+    testMatrix.put("    user:bilbo:wr   ", Boolean.TRUE);
+
+
+    // ACLs makes no judgement on the quality of
+    // user names. it is for the userAuth interface
+    // to determine if a user name is really a name
+    testMatrix.put(" user:*:rw", Boolean.TRUE);
+    testMatrix.put(" user:~!:rw", Boolean.TRUE);
+
+
+    testMatrix.put("", Boolean.FALSE);
+    testMatrix.put(null, Boolean.FALSE);
+    testMatrix.put(" user:bilbo:", Boolean.FALSE);
+    testMatrix.put(" user:bilbo:rx", Boolean.FALSE);
+    testMatrix.put(" user:bilbo:mk", Boolean.FALSE);
+    testMatrix.put(" user::rw", Boolean.FALSE);
+    testMatrix.put("user11:bilbo:rw", Boolean.FALSE);
+    testMatrix.put(" user:::rw", Boolean.FALSE);
+
+    testMatrix.put(" group:hobbit:r", Boolean.TRUE);
+    testMatrix.put(" group:hobbit:w", Boolean.TRUE);
+    testMatrix.put(" group:hobbit:rw", Boolean.TRUE);
+    testMatrix.put(" group:hobbit:wr", Boolean.TRUE);
+    testMatrix.put(" group:*:rw", Boolean.TRUE);
+    testMatrix.put(" group:~!:rw", Boolean.TRUE);
+
+    testMatrix.put(" group:hobbit:", Boolean.FALSE);
+    testMatrix.put(" group:hobbit:rx", Boolean.FALSE);
+    testMatrix.put(" group:hobbit:mk", Boolean.FALSE);
+    testMatrix.put(" group::", Boolean.FALSE);
+    testMatrix.put(" group::rw", Boolean.FALSE);
+    testMatrix.put(" group22:hobbit:", Boolean.FALSE);
+    testMatrix.put(" group:::rw", Boolean.FALSE);
+
+    testMatrix.put("JUNK group:hobbit:r", Boolean.FALSE);
+    testMatrix.put("JUNK group:hobbit:w", Boolean.FALSE);
+    testMatrix.put("JUNK group:hobbit:rw", Boolean.FALSE);
+    testMatrix.put("JUNK group:hobbit:wr", Boolean.FALSE);
+    testMatrix.put("JUNK group:*:rw", Boolean.FALSE);
+    testMatrix.put("JUNK group:~!:rw", Boolean.FALSE);
+
+    testMatrix.put(" world::r", Boolean.TRUE);
+    testMatrix.put(" world::w", Boolean.TRUE);
+    testMatrix.put(" world::rw", Boolean.TRUE);
+    testMatrix.put(" world::wr", Boolean.TRUE);
+
+    testMatrix.put(" world:bilbo:w", Boolean.FALSE);
+    testMatrix.put(" world:bilbo:rw", Boolean.FALSE);
+
+    Set<String> keys = testMatrix.keySet();
+    for (String key : keys) {
+      if (testMatrix.get(key)) {
+        OzoneAcl.parseAcl(key);
+      } else {
+        try {
+          OzoneAcl.parseAcl(key);
+          // should never get here since parseAcl will throw
+          fail("An exception was expected but did not happen.");
+        } catch (IllegalArgumentException e) {
+          // nothing to do
+        }
+      }
+    }
+  }
+
+  @Test
+  public void TestACLValues() {
+    OzoneAcl acl = OzoneAcl.parseAcl("user:bilbo:rw");
+    assertEquals(acl.getName(), "bilbo");
+    assertEquals(acl.getRights(), OzoneAcl.OzoneACLRights.READ_WRITE);
+    assertEquals(acl.getType(), OzoneAcl.OzoneACLType.USER);
+
+    acl = OzoneAcl.parseAcl("user:bilbo:wr");
+    assertEquals(acl.getName(), "bilbo");
+    assertEquals(acl.getRights(), OzoneAcl.OzoneACLRights.READ_WRITE);
+    assertEquals(acl.getType(), OzoneAcl.OzoneACLType.USER);
+
+    acl = OzoneAcl.parseAcl("user:bilbo:r");
+    assertEquals(acl.getName(), "bilbo");
+    assertEquals(acl.getRights(), OzoneAcl.OzoneACLRights.READ);
+    assertEquals(acl.getType(), OzoneAcl.OzoneACLType.USER);
+
+    acl = OzoneAcl.parseAcl("user:bilbo:w");
+    assertEquals(acl.getName(), "bilbo");
+    assertEquals(acl.getRights(), OzoneAcl.OzoneACLRights.WRITE);
+    assertEquals(acl.getType(), OzoneAcl.OzoneACLType.USER);
+
+    acl = OzoneAcl.parseAcl("group:hobbit:wr");
+    assertEquals(acl.getName(), "hobbit");
+    assertEquals(acl.getRights(), OzoneAcl.OzoneACLRights.READ_WRITE);
+    assertEquals(acl.getType(), OzoneAcl.OzoneACLType.GROUP);
+
+    acl = OzoneAcl.parseAcl("world::wr");
+    assertEquals(acl.getName(), "");
+    assertEquals(acl.getRights(), OzoneAcl.OzoneACLRights.READ_WRITE);
+    assertEquals(acl.getType(), OzoneAcl.OzoneACLType.WORLD);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da8bc385/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
deleted file mode 100644
index a783249..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
+++ /dev/null
@@ -1,112 +0,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.
- */
-
-package org.apache.hadoop.ozone;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-
-/**
- * Set of constants used in Ozone implementation.
- */
-@InterfaceAudience.Private
-public final class OzoneConsts {
-  public static final String OZONE_SIMPLE_ROOT_USER = "root";
-  public static final String OZONE_SIMPLE_HDFS_USER = "hdfs";
-
-  /*
-   * BucketName length is used for both buckets and volume lengths
-   */
-  public static final int OZONE_MIN_BUCKET_NAME_LENGTH = 3;
-  public static final int OZONE_MAX_BUCKET_NAME_LENGTH = 63;
-
-  public static final String OZONE_ACL_USER_TYPE = "user";
-  public static final String OZONE_ACL_GROUP_TYPE = "group";
-  public static final String OZONE_ACL_WORLD_TYPE = "world";
-
-  public static final String OZONE_ACL_READ = "r";
-  public static final String OZONE_ACL_WRITE = "w";
-  public static final String OZONE_ACL_READ_WRITE = "rw";
-  public static final String OZONE_ACL_WRITE_READ = "wr";
-
-  public static final String OZONE_DATE_FORMAT =
-      "EEE, dd MMM yyyy HH:mm:ss zzz";
-  public static final String OZONE_TIME_ZONE = "GMT";
-
-  public static final String OZONE_COMPONENT = "component";
-  public static final String OZONE_FUNCTION  = "function";
-  public static final String OZONE_RESOURCE = "resource";
-  public static final String OZONE_USER = "user";
-  public static final String OZONE_REQUEST = "request";
-
-  public static final String CONTAINER_EXTENSION = ".container";
-  public static final String CONTAINER_META = ".meta";
-
-  //  container storage is in the following format.
-  //  Data Volume basePath/containers/<containerName>/metadata and
-  //  Data Volume basePath/containers/<containerName>/data/...
-  public static final String CONTAINER_PREFIX  = "containers";
-  public static final String CONTAINER_META_PATH = "metadata";
-  public static final String CONTAINER_DATA_PATH = "data";
-  public static final String CONTAINER_ROOT_PREFIX = "repository";
-
-  public static final String FILE_HASH = "SHA-256";
-  public final static String CHUNK_OVERWRITE = "OverWriteRequested";
-
-  public static final int CHUNK_SIZE = 1 * 1024 * 1024; // 1 MB
-  public static final long KB = 1024L;
-  public static final long MB = KB * 1024L;
-  public static final long GB = MB * 1024L;
-  public static final long TB = GB * 1024L;
-
-  /**
-   * level DB names used by SCM and data nodes.
-   */
-  public static final String CONTAINER_DB = "container.db";
-  public static final String BLOCK_DB = "block.db";
-  public static final String NODEPOOL_DB = "nodepool.db";
-  public static final String OPEN_CONTAINERS_DB = "openContainers.db";
-  public static final String KSM_DB_NAME = "ksm.db";
-
-  /**
-   * Supports Bucket Versioning.
-   */
-  public enum Versioning {NOT_DEFINED, ENABLED, DISABLED}
-
-  /**
-   * Ozone handler types.
-   */
-  public static final String OZONE_HANDLER_DISTRIBUTED = "distributed";
-  public static final String OZONE_HANDLER_LOCAL = "local";
-
-  /**
-   * KSM LevelDB prefixes.
-   */
-  public static final String KSM_VOLUME_PREFIX = "/";
-  public static final String KSM_BUCKET_PREFIX = KSM_VOLUME_PREFIX;
-  public static final String KSM_KEY_PREFIX = KSM_VOLUME_PREFIX;
-  public static final String KSM_USER_PREFIX = "$";
-
-  /**
-   * Max KSM Quota size of 1024 PB.
-   */
-  public static final long MAX_QUOTA_IN_BYTES = 1024L * 1024 * TB;
-
-  private OzoneConsts() {
-    // Never Constructed
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da8bc385/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/BucketManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/BucketManagerImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/BucketManagerImpl.java
index 638fe42..4c27013 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/BucketManagerImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/BucketManagerImpl.java
@@ -17,24 +17,19 @@
 package org.apache.hadoop.ozone.ksm;
 
 import com.google.common.base.Preconditions;
-import org.apache.hadoop.hdfs.protocol.proto
-    .HdfsProtos.StorageTypeProto;
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.ksm.helpers.KsmBucketArgs;
 import org.apache.hadoop.ksm.helpers.KsmBucketInfo;
 import org.apache.hadoop.ozone.ksm.exceptions.KSMException;
 import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.OzoneAclInfo;
-import org.apache.hadoop.ozone.protocol.proto
     .KeySpaceManagerProtocolProtos.BucketInfo;
-import org.apache.hadoop.ozone.protocolPB.KSMPBHelper;
-import org.apache.hadoop.ozone.web.request.OzoneAcl;
+import org.apache.hadoop.ozone.OzoneAcl;
 import org.iq80.leveldb.DBException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.util.List;
-import java.util.stream.Collectors;
 
 /**
  * KSM bucket manager.
@@ -179,10 +174,8 @@ public class BucketManagerImpl implements BucketManager {
 
       //Check ACLs to update
       if(args.getAddAcls() != null || args.getRemoveAcls() != null) {
-        List<OzoneAcl> acls = getUpdatedAclList(oldBucketInfo.getAcls(),
-            args.getRemoveAcls(), args.getAddAcls());
-        bucketInfoBuilder.setAcls(acls.stream().map(
-            KSMPBHelper::convertOzoneAcl).collect(Collectors.toList()));
+        bucketInfoBuilder.setAcls(getUpdatedAclList(oldBucketInfo.getAcls(),
+            args.getRemoveAcls(), args.getAddAcls()));
         LOG.debug("Updating ACLs for bucket: {} in volume: {}",
             bucketName, volumeName);
       } else {
@@ -190,9 +183,9 @@ public class BucketManagerImpl implements BucketManager {
       }
 
       //Check StorageType to update
-      StorageTypeProto storageTypeProto = args.getStorageType();
-      if(storageTypeProto != null) {
-        bucketInfoBuilder.setStorageType(storageTypeProto);
+      StorageType storageType = args.getStorageType();
+      if(storageType != null) {
+        bucketInfoBuilder.setStorageType(storageType);
         LOG.debug("Updating bucket storage type for bucket: {} in volume: {}",
             bucketName, volumeName);
       } else {
@@ -226,26 +219,20 @@ public class BucketManagerImpl implements BucketManager {
    * Remove is done before Add.
    *
    * @param existingAcls - old ACL list.
-   * @param removeAclInfos - ACLs to be removed.
-   * @param addAclInfos - ACLs to be added.
+   * @param removeAcls - ACLs to be removed.
+   * @param addAcls - ACLs to be added.
    * @return updated ACL list.
    */
-  private List<OzoneAcl> getUpdatedAclList(List<OzoneAclInfo> existingAcls,
-      List<OzoneAclInfo> removeAclInfos, List<OzoneAclInfo> addAclInfos) {
-    List<OzoneAcl> acls = existingAcls.stream().map(
-        KSMPBHelper::convertOzoneAcl).collect(Collectors.toList());
-    if(removeAclInfos != null && !removeAclInfos.isEmpty()) {
-      List<OzoneAcl> removeAcls = removeAclInfos.stream().map(
-          KSMPBHelper::convertOzoneAcl).collect(Collectors.toList());
-      acls.removeAll(removeAcls);
+  private List<OzoneAcl> getUpdatedAclList(List<OzoneAcl> existingAcls,
+      List<OzoneAcl> removeAcls, List<OzoneAcl> addAcls) {
+    if(removeAcls != null && !removeAcls.isEmpty()) {
+      existingAcls.removeAll(removeAcls);
     }
-    if(addAclInfos != null && !addAclInfos.isEmpty()) {
-      List<OzoneAcl> addAcls = addAclInfos.stream().map(
-          KSMPBHelper::convertOzoneAcl).collect(Collectors.toList());
-      addAcls.stream().filter(acl -> !acls.contains(acl)).forEach(
-          acls::add);
+    if(addAcls != null && !addAcls.isEmpty()) {
+      addAcls.stream().filter(acl -> !existingAcls.contains(acl)).forEach(
+          existingAcls::add);
     }
-    return acls;
+    return existingAcls;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da8bc385/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/KSMConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/KSMConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/KSMConfigKeys.java
index 3eb8a56..21f7cad 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/KSMConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/KSMConfigKeys.java
@@ -17,7 +17,7 @@
 
 package org.apache.hadoop.ozone.ksm;
 
-import org.apache.hadoop.ozone.web.request.OzoneAcl;
+import org.apache.hadoop.ozone.OzoneAcl;
 /**
  * KSM Constants.
  */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da8bc385/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/protocolPB/KSMPBHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/protocolPB/KSMPBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/protocolPB/KSMPBHelper.java
deleted file mode 100644
index 8925ff0..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/protocolPB/KSMPBHelper.java
+++ /dev/null
@@ -1,113 +0,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.
- */
-package org.apache.hadoop.ozone.protocolPB;
-
-import org.apache.hadoop.ozone.web.request.OzoneAcl;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.OzoneAclInfo;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.OzoneAclInfo.OzoneAclType;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.OzoneAclInfo.OzoneAclRights;
-
-/**
- * Utilities for converting protobuf classes.
- */
-public final class KSMPBHelper {
-
-  private KSMPBHelper() {
-    /** Hidden constructor */
-  }
-
-  /**
-   * Converts OzoneAcl into protobuf's OzoneAclInfo.
-   * @return OzoneAclInfo
-   */
-  public static OzoneAclInfo convertOzoneAcl(OzoneAcl acl) {
-    OzoneAclInfo.OzoneAclType aclType;
-    switch(acl.getType()) {
-    case USER:
-      aclType = OzoneAclType.USER;
-      break;
-    case GROUP:
-      aclType = OzoneAclType.GROUP;
-      break;
-    case WORLD:
-      aclType = OzoneAclType.WORLD;
-      break;
-    default:
-      throw new IllegalArgumentException("ACL type is not recognized");
-    }
-    OzoneAclInfo.OzoneAclRights aclRights;
-    switch(acl.getRights()) {
-    case READ:
-      aclRights = OzoneAclRights.READ;
-      break;
-    case WRITE:
-      aclRights = OzoneAclRights.WRITE;
-      break;
-    case READ_WRITE:
-      aclRights = OzoneAclRights.READ_WRITE;
-      break;
-    default:
-      throw new IllegalArgumentException("ACL right is not recognized");
-    }
-
-    return OzoneAclInfo.newBuilder().setType(aclType)
-        .setName(acl.getName())
-        .setRights(aclRights)
-        .build();
-  }
-
-  /**
-   * Converts protobuf's OzoneAclInfo into OzoneAcl.
-   * @return OzoneAcl
-   */
-  public static OzoneAcl convertOzoneAcl(OzoneAclInfo aclInfo) {
-    OzoneAcl.OzoneACLType aclType;
-    switch(aclInfo.getType()) {
-    case USER:
-      aclType = OzoneAcl.OzoneACLType.USER;
-      break;
-    case GROUP:
-      aclType = OzoneAcl.OzoneACLType.GROUP;
-      break;
-    case WORLD:
-      aclType = OzoneAcl.OzoneACLType.WORLD;
-      break;
-    default:
-      throw new IllegalArgumentException("ACL type is not recognized");
-    }
-    OzoneAcl.OzoneACLRights aclRights;
-    switch(aclInfo.getRights()) {
-    case READ:
-      aclRights = OzoneAcl.OzoneACLRights.READ;
-      break;
-    case WRITE:
-      aclRights = OzoneAcl.OzoneACLRights.WRITE;
-      break;
-    case READ_WRITE:
-      aclRights = OzoneAcl.OzoneACLRights.READ_WRITE;
-      break;
-    default:
-      throw new IllegalArgumentException("ACL right is not recognized");
-    }
-
-    return new OzoneAcl(aclType, aclInfo.getName(), aclRights);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da8bc385/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneBucket.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneBucket.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneBucket.java
index d070f0a..7f92137 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneBucket.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneBucket.java
@@ -25,7 +25,7 @@ import org.apache.hadoop.ozone.OzoneClientUtils;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.web.exceptions.OzoneException;
 import org.apache.hadoop.ozone.web.headers.Header;
-import org.apache.hadoop.ozone.web.request.OzoneAcl;
+import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.web.response.BucketInfo;
 import org.apache.hadoop.ozone.web.response.KeyInfo;
 import org.apache.hadoop.ozone.web.response.ListKeys;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da8bc385/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/BucketArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/BucketArgs.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/BucketArgs.java
index 7d47948..0d5248d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/BucketArgs.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/BucketArgs.java
@@ -18,7 +18,7 @@
 package org.apache.hadoop.ozone.web.handlers;
 
 import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.ozone.web.request.OzoneAcl;
+import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.OzoneConsts;
 
 import java.util.LinkedList;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da8bc385/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/StorageHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/StorageHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/StorageHandler.java
index 3999f91..52eb349 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/StorageHandler.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/StorageHandler.java
@@ -25,7 +25,7 @@ import org.apache.hadoop.ozone.web.handlers.BucketArgs;
 import org.apache.hadoop.ozone.web.handlers.KeyArgs;
 import org.apache.hadoop.ozone.web.handlers.ListArgs;
 import org.apache.hadoop.ozone.web.handlers.VolumeArgs;
-import org.apache.hadoop.ozone.web.request.OzoneAcl;
+import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.web.response.BucketInfo;
 import org.apache.hadoop.ozone.web.response.ListBuckets;
 import org.apache.hadoop.ozone.web.response.ListKeys;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da8bc385/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/LocalStorageHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/LocalStorageHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/LocalStorageHandler.java
index 36ad9b0..cb8db32 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/LocalStorageHandler.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/LocalStorageHandler.java
@@ -27,7 +27,7 @@ import org.apache.hadoop.ozone.web.handlers.KeyArgs;
 import org.apache.hadoop.ozone.web.handlers.ListArgs;
 import org.apache.hadoop.ozone.web.handlers.VolumeArgs;
 import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
-import org.apache.hadoop.ozone.web.request.OzoneAcl;
+import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.web.request.OzoneQuota;
 import org.apache.hadoop.ozone.web.response.BucketInfo;
 import org.apache.hadoop.ozone.web.response.ListBuckets;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da8bc385/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/OzoneMetadataManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/OzoneMetadataManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/OzoneMetadataManager.java
index fed77f1..3b46373 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/OzoneMetadataManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/OzoneMetadataManager.java
@@ -31,7 +31,7 @@ import org.apache.hadoop.ozone.web.handlers.KeyArgs;
 import org.apache.hadoop.ozone.web.handlers.ListArgs;
 import org.apache.hadoop.ozone.web.handlers.UserArgs;
 import org.apache.hadoop.ozone.web.handlers.VolumeArgs;
-import org.apache.hadoop.ozone.web.request.OzoneAcl;
+import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.web.response.BucketInfo;
 import org.apache.hadoop.ozone.web.response.KeyInfo;
 import org.apache.hadoop.ozone.web.response.ListBuckets;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da8bc385/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/request/OzoneAcl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/request/OzoneAcl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/request/OzoneAcl.java
deleted file mode 100644
index fe7532e..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/request/OzoneAcl.java
+++ /dev/null
@@ -1,208 +0,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.
- */
-
-
-package org.apache.hadoop.ozone.web.request;
-
-import org.apache.hadoop.ozone.OzoneConsts;
-
-import java.util.Objects;
-
-/**
- * OzoneACL classes define bucket ACLs used in OZONE.
- *
- * ACLs in Ozone follow this pattern.
- * • user:name:rw
- * • group:name:rw
- * • world::rw
- */
-public class OzoneAcl {
-  private OzoneACLType type;
-  private String name;
-  private OzoneACLRights rights;
-
-  /**
-   * Constructor for OzoneAcl.
-   */
-  public OzoneAcl() {
-  }
-
-  /**
-   * Constructor for OzoneAcl.
-   *
-   * @param type - Type
-   * @param name - Name of user
-   * @param rights - Rights
-   */
-  public OzoneAcl(OzoneACLType type, String name, OzoneACLRights rights) {
-    this.name = name;
-    this.rights = rights;
-    this.type = type;
-    if (type == OzoneACLType.WORLD && name.length() != 0) {
-      throw new IllegalArgumentException("Unexpected name part in world type");
-    }
-    if (((type == OzoneACLType.USER) || (type == OzoneACLType.GROUP))
-        && (name.length() == 0)) {
-      throw new IllegalArgumentException("User or group name is required");
-    }
-  }
-
-  /**
-   * Parses an ACL string and returns the ACL object.
-   *
-   * @param acl - Acl String , Ex. user:anu:rw
-   *
-   * @return - Ozone ACLs
-   */
-  public static OzoneAcl parseAcl(String acl) throws IllegalArgumentException {
-    if ((acl == null) || acl.isEmpty()) {
-      throw new IllegalArgumentException("ACLs cannot be null or empty");
-    }
-    String[] parts = acl.trim().split(":");
-    if (parts.length < 3) {
-      throw new IllegalArgumentException("ACLs are not in expected format");
-    }
-
-    OzoneACLType aclType = OzoneACLType.valueOf(parts[0].toUpperCase());
-    OzoneACLRights rights = OzoneACLRights.getACLRight(parts[2].toLowerCase());
-
-    // TODO : Support sanitation of these user names by calling into
-    // userAuth Interface.
-    return new OzoneAcl(aclType, parts[1], rights);
-  }
-
-  /**
-   * Returns a hash code value for the object. This method is
-   * supported for the benefit of hash tables.
-   *
-   * @return a hash code value for this object.
-   *
-   * @see Object#equals(Object)
-   * @see System#identityHashCode
-   */
-  @Override
-  public int hashCode() {
-    return Objects.hash(this.getName(), this.getRights().toString(),
-                        this.getType().toString());
-  }
-
-  /**
-   * Returns name.
-   *
-   * @return name
-   */
-  public String getName() {
-    return name;
-  }
-
-  /**
-   * Returns Rights.
-   *
-   * @return - Rights
-   */
-  public OzoneACLRights getRights() {
-    return rights;
-  }
-
-  /**
-   * Returns Type.
-   *
-   * @return type
-   */
-  public OzoneACLType getType() {
-    return type;
-  }
-
-  /**
-   * Indicates whether some other object is "equal to" this one.
-   *
-   * @param obj the reference object with which to compare.
-   *
-   * @return {@code true} if this object is the same as the obj
-   * argument; {@code false} otherwise.
-   */
-  @Override
-  public boolean equals(Object obj) {
-    if (obj == null) {
-      return false;
-    }
-    if (getClass() != obj.getClass()) {
-      return false;
-    }
-    OzoneAcl otherAcl = (OzoneAcl) obj;
-    return otherAcl.getName().equals(this.getName()) &&
-        otherAcl.getRights() == this.getRights() &&
-        otherAcl.getType() == this.getType();
-  }
-
-  /**
-   * ACL types.
-   */
-  public enum OzoneACLType {
-    USER(OzoneConsts.OZONE_ACL_USER_TYPE),
-    GROUP(OzoneConsts.OZONE_ACL_GROUP_TYPE),
-    WORLD(OzoneConsts.OZONE_ACL_WORLD_TYPE);
-
-    /**
-     * String value for this Enum.
-     */
-    private final String value;
-
-    /**
-     * Init OzoneACLtypes enum.
-     *
-     * @param val String type for this enum.
-     */
-    OzoneACLType(String val) {
-      value = val;
-    }
-  }
-
-  /**
-   * ACL rights.
-   */
-  public enum OzoneACLRights {
-    READ, WRITE, READ_WRITE;
-
-    /**
-     * Returns the ACL rights based on passed in String.
-     *
-     * @param type ACL right string
-     *
-     * @return OzoneACLRights
-     */
-    public static OzoneACLRights getACLRight(String type) {
-      if (type == null || type.isEmpty()) {
-        throw new IllegalArgumentException("ACL right cannot be empty");
-      }
-
-      switch (type) {
-      case OzoneConsts.OZONE_ACL_READ:
-        return OzoneACLRights.READ;
-      case OzoneConsts.OZONE_ACL_WRITE:
-        return OzoneACLRights.WRITE;
-      case OzoneConsts.OZONE_ACL_READ_WRITE:
-      case OzoneConsts.OZONE_ACL_WRITE_READ:
-        return OzoneACLRights.READ_WRITE;
-      default:
-        throw new IllegalArgumentException("ACL right is not recognized");
-      }
-
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da8bc385/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/response/BucketInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/response/BucketInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/response/BucketInfo.java
index c6285e1..1e47c16 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/response/BucketInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/response/BucketInfo.java
@@ -23,7 +23,7 @@ import java.util.List;
 
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.web.request.OzoneAcl;
+import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.web.utils.JsonUtils;
 
 import com.fasterxml.jackson.annotation.JsonAutoDetect;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da8bc385/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java
index 2a1691d..cc4a752 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java
@@ -25,7 +25,6 @@ import org.apache.hadoop.hdfs.ozone.protocol.proto
 import org.apache.hadoop.hdfs.ozone.protocol.proto
     .ContainerProtos.KeyData;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset
     .LengthInputStream;
 import org.apache.hadoop.ksm.helpers.KsmBucketArgs;
@@ -40,7 +39,7 @@ import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.OzoneConsts.Versioning;
 import org.apache.hadoop.ozone.protocolPB.KSMPBHelper;
 import org.apache.hadoop.ozone.ksm.KSMConfigKeys;
-import org.apache.hadoop.ozone.web.request.OzoneAcl;
+import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.web.request.OzoneQuota;
 import org.apache.hadoop.scm.container.common.helpers.Pipeline;
 import org.apache.hadoop.scm.ScmConfigKeys;
@@ -78,7 +77,6 @@ import java.util.Locale;
 import java.util.HashSet;
 import java.util.Arrays;
 import java.util.List;
-import java.util.stream.Collectors;
 
 /**
  * A {@link StorageHandler} implementation that distributes object storage
@@ -205,12 +203,10 @@ public final class DistributedStorageHandler implements StorageHandler {
     builder.setVolumeName(args.getVolumeName())
         .setBucketName(args.getBucketName());
     if(args.getAddAcls() != null) {
-      builder.setAcls(args.getAddAcls().stream().map(
-          KSMPBHelper::convertOzoneAcl).collect(Collectors.toList()));
+      builder.setAcls(args.getAddAcls());
     }
     if(args.getStorageType() != null) {
-      builder.setStorageType(PBHelperClient.convertStorageType(
-          args.getStorageType()));
+      builder.setStorageType(args.getStorageType());
     }
     if(args.getVersioning() != null) {
       builder.setIsVersionEnabled(getBucketVersioningProtobuf(
@@ -250,12 +246,10 @@ public final class DistributedStorageHandler implements StorageHandler {
       builder.setVolumeName(args.getVolumeName())
           .setBucketName(args.getBucketName());
       if(removeAcls != null && !removeAcls.isEmpty()) {
-        builder.setRemoveAcls(args.getRemoveAcls().stream().map(
-            KSMPBHelper::convertOzoneAcl).collect(Collectors.toList()));
+        builder.setRemoveAcls(args.getRemoveAcls());
       }
       if(addAcls != null && !addAcls.isEmpty()) {
-        builder.setAddAcls(args.getAddAcls().stream().map(
-            KSMPBHelper::convertOzoneAcl).collect(Collectors.toList()));
+        builder.setAddAcls(args.getAddAcls());
       }
       keySpaceManagerClient.setBucketProperty(builder.build());
     }
@@ -278,8 +272,7 @@ public final class DistributedStorageHandler implements StorageHandler {
     KsmBucketArgs.Builder builder = KsmBucketArgs.newBuilder();
     builder.setVolumeName(args.getVolumeName())
         .setBucketName(args.getBucketName())
-        .setStorageType(PBHelperClient.convertStorageType(
-            args.getStorageType()));
+        .setStorageType(args.getStorageType());
     keySpaceManagerClient.setBucketProperty(builder.build());
   }
 
@@ -317,10 +310,8 @@ public final class DistributedStorageHandler implements StorageHandler {
     } else {
       bucketInfo.setVersioning(Versioning.DISABLED);
     }
-    bucketInfo.setStorageType(PBHelperClient.convertStorageType(
-        ksmBucketInfo.getStorageType()));
-    bucketInfo.setAcls(ksmBucketInfo.getAcls().stream().map(
-        KSMPBHelper::convertOzoneAcl).collect(Collectors.toList()));
+    bucketInfo.setStorageType(ksmBucketInfo.getStorageType());
+    bucketInfo.setAcls(ksmBucketInfo.getAcls());
     return bucketInfo;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da8bc385/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/ksm/TestBucketManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/ksm/TestBucketManagerImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/ksm/TestBucketManagerImpl.java
index 4fbb9c6..a40a6d0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/ksm/TestBucketManagerImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/ksm/TestBucketManagerImpl.java
@@ -16,18 +16,15 @@
  */
 package org.apache.hadoop.ozone.ksm;
 
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.ksm.helpers.KsmBucketArgs;
 import org.apache.hadoop.ksm.helpers.KsmBucketInfo;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.ksm.exceptions.KSMException;
 import org.apache.hadoop.ozone.ksm.exceptions
     .KSMException.ResultCodes;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.OzoneAclInfo;
-import org.apache.hadoop.ozone.protocolPB.KSMPBHelper;
-import org.apache.hadoop.ozone.web.request.OzoneAcl;
+import org.apache.hadoop.ozone.OzoneAcl;
 import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
@@ -203,7 +200,7 @@ public class TestBucketManagerImpl {
     KsmBucketInfo bucketInfo = KsmBucketInfo.newBuilder()
         .setVolumeName("sampleVol")
         .setBucketName("bucketOne")
-        .setStorageType(HdfsProtos.StorageTypeProto.DISK)
+        .setStorageType(StorageType.DISK)
         .setIsVersionEnabled(false)
         .build();
     bucketManager.createBucket(bucketInfo);
@@ -211,7 +208,7 @@ public class TestBucketManagerImpl {
         "sampleVol", "bucketOne");
     Assert.assertEquals("sampleVol", result.getVolumeName());
     Assert.assertEquals("bucketOne", result.getBucketName());
-    Assert.assertEquals(HdfsProtos.StorageTypeProto.DISK,
+    Assert.assertEquals(StorageType.DISK,
         result.getStorageType());
     Assert.assertEquals(false, result.getIsVersionEnabled());
   }
@@ -219,16 +216,16 @@ public class TestBucketManagerImpl {
   @Test
   public void testSetBucketPropertyAddACL() throws IOException {
     MetadataManager metaMgr = getMetadataManagerMock("sampleVol");
-    List<OzoneAclInfo> acls = new LinkedList<>();
+    List<OzoneAcl> acls = new LinkedList<>();
     OzoneAcl ozoneAcl = new OzoneAcl(OzoneAcl.OzoneACLType.USER,
         "root", OzoneAcl.OzoneACLRights.READ);
-    acls.add(KSMPBHelper.convertOzoneAcl(ozoneAcl));
+    acls.add(ozoneAcl);
     BucketManager bucketManager = new BucketManagerImpl(metaMgr);
     KsmBucketInfo bucketInfo = KsmBucketInfo.newBuilder()
         .setVolumeName("sampleVol")
         .setBucketName("bucketOne")
         .setAcls(acls)
-        .setStorageType(HdfsProtos.StorageTypeProto.DISK)
+        .setStorageType(StorageType.DISK)
         .setIsVersionEnabled(false)
         .build();
     bucketManager.createBucket(bucketInfo);
@@ -237,10 +234,10 @@ public class TestBucketManagerImpl {
     Assert.assertEquals("sampleVol", result.getVolumeName());
     Assert.assertEquals("bucketOne", result.getBucketName());
     Assert.assertEquals(1, result.getAcls().size());
-    List<OzoneAclInfo> addAcls = new LinkedList<>();
+    List<OzoneAcl> addAcls = new LinkedList<>();
     OzoneAcl newAcl = new OzoneAcl(OzoneAcl.OzoneACLType.USER,
         "ozone", OzoneAcl.OzoneACLRights.READ);
-    addAcls.add(KSMPBHelper.convertOzoneAcl(newAcl));
+    addAcls.add(newAcl);
     KsmBucketArgs bucketArgs = KsmBucketArgs.newBuilder()
         .setVolumeName("sampleVol")
         .setBucketName("bucketOne")
@@ -250,34 +247,33 @@ public class TestBucketManagerImpl {
     KsmBucketInfo updatedResult = bucketManager.getBucketInfo(
         "sampleVol", "bucketOne");
     Assert.assertEquals(2, updatedResult.getAcls().size());
-    Assert.assertTrue(updatedResult.getAcls().contains(
-        KSMPBHelper.convertOzoneAcl(newAcl)));
+    Assert.assertTrue(updatedResult.getAcls().contains(newAcl));
   }
 
   @Test
   public void testSetBucketPropertyRemoveACL() throws IOException {
     MetadataManager metaMgr = getMetadataManagerMock("sampleVol");
-    List<OzoneAclInfo> acls = new LinkedList<>();
+    List<OzoneAcl> acls = new LinkedList<>();
     OzoneAcl aclOne = new OzoneAcl(OzoneAcl.OzoneACLType.USER,
         "root", OzoneAcl.OzoneACLRights.READ);
     OzoneAcl aclTwo = new OzoneAcl(OzoneAcl.OzoneACLType.USER,
         "ozone", OzoneAcl.OzoneACLRights.READ);
-    acls.add(KSMPBHelper.convertOzoneAcl(aclOne));
-    acls.add(KSMPBHelper.convertOzoneAcl(aclTwo));
+    acls.add(aclOne);
+    acls.add(aclTwo);
     BucketManager bucketManager = new BucketManagerImpl(metaMgr);
     KsmBucketInfo bucketInfo = KsmBucketInfo.newBuilder()
         .setVolumeName("sampleVol")
         .setBucketName("bucketOne")
         .setAcls(acls)
-        .setStorageType(HdfsProtos.StorageTypeProto.DISK)
+        .setStorageType(StorageType.DISK)
         .setIsVersionEnabled(false)
         .build();
     bucketManager.createBucket(bucketInfo);
     KsmBucketInfo result = bucketManager.getBucketInfo(
         "sampleVol", "bucketOne");
     Assert.assertEquals(2, result.getAcls().size());
-    List<OzoneAclInfo> removeAcls = new LinkedList<>();
-    removeAcls.add(KSMPBHelper.convertOzoneAcl(aclTwo));
+    List<OzoneAcl> removeAcls = new LinkedList<>();
+    removeAcls.add(aclTwo);
     KsmBucketArgs bucketArgs = KsmBucketArgs.newBuilder()
         .setVolumeName("sampleVol")
         .setBucketName("bucketOne")
@@ -287,8 +283,7 @@ public class TestBucketManagerImpl {
     KsmBucketInfo updatedResult = bucketManager.getBucketInfo(
         "sampleVol", "bucketOne");
     Assert.assertEquals(1, updatedResult.getAcls().size());
-    Assert.assertFalse(updatedResult.getAcls().contains(
-        KSMPBHelper.convertOzoneAcl(aclTwo)));
+    Assert.assertFalse(updatedResult.getAcls().contains(aclTwo));
   }
 
   @Test
@@ -298,22 +293,22 @@ public class TestBucketManagerImpl {
     KsmBucketInfo bucketInfo = KsmBucketInfo.newBuilder()
         .setVolumeName("sampleVol")
         .setBucketName("bucketOne")
-        .setStorageType(HdfsProtos.StorageTypeProto.DISK)
+        .setStorageType(StorageType.DISK)
         .build();
     bucketManager.createBucket(bucketInfo);
     KsmBucketInfo result = bucketManager.getBucketInfo(
         "sampleVol", "bucketOne");
-    Assert.assertEquals(HdfsProtos.StorageTypeProto.DISK,
+    Assert.assertEquals(StorageType.DISK,
         result.getStorageType());
     KsmBucketArgs bucketArgs = KsmBucketArgs.newBuilder()
         .setVolumeName("sampleVol")
         .setBucketName("bucketOne")
-        .setStorageType(HdfsProtos.StorageTypeProto.SSD)
+        .setStorageType(StorageType.SSD)
         .build();
     bucketManager.setBucketProperty(bucketArgs);
     KsmBucketInfo updatedResult = bucketManager.getBucketInfo(
         "sampleVol", "bucketOne");
-    Assert.assertEquals(HdfsProtos.StorageTypeProto.SSD,
+    Assert.assertEquals(StorageType.SSD,
         updatedResult.getStorageType());
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da8bc385/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/ksm/TestKeySpaceManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/ksm/TestKeySpaceManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/ksm/TestKeySpaceManager.java
index 64e5d83..a84a8f9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/ksm/TestKeySpaceManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/ksm/TestKeySpaceManager.java
@@ -31,7 +31,7 @@ import org.apache.hadoop.ozone.web.handlers.KeyArgs;
 import org.apache.hadoop.ozone.web.handlers.UserArgs;
 import org.apache.hadoop.ozone.web.handlers.VolumeArgs;
 import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
-import org.apache.hadoop.ozone.web.request.OzoneAcl;
+import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.web.request.OzoneQuota;
 import org.apache.hadoop.ozone.web.response.BucketInfo;
 import org.apache.hadoop.ozone.web.response.VolumeInfo;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da8bc385/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestBucketInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestBucketInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestBucketInfo.java
index 7599846..c7b3b16 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestBucketInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestBucketInfo.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.ozone.web;
 
 
 import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.ozone.web.request.OzoneAcl;
+import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.web.response.BucketInfo;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da8bc385/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneAcls.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneAcls.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneAcls.java
deleted file mode 100644
index 87e0a28..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneAcls.java
+++ /dev/null
@@ -1,139 +0,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.
- */
-
-package org.apache.hadoop.ozone.web;
-
-import org.apache.hadoop.ozone.web.request.OzoneAcl;
-import org.junit.Test;
-
-import java.util.HashMap;
-import java.util.Set;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-public class TestOzoneAcls {
-
-  @Test
-  public void TestACLParse() {
-    HashMap<String, Boolean> testMatrix;
-    testMatrix = new HashMap<>();
-
-    testMatrix.put("user:bilbo:r", Boolean.TRUE);
-    testMatrix.put("user:bilbo:w", Boolean.TRUE);
-    testMatrix.put("user:bilbo:rw", Boolean.TRUE);
-    testMatrix.put("user:bilbo:wr", Boolean.TRUE);
-    testMatrix.put("    user:bilbo:wr   ", Boolean.TRUE);
-
-
-    // ACLs makes no judgement on the quality of
-    // user names. it is for the userAuth interface
-    // to determine if a user name is really a name
-    testMatrix.put(" user:*:rw", Boolean.TRUE);
-    testMatrix.put(" user:~!:rw", Boolean.TRUE);
-
-
-    testMatrix.put("", Boolean.FALSE);
-    testMatrix.put(null, Boolean.FALSE);
-    testMatrix.put(" user:bilbo:", Boolean.FALSE);
-    testMatrix.put(" user:bilbo:rx", Boolean.FALSE);
-    testMatrix.put(" user:bilbo:mk", Boolean.FALSE);
-    testMatrix.put(" user::rw", Boolean.FALSE);
-    testMatrix.put("user11:bilbo:rw", Boolean.FALSE);
-    testMatrix.put(" user:::rw", Boolean.FALSE);
-
-    testMatrix.put(" group:hobbit:r", Boolean.TRUE);
-    testMatrix.put(" group:hobbit:w", Boolean.TRUE);
-    testMatrix.put(" group:hobbit:rw", Boolean.TRUE);
-    testMatrix.put(" group:hobbit:wr", Boolean.TRUE);
-    testMatrix.put(" group:*:rw", Boolean.TRUE);
-    testMatrix.put(" group:~!:rw", Boolean.TRUE);
-
-    testMatrix.put(" group:hobbit:", Boolean.FALSE);
-    testMatrix.put(" group:hobbit:rx", Boolean.FALSE);
-    testMatrix.put(" group:hobbit:mk", Boolean.FALSE);
-    testMatrix.put(" group::", Boolean.FALSE);
-    testMatrix.put(" group::rw", Boolean.FALSE);
-    testMatrix.put(" group22:hobbit:", Boolean.FALSE);
-    testMatrix.put(" group:::rw", Boolean.FALSE);
-
-    testMatrix.put("JUNK group:hobbit:r", Boolean.FALSE);
-    testMatrix.put("JUNK group:hobbit:w", Boolean.FALSE);
-    testMatrix.put("JUNK group:hobbit:rw", Boolean.FALSE);
-    testMatrix.put("JUNK group:hobbit:wr", Boolean.FALSE);
-    testMatrix.put("JUNK group:*:rw", Boolean.FALSE);
-    testMatrix.put("JUNK group:~!:rw", Boolean.FALSE);
-
-    testMatrix.put(" world::r", Boolean.TRUE);
-    testMatrix.put(" world::w", Boolean.TRUE);
-    testMatrix.put(" world::rw", Boolean.TRUE);
-    testMatrix.put(" world::wr", Boolean.TRUE);
-
-    testMatrix.put(" world:bilbo:w", Boolean.FALSE);
-    testMatrix.put(" world:bilbo:rw", Boolean.FALSE);
-
-    Set<String> keys = testMatrix.keySet();
-    for (String key : keys) {
-      if (testMatrix.get(key)) {
-        OzoneAcl.parseAcl(key);
-      } else {
-        try {
-          OzoneAcl.parseAcl(key);
-          // should never get here since parseAcl will throw
-          fail("An exception was expected but did not happen.");
-        } catch (IllegalArgumentException e) {
-          // nothing to do
-        }
-      }
-    }
-  }
-
-  @Test
-  public void TestACLValues() {
-    OzoneAcl acl = OzoneAcl.parseAcl("user:bilbo:rw");
-    assertEquals(acl.getName(), "bilbo");
-    assertEquals(acl.getRights(), OzoneAcl.OzoneACLRights.READ_WRITE);
-    assertEquals(acl.getType(), OzoneAcl.OzoneACLType.USER);
-
-    acl = OzoneAcl.parseAcl("user:bilbo:wr");
-    assertEquals(acl.getName(), "bilbo");
-    assertEquals(acl.getRights(), OzoneAcl.OzoneACLRights.READ_WRITE);
-    assertEquals(acl.getType(), OzoneAcl.OzoneACLType.USER);
-
-    acl = OzoneAcl.parseAcl("user:bilbo:r");
-    assertEquals(acl.getName(), "bilbo");
-    assertEquals(acl.getRights(), OzoneAcl.OzoneACLRights.READ);
-    assertEquals(acl.getType(), OzoneAcl.OzoneACLType.USER);
-
-    acl = OzoneAcl.parseAcl("user:bilbo:w");
-    assertEquals(acl.getName(), "bilbo");
-    assertEquals(acl.getRights(), OzoneAcl.OzoneACLRights.WRITE);
-    assertEquals(acl.getType(), OzoneAcl.OzoneACLType.USER);
-
-    acl = OzoneAcl.parseAcl("group:hobbit:wr");
-    assertEquals(acl.getName(), "hobbit");
-    assertEquals(acl.getRights(), OzoneAcl.OzoneACLRights.READ_WRITE);
-    assertEquals(acl.getType(), OzoneAcl.OzoneACLType.GROUP);
-
-    acl = OzoneAcl.parseAcl("world::wr");
-    assertEquals(acl.getName(), "");
-    assertEquals(acl.getRights(), OzoneAcl.OzoneACLRights.READ_WRITE);
-    assertEquals(acl.getType(), OzoneAcl.OzoneACLType.WORLD);
-  }
-
-}


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