Browse Source

HDFS-5685. Implement ACL as a INode feature. Contributed by Haohui Mai.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-4685@1552465 13f79535-47bb-0310-9956-ffa450edef68
Chris Nauroth 11 years ago
parent
commit
c552d5d72e

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-4685.txt

@@ -14,6 +14,8 @@ HDFS-4685 (Unreleased)
 
     HDFS-5596. Implement RPC stubs. (Haohui Mai via cnauroth)
 
+    HDFS-5685. Implement ACL as a INode feature. (Haohui Mai via cnauroth)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 40 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclFeature.java

@@ -0,0 +1,40 @@
+/**
+ * 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.hdfs.server.namenode;
+
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.permission.AclEntry;
+
+/**
+ * Feature that represents the ACLs of the inode.
+ */
+@InterfaceAudience.Private
+public class AclFeature implements INode.Feature {
+  private List<AclEntry> entries;
+
+  public List<AclEntry> getEntries() {
+    return entries;
+  }
+
+  public void setEntries(List<AclEntry> entries) {
+    this.entries = entries;
+  }
+}

+ 62 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -39,6 +39,8 @@ import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathIsNotDirectoryException;
 import org.apache.hadoop.fs.UnresolvedLinkException;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
@@ -75,6 +77,7 @@ import org.apache.hadoop.hdfs.util.ReadOnlyList;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
 
 /*************************************************
  * FSDirectory stores the filesystem directory state.
@@ -2633,7 +2636,65 @@ public class FSDirectory implements Closeable {
         target);
     return addINode(path, symlink) ? symlink : null;
   }
-  
+
+  void removeAcl(String src) throws IOException {
+    writeLock();
+    try {
+      final INodeWithAdditionalFields node = resolveINodeWithAdditionalField(src);
+      AclFeature f = node.getAclFeature();
+      if (f != null)
+        node.removeAclFeature();
+    } finally {
+      writeUnlock();
+    }
+  }
+
+  void setAcl(String src, Iterable<AclEntry> aclSpec) throws IOException {
+    writeLock();
+    try {
+      final INodeWithAdditionalFields node = resolveINodeWithAdditionalField(src);
+      AclFeature f = node.getAclFeature();
+      if (f == null) {
+        f = new AclFeature();
+        node.addAclFeature(f);
+      }
+      f.setEntries(Lists.newArrayList(aclSpec));
+    } finally {
+      writeUnlock();
+    }
+  }
+
+  AclStatus getAclStatus(String src) throws IOException {
+    readLock();
+    try {
+      final INodeWithAdditionalFields node = resolveINodeWithAdditionalField(src);
+      AclFeature f = node.getAclFeature();
+
+      AclStatus.Builder builder = new AclStatus.Builder()
+          .owner(node.getUserName()).group(node.getGroupName())
+          .stickyBit(node.getFsPermission().getStickyBit());
+      if (f != null) {
+        builder.addEntries(f.getEntries());
+      }
+      return builder.build();
+    } finally {
+      readUnlock();
+    }
+  }
+
+  private INodeWithAdditionalFields resolveINodeWithAdditionalField(String src)
+      throws UnresolvedLinkException, SnapshotAccessControlException,
+      FileNotFoundException {
+    String srcs = normalizePath(src);
+    final INodesInPath iip = rootDir.getINodesInPath4Write(srcs, true);
+    INode inode = iip.getLastINode();
+    if (!(inode instanceof INodeWithAdditionalFields))
+      throw new FileNotFoundException("cannot find " + src);
+
+    final INodeWithAdditionalFields node = (INodeWithAdditionalFields) inode;
+    return node;
+  }
+
   /**
    * Caches frequently used file names to reuse file name objects and
    * reduce heap size.

+ 29 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -7314,16 +7314,39 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     throw new UnsupportedOperationException("Unimplemented");
   }
 
-  void removeAcl(String src) {
-    throw new UnsupportedOperationException("Unimplemented");
+  void removeAcl(String src) throws IOException {
+    checkOperation(OperationCategory.WRITE);
+    writeLock();
+    try {
+      checkOperation(OperationCategory.WRITE);
+      checkNameNodeSafeMode("Cannot remove acl on " + src);
+      dir.removeAcl(src);
+    } finally {
+      writeUnlock();
+    }
   }
 
-  void setAcl(String src, Iterable<AclEntry> aclSpec) {
-    throw new UnsupportedOperationException("Unimplemented");
+  void setAcl(String src, Iterable<AclEntry> aclSpec) throws IOException {
+    checkOperation(OperationCategory.WRITE);
+    writeLock();
+    try {
+      checkOperation(OperationCategory.WRITE);
+      checkNameNodeSafeMode("Cannot set acl on " + src);
+      dir.setAcl(src, aclSpec);
+    } finally {
+      writeUnlock();
+    }
   }
 
-  AclStatus getAclStatus(String src) {
-    throw new UnsupportedOperationException("Unimplemented");
+  AclStatus getAclStatus(String src) throws IOException {
+    checkOperation(OperationCategory.READ);
+    readLock();
+    try {
+      checkOperation(OperationCategory.READ);
+      return dir.getAclStatus(src);
+    } finally {
+      readUnlock();
+    }
   }
 
   /**

+ 2 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -148,12 +148,7 @@ public class INodeFile extends INodeWithAdditionalFields
    * otherwise, return null.
    */
   public final FileUnderConstructionFeature getFileUnderConstructionFeature() {
-    for (Feature f : features) {
-      if (f instanceof FileUnderConstructionFeature) {
-        return (FileUnderConstructionFeature) f;
-      }
-    }
-    return null;
+    return getFeature(FileUnderConstructionFeature.class);
   }
 
   /** Is this file under construction? */
@@ -262,12 +257,7 @@ public class INodeFile extends INodeWithAdditionalFields
    * otherwise, return null.
    */
   public final FileWithSnapshotFeature getFileWithSnapshotFeature() {
-    for (Feature f: features) {
-      if (f instanceof FileWithSnapshotFeature) {
-        return (FileWithSnapshotFeature) f;
-      }
-    }
-    return null;
+    return getFeature(FileWithSnapshotFeature.class);
   }
 
   /** Is this file has the snapshot feature? */

+ 30 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeWithAdditionalFields.java

@@ -21,6 +21,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
+import org.apache.hadoop.hdfs.server.namenode.INode.Feature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.util.LightWeightGSet.LinkedElement;
 
@@ -306,4 +307,33 @@ public abstract class INodeWithAdditionalFields extends INode
         + f.getClass().getSimpleName() + " not found.");
     features = arr;
   }
+
+  protected <T extends Feature> T getFeature(Class<? extends Feature> clazz) {
+    for (Feature f : features) {
+      if (f.getClass() == clazz) {
+        @SuppressWarnings("unchecked")
+        T ret = (T) f;
+        return ret;
+      }
+    }
+    return null;
+  }
+
+  public AclFeature getAclFeature() {
+    return getFeature(AclFeature.class);
+  }
+
+  public void removeAclFeature() {
+    AclFeature f = getAclFeature();
+    Preconditions.checkNotNull(f);
+    removeFeature(f);
+  }
+
+  public void addAclFeature(AclFeature f) {
+    AclFeature f1 = getAclFeature();
+    if (f1 != null)
+      throw new IllegalStateException("Duplicated ACLFeature");
+
+    addFeature(f);
+  }
 }

+ 70 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeAcl.java

@@ -0,0 +1,70 @@
+/**
+ * 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.hdfs.server.namenode;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.AclEntryScope;
+import org.apache.hadoop.fs.permission.AclEntryType;
+import org.apache.hadoop.fs.permission.AclStatus;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+
+public class TestNameNodeAcl {
+
+  private static MiniDFSCluster cluster;
+  private static Configuration conf;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    conf = new Configuration();
+
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    cluster.waitActive();
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    cluster.shutdown();
+  }
+
+  @Test
+  public void testSetAcl() throws IOException {
+    Path p = new Path("/p");
+    FileSystem fs = cluster.getFileSystem();
+    fs.create(p).close();
+    AclEntry e = new AclEntry.Builder().setName("foo")
+        .setPermission(FsAction.READ_EXECUTE).setScope(AclEntryScope.DEFAULT)
+        .setType(AclEntryType.OTHER).build();
+    fs.setAcl(p, Lists.newArrayList(e));
+    AclStatus s = fs.getAclStatus(p);
+    AclEntry[] returned = Lists.newArrayList(s.getEntries()).toArray(
+        new AclEntry[0]);
+    Assert.assertArrayEquals(new AclEntry[] { e }, returned);
+  }
+}