瀏覽代碼

HDDS-880. Create api for ACL handling in Ozone. (Contributed by Ajay Kumar)

Ajay Kumar 6 年之前
父節點
當前提交
8d882c3786

+ 13 - 0
hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java

@@ -47,11 +47,20 @@ public final class OzoneConsts {
   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_IP_TYPE = "ip";
 
   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_ACL_DELETE = "d";
+  public static final String OZONE_ACL_LIST = "l";
+  public static final String OZONE_ACL_ALL = "a";
+  public static final String OZONE_ACL_NONE = "n";
+  public static final String OZONE_ACL_CREATE = "c";
+  public static final String OZONE_ACL_READ_ACL = "x";
+  public static final String OZONE_ACL_WRITE_ACL = "y";
+
 
   public static final String OZONE_DATE_FORMAT =
       "EEE, dd MMM yyyy HH:mm:ss zzz";
@@ -196,6 +205,10 @@ public final class OzoneConsts {
   public static final String ORIGIN_PIPELINE_ID = "originPipelineId";
   public static final String ORIGIN_NODE_ID = "originNodeId";
 
+  // Supported store types.
+  public static final String OZONE = "ozone";
+  public static final String S3 = "s3";
+
   // For OM Audit usage
   public static final String VOLUME = "volume";
   public static final String BUCKET = "bucket";

+ 153 - 0
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/security/acl/IAccessAuthorizer.java

@@ -0,0 +1,153 @@
+/**
+ * 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.security.acl;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.ozone.OzoneConsts;
+
+/**
+ * Public API for Ozone ACLs. Security providers providing support for Ozone
+ * ACLs should implement this.
+ */
+@InterfaceAudience.LimitedPrivate({"HDFS", "Yarn", "Ranger", "Hive", "HBase"})
+@InterfaceStability.Evolving
+public interface IAccessAuthorizer {
+
+  /**
+   * Check access for given ozoneObject.
+   *
+   * @param ozoneObject object for which access needs to be checked.
+   * @param context Context object encapsulating all user related information.
+   * @throws OzoneAclException
+   * @return true if user has access else false.
+   */
+  boolean checkAccess(IOzoneObj ozoneObject, RequestContext context)
+      throws OzoneAclException;
+
+  /**
+   * ACL rights.
+   */
+  enum ACLType {
+    READ,
+    WRITE,
+    CREATE,
+    LIST,
+    DELETE,
+    READ_ACL,
+    WRITE_ACL,
+    ALL,
+    NONE;
+
+    /**
+     * Returns the ACL rights based on passed in String.
+     *
+     * @param type ACL right string
+     * @return ACLType
+     */
+    public static ACLType getACLRight(String type) {
+      if (type == null || type.isEmpty()) {
+        throw new IllegalArgumentException("ACL right cannot be empty");
+      }
+
+      switch (type) {
+      case OzoneConsts.OZONE_ACL_READ:
+        return ACLType.READ;
+      case OzoneConsts.OZONE_ACL_WRITE:
+        return ACLType.WRITE;
+      case OzoneConsts.OZONE_ACL_CREATE:
+        return ACLType.CREATE;
+      case OzoneConsts.OZONE_ACL_DELETE:
+        return ACLType.DELETE;
+      case OzoneConsts.OZONE_ACL_LIST:
+        return ACLType.LIST;
+      case OzoneConsts.OZONE_ACL_READ_ACL:
+        return ACLType.READ_ACL;
+      case OzoneConsts.OZONE_ACL_WRITE_ACL:
+        return ACLType.WRITE_ACL;
+      case OzoneConsts.OZONE_ACL_ALL:
+        return ACLType.ALL;
+      case OzoneConsts.OZONE_ACL_NONE:
+        return ACLType.NONE;
+      default:
+        throw new IllegalArgumentException("ACL right is not recognized");
+      }
+
+    }
+
+    /**
+     * Returns String representation of ACL rights.
+     *
+     * @param acl ACLType
+     * @return String representation of acl
+     */
+    public static String getACLRightsString(ACLType acl) {
+      switch (acl) {
+      case READ:
+        return OzoneConsts.OZONE_ACL_READ;
+      case WRITE:
+        return OzoneConsts.OZONE_ACL_WRITE;
+      case CREATE:
+        return OzoneConsts.OZONE_ACL_CREATE;
+      case DELETE:
+        return OzoneConsts.OZONE_ACL_DELETE;
+      case LIST:
+        return OzoneConsts.OZONE_ACL_LIST;
+      case READ_ACL:
+        return OzoneConsts.OZONE_ACL_READ_ACL;
+      case WRITE_ACL:
+        return OzoneConsts.OZONE_ACL_WRITE_ACL;
+      case ALL:
+        return OzoneConsts.OZONE_ACL_ALL;
+      case NONE:
+        return OzoneConsts.OZONE_ACL_NONE;
+      default:
+        throw new IllegalArgumentException("ACL right is not recognized");
+      }
+    }
+
+  }
+
+  /**
+   * Type of acl identity.
+   */
+  enum ACLIdentityType {
+    USER(OzoneConsts.OZONE_ACL_USER_TYPE),
+    GROUP(OzoneConsts.OZONE_ACL_GROUP_TYPE),
+    CLIENT_IP(OzoneConsts.OZONE_ACL_IP_TYPE),
+    WORLD(OzoneConsts.OZONE_ACL_WORLD_TYPE);
+
+    @Override
+    public String toString() {
+      return value;
+    }
+    /**
+     * String value for this Enum.
+     */
+    private final String value;
+
+    /**
+     * Init OzoneACLtypes enum.
+     *
+     * @param val String type for this enum.
+     */
+    ACLIdentityType(String val) {
+      value = val;
+    }
+  }
+
+}

+ 24 - 0
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/security/acl/IOzoneObj.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.security.acl;
+
+/**
+ * Marker interface for objects supported by Ozone.
+ * */
+public interface IOzoneObj {
+}

+ 55 - 0
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/security/acl/OzoneAclException.java

@@ -0,0 +1,55 @@
+/**
+ * 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.security.acl;
+
+/**
+ * Timeout exception thrown by Ozone. Ex: When checking ACLs for an Object if
+ * security manager is not able to process the request in configured time than
+ * {@link OzoneAclException} should be thrown.
+ */
+public class OzoneAclException extends Exception {
+
+  private ErrorCode errorCode;
+
+  /**
+   * Constructs a new exception with {@code null} as its detail message. The
+   * cause is not initialized, and may subsequently be initialized by a call to
+   * {@link #initCause}.
+   */
+  public OzoneAclException() {
+    super("");
+  }
+
+  /**
+   * Constructs a new exception with {@code null} as its detail message. The
+   * cause is not initialized, and may subsequently be initialized by a call to
+   * {@link #initCause}.
+   */
+  public OzoneAclException(String errorMsg, ErrorCode code, Throwable ex) {
+    super(errorMsg, ex);
+    this.errorCode = code;
+  }
+
+  enum ErrorCode {
+    TIMEOUT,
+    OTHER
+  }
+
+  public ErrorCode getErrorCode() {
+    return errorCode;
+  }
+}

+ 105 - 0
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/security/acl/OzoneObj.java

@@ -0,0 +1,105 @@
+/**
+ * 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.security.acl;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.ozone.OzoneConsts;
+
+/**
+ * Class representing an unique ozone object.
+ * */
+public abstract class OzoneObj implements IOzoneObj {
+
+  private final ResourceType resType;
+
+  private final StoreType storeType;
+  // Full path of resource.
+  private final String path;
+
+  OzoneObj(ResourceType resType, StoreType storeType, String path) {
+    Preconditions.checkNotNull(path);
+    Preconditions.checkNotNull(resType);
+    Preconditions.checkNotNull(storeType);
+    this.resType = resType;
+    this.storeType = storeType;
+    this.path = path;
+  }
+
+  public ResourceType getResourceType() {
+    return resType;
+  }
+
+  public StoreType getStoreType() {
+    return storeType;
+  }
+
+  public abstract String getVolumeName();
+
+  public abstract String getBucketName();
+
+  public abstract String getKeyName();
+
+  public String getPath() {
+    return path;
+  }
+
+  /**
+   * Ozone Objects supported for ACL.
+   */
+  enum ResourceType {
+    VOLUME(OzoneConsts.VOLUME),
+    BUCKET(OzoneConsts.BUCKET),
+    KEY(OzoneConsts.KEY);
+
+    /**
+     * String value for this Enum.
+     */
+    private final String value;
+
+    @Override
+    public String toString() {
+      return value;
+    }
+
+    ResourceType(String resType) {
+      value = resType;
+    }
+  }
+
+  /**
+   * Ozone Objects supported for ACL.
+   */
+  enum StoreType {
+    OZONE(OzoneConsts.OZONE),
+    S3(OzoneConsts.S3);
+
+    /**
+     * String value for this Enum.
+     */
+    private final String value;
+
+    @Override
+    public String toString() {
+      return value;
+    }
+
+    StoreType(String objType) {
+      value = objType;
+    }
+  }
+}

+ 118 - 0
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/security/acl/RequestContext.java

@@ -0,0 +1,118 @@
+/**
+ * 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.security.acl;
+
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLIdentityType;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLType;
+
+import java.net.InetAddress;
+
+/**
+ * This class encapsulates information required for Ozone ACLs.
+ * */
+public class RequestContext {
+  private final String host;
+  private final InetAddress ip;
+  private final UserGroupInformation clientUgi;
+  private final String serviceId;
+  private final ACLIdentityType aclType;
+  private final ACLType aclRights;
+
+  RequestContext(String host, InetAddress ip,
+      UserGroupInformation clientUgi, String serviceId,
+      ACLIdentityType aclType, ACLType aclRights) {
+    this.host = host;
+    this.ip = ip;
+    this.clientUgi = clientUgi;
+    this.serviceId = serviceId;
+    this.aclType = aclType;
+    this.aclRights = aclRights;
+  }
+
+  /**
+   * Builder class for @{@link RequestContext}.
+   */
+  public static class Builder {
+    private String host;
+    private InetAddress ip;
+    private UserGroupInformation clientUgi;
+    private String serviceId;
+    private IAccessAuthorizer.ACLIdentityType aclType;
+    private IAccessAuthorizer.ACLType aclRights;
+
+    public Builder setHost(String bHost) {
+      this.host = bHost;
+      return this;
+    }
+
+    public Builder setIp(InetAddress cIp) {
+      this.ip = cIp;
+      return this;
+    }
+
+    public Builder setClientUgi(UserGroupInformation cUgi) {
+      this.clientUgi = cUgi;
+      return this;
+    }
+
+    public Builder setServiceId(String sId) {
+      this.serviceId = sId;
+      return this;
+    }
+
+    public Builder setAclType(ACLIdentityType acl) {
+      this.aclType = acl;
+      return this;
+    }
+
+    public Builder setAclRights(ACLType aclRight) {
+      this.aclRights = aclRight;
+      return this;
+    }
+
+    public RequestContext build() {
+      return new RequestContext(host, ip, clientUgi, serviceId, aclType,
+          aclRights);
+    }
+  }
+
+  public String getHost() {
+    return host;
+  }
+
+  public InetAddress getIp() {
+    return ip;
+  }
+
+  public UserGroupInformation getClientUgi() {
+    return clientUgi;
+  }
+
+  public String getServiceId() {
+    return serviceId;
+  }
+
+  public ACLIdentityType getAclType() {
+    return aclType;
+  }
+
+  public ACLType getAclRights() {
+    return aclRights;
+  }
+
+}

+ 22 - 0
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/security/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
+ *
+ *     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.security.acl;
+
+/**
+ * Classes related to ozone Ozone ACL.
+ */