1 /**
2  * Licensed to the Apache Software Foundation (ASF) under one
3  * or more contributor license agreements.  See the NOTICE file
4  * distributed with this work for additional information
5  * regarding copyright ownership.  The ASF licenses this file
6  * to you under the Apache License, Version 2.0 (the
7  * "License"); you may not use this file except in compliance
8  * with the License.  You may obtain a copy of the License at
9  *
10  *     http://www.apache.org/licenses/LICENSE-2.0
11  *
12  * Unless required by applicable law or agreed to in writing, software
13  * distributed under the License is distributed on an "AS IS" BASIS,
14  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15  * See the License for the specific language governing permissions and
16  * limitations under the License.
17  */
18 package org.apache.hadoop.hdfs.server.namenode;
19 
20 import java.util.Arrays;
21 import java.util.Collections;
22 import java.util.HashSet;
23 import java.util.Set;
24 import java.util.Stack;
25 
26 import org.apache.commons.logging.Log;
27 import org.apache.commons.logging.LogFactory;
28 import org.apache.hadoop.fs.permission.AclEntryScope;
29 import org.apache.hadoop.fs.permission.AclEntryType;
30 import org.apache.hadoop.fs.permission.FsAction;
31 import org.apache.hadoop.fs.permission.FsPermission;
32 import org.apache.hadoop.hdfs.DFSUtil;
33 import org.apache.hadoop.hdfs.server.namenode.INodeAttributeProvider.AccessControlEnforcer;
34 import org.apache.hadoop.hdfs.util.ReadOnlyList;
35 import org.apache.hadoop.security.AccessControlException;
36 import org.apache.hadoop.security.UserGroupInformation;
37 
38 /**
39  * Class that helps in checking file system permission.
40  * The state of this class need not be synchronized as it has data structures that
41  * are read-only.
42  *
43  * Some of the helper methods are gaurded by {@link FSNamesystem#readLock()}.
44  */
45 class FSPermissionChecker implements AccessControlEnforcer {
46   static final Log LOG = LogFactory.getLog(UserGroupInformation.class);
47 
48   /** @return a string for throwing {@link AccessControlException} */
toAccessControlString(INodeAttributes inodeAttrib, String path, FsAction access, FsPermission mode)49   private String toAccessControlString(INodeAttributes inodeAttrib, String path,
50       FsAction access, FsPermission mode) {
51     return toAccessControlString(inodeAttrib, path, access, mode, false);
52   }
53 
54   /** @return a string for throwing {@link AccessControlException} */
toAccessControlString(INodeAttributes inodeAttrib, String path, FsAction access, FsPermission mode, boolean deniedFromAcl)55   private String toAccessControlString(INodeAttributes inodeAttrib,
56       String path, FsAction access, FsPermission mode, boolean deniedFromAcl) {
57     StringBuilder sb = new StringBuilder("Permission denied: ")
58       .append("user=").append(getUser()).append(", ")
59       .append("access=").append(access).append(", ")
60       .append("inode=\"").append(path).append("\":")
61       .append(inodeAttrib.getUserName()).append(':')
62       .append(inodeAttrib.getGroupName()).append(':')
63       .append(inodeAttrib.isDirectory() ? 'd' : '-')
64       .append(mode);
65     if (deniedFromAcl) {
66       sb.append("+");
67     }
68     return sb.toString();
69   }
70 
71   private final String fsOwner;
72   private final String supergroup;
73   private final UserGroupInformation callerUgi;
74 
75   private final String user;
76   private final Set<String> groups;
77   private final boolean isSuper;
78   private final INodeAttributeProvider attributeProvider;
79 
80 
FSPermissionChecker(String fsOwner, String supergroup, UserGroupInformation callerUgi, INodeAttributeProvider attributeProvider)81   FSPermissionChecker(String fsOwner, String supergroup,
82       UserGroupInformation callerUgi,
83       INodeAttributeProvider attributeProvider) {
84     this.fsOwner = fsOwner;
85     this.supergroup = supergroup;
86     this.callerUgi = callerUgi;
87     HashSet<String> s =
88         new HashSet<String>(Arrays.asList(callerUgi.getGroupNames()));
89     groups = Collections.unmodifiableSet(s);
90     user = callerUgi.getShortUserName();
91     isSuper = user.equals(fsOwner) || groups.contains(supergroup);
92     this.attributeProvider = attributeProvider;
93   }
94 
containsGroup(String group)95   public boolean containsGroup(String group) {
96     return groups.contains(group);
97   }
98 
getUser()99   public String getUser() {
100     return user;
101   }
102 
getGroups()103   public Set<String> getGroups() {
104     return groups;
105   }
106 
isSuperUser()107   public boolean isSuperUser() {
108     return isSuper;
109   }
110 
getAttributesProvider()111   public INodeAttributeProvider getAttributesProvider() {
112     return attributeProvider;
113   }
114 
115   /**
116    * Verify if the caller has the required permission. This will result into
117    * an exception if the caller is not allowed to access the resource.
118    */
checkSuperuserPrivilege()119   public void checkSuperuserPrivilege()
120       throws AccessControlException {
121     if (!isSuperUser()) {
122       throw new AccessControlException("Access denied for user "
123           + getUser() + ". Superuser privilege is required");
124     }
125   }
126 
127   /**
128    * Check whether current user have permissions to access the path.
129    * Traverse is always checked.
130    *
131    * Parent path means the parent directory for the path.
132    * Ancestor path means the last (the closest) existing ancestor directory
133    * of the path.
134    * Note that if the parent path exists,
135    * then the parent path and the ancestor path are the same.
136    *
137    * For example, suppose the path is "/foo/bar/baz".
138    * No matter baz is a file or a directory,
139    * the parent path is "/foo/bar".
140    * If bar exists, then the ancestor path is also "/foo/bar".
141    * If bar does not exist and foo exists,
142    * then the ancestor path is "/foo".
143    * Further, if both foo and bar do not exist,
144    * then the ancestor path is "/".
145    *
146    * @param doCheckOwner Require user to be the owner of the path?
147    * @param ancestorAccess The access required by the ancestor of the path.
148    * @param parentAccess The access required by the parent of the path.
149    * @param access The access required by the path.
150    * @param subAccess If path is a directory,
151    * it is the access required of the path and all the sub-directories.
152    * If path is not a directory, there is no effect.
153    * @param ignoreEmptyDir Ignore permission checking for empty directory?
154    * @throws AccessControlException
155    *
156    * Guarded by {@link FSNamesystem#readLock()}
157    * Caller of this method must hold that lock.
158    */
checkPermission(INodesInPath inodesInPath, boolean doCheckOwner, FsAction ancestorAccess, FsAction parentAccess, FsAction access, FsAction subAccess, boolean ignoreEmptyDir)159   void checkPermission(INodesInPath inodesInPath, boolean doCheckOwner,
160       FsAction ancestorAccess, FsAction parentAccess, FsAction access,
161       FsAction subAccess, boolean ignoreEmptyDir)
162       throws AccessControlException {
163     if (LOG.isDebugEnabled()) {
164       LOG.debug("ACCESS CHECK: " + this
165           + ", doCheckOwner=" + doCheckOwner
166           + ", ancestorAccess=" + ancestorAccess
167           + ", parentAccess=" + parentAccess
168           + ", access=" + access
169           + ", subAccess=" + subAccess
170           + ", ignoreEmptyDir=" + ignoreEmptyDir);
171     }
172     // check if (parentAccess != null) && file exists, then check sb
173     // If resolveLink, the check is performed on the link target.
174     final int snapshotId = inodesInPath.getPathSnapshotId();
175     final INode[] inodes = inodesInPath.getINodesArray();
176     final INodeAttributes[] inodeAttrs = new INodeAttributes[inodes.length];
177     final byte[][] pathByNameArr = new byte[inodes.length][];
178     for (int i = 0; i < inodes.length && inodes[i] != null; i++) {
179       if (inodes[i] != null) {
180         pathByNameArr[i] = inodes[i].getLocalNameBytes();
181         inodeAttrs[i] = getINodeAttrs(pathByNameArr, i, inodes[i], snapshotId);
182       }
183     }
184 
185     String path = inodesInPath.getPath();
186     int ancestorIndex = inodes.length - 2;
187 
188     AccessControlEnforcer enforcer =
189         getAttributesProvider().getExternalAccessControlEnforcer(this);
190     enforcer.checkPermission(fsOwner, supergroup, callerUgi, inodeAttrs, inodes,
191         pathByNameArr, snapshotId, path, ancestorIndex, doCheckOwner,
192         ancestorAccess, parentAccess, access, subAccess, ignoreEmptyDir);
193   }
194 
195   @Override
checkPermission(String fsOwner, String supergroup, UserGroupInformation callerUgi, INodeAttributes[] inodeAttrs, INode[] inodes, byte[][] pathByNameArr, int snapshotId, String path, int ancestorIndex, boolean doCheckOwner, FsAction ancestorAccess, FsAction parentAccess, FsAction access, FsAction subAccess, boolean ignoreEmptyDir)196   public void checkPermission(String fsOwner, String supergroup,
197       UserGroupInformation callerUgi, INodeAttributes[] inodeAttrs,
198       INode[] inodes, byte[][] pathByNameArr, int snapshotId, String path,
199       int ancestorIndex, boolean doCheckOwner, FsAction ancestorAccess,
200       FsAction parentAccess, FsAction access, FsAction subAccess,
201       boolean ignoreEmptyDir)
202       throws AccessControlException {
203     for(; ancestorIndex >= 0 && inodes[ancestorIndex] == null;
204         ancestorIndex--);
205     checkTraverse(inodeAttrs, path, ancestorIndex);
206 
207     final INodeAttributes last = inodeAttrs[inodeAttrs.length - 1];
208     if (parentAccess != null && parentAccess.implies(FsAction.WRITE)
209         && inodeAttrs.length > 1 && last != null) {
210       checkStickyBit(inodeAttrs[inodeAttrs.length - 2], last);
211     }
212     if (ancestorAccess != null && inodeAttrs.length > 1) {
213       check(inodeAttrs, path, ancestorIndex, ancestorAccess);
214     }
215     if (parentAccess != null && inodeAttrs.length > 1) {
216       check(inodeAttrs, path, inodeAttrs.length - 2, parentAccess);
217     }
218     if (access != null) {
219       check(last, path, access);
220     }
221     if (subAccess != null) {
222       INode rawLast = inodes[inodeAttrs.length - 1];
223       checkSubAccess(pathByNameArr, inodeAttrs.length - 1, rawLast,
224           snapshotId, subAccess, ignoreEmptyDir);
225     }
226     if (doCheckOwner) {
227       checkOwner(last);
228     }
229   }
230 
getINodeAttrs(byte[][] pathByNameArr, int pathIdx, INode inode, int snapshotId)231   private INodeAttributes getINodeAttrs(byte[][] pathByNameArr, int pathIdx,
232       INode inode, int snapshotId) {
233     INodeAttributes inodeAttrs = inode.getSnapshotINode(snapshotId);
234     if (getAttributesProvider() != null) {
235       String[] elements = new String[pathIdx + 1];
236       for (int i = 0; i < elements.length; i++) {
237         elements[i] = DFSUtil.bytes2String(pathByNameArr[i]);
238       }
239       inodeAttrs = getAttributesProvider().getAttributes(elements, inodeAttrs);
240     }
241     return inodeAttrs;
242   }
243 
244   /** Guarded by {@link FSNamesystem#readLock()} */
checkOwner(INodeAttributes inode )245   private void checkOwner(INodeAttributes inode
246       ) throws AccessControlException {
247     if (getUser().equals(inode.getUserName())) {
248       return;
249     }
250     throw new AccessControlException(
251             "Permission denied. user="
252             + getUser() + " is not the owner of inode=" + inode);
253   }
254 
255   /** Guarded by {@link FSNamesystem#readLock()} */
checkTraverse(INodeAttributes[] inodes, String path, int last )256   private void checkTraverse(INodeAttributes[] inodes, String path, int last
257       ) throws AccessControlException {
258     for(int j = 0; j <= last; j++) {
259       check(inodes[j], path, FsAction.EXECUTE);
260     }
261   }
262 
263   /** Guarded by {@link FSNamesystem#readLock()} */
checkSubAccess(byte[][] pathByNameArr, int pathIdx, INode inode, int snapshotId, FsAction access, boolean ignoreEmptyDir)264   private void checkSubAccess(byte[][] pathByNameArr, int pathIdx, INode inode,
265       int snapshotId, FsAction access, boolean ignoreEmptyDir)
266       throws AccessControlException {
267     if (inode == null || !inode.isDirectory()) {
268       return;
269     }
270 
271     Stack<INodeDirectory> directories = new Stack<INodeDirectory>();
272     for(directories.push(inode.asDirectory()); !directories.isEmpty(); ) {
273       INodeDirectory d = directories.pop();
274       ReadOnlyList<INode> cList = d.getChildrenList(snapshotId);
275       if (!(cList.isEmpty() && ignoreEmptyDir)) {
276         //TODO have to figure this out with inodeattribute provider
277         check(getINodeAttrs(pathByNameArr, pathIdx, d, snapshotId),
278             inode.getFullPathName(), access);
279       }
280 
281       for(INode child : cList) {
282         if (child.isDirectory()) {
283           directories.push(child.asDirectory());
284         }
285       }
286     }
287   }
288 
289   /** Guarded by {@link FSNamesystem#readLock()} */
check(INodeAttributes[] inodes, String path, int i, FsAction access )290   private void check(INodeAttributes[] inodes, String path, int i, FsAction access
291       ) throws AccessControlException {
292     check(i >= 0 ? inodes[i] : null, path, access);
293   }
294 
check(INodeAttributes inode, String path, FsAction access )295   private void check(INodeAttributes inode, String path, FsAction access
296       ) throws AccessControlException {
297     if (inode == null) {
298       return;
299     }
300     final FsPermission mode = inode.getFsPermission();
301     final AclFeature aclFeature = inode.getAclFeature();
302     if (aclFeature != null) {
303       // It's possible that the inode has a default ACL but no access ACL.
304       int firstEntry = aclFeature.getEntryAt(0);
305       if (AclEntryStatusFormat.getScope(firstEntry) == AclEntryScope.ACCESS) {
306         checkAccessAcl(inode, path, access, mode, aclFeature);
307         return;
308       }
309     }
310     if (getUser().equals(inode.getUserName())) { //user class
311       if (mode.getUserAction().implies(access)) { return; }
312     }
313     else if (getGroups().contains(inode.getGroupName())) { //group class
314       if (mode.getGroupAction().implies(access)) { return; }
315     }
316     else { //other class
317       if (mode.getOtherAction().implies(access)) { return; }
318     }
319     throw new AccessControlException(
320         toAccessControlString(inode, path, access, mode));
321   }
322 
323   /**
324    * Checks requested access against an Access Control List.  This method relies
325    * on finding the ACL data in the relevant portions of {@link FsPermission} and
326    * {@link AclFeature} as implemented in the logic of {@link AclStorage}.  This
327    * method also relies on receiving the ACL entries in sorted order.  This is
328    * assumed to be true, because the ACL modification methods in
329    * {@link AclTransformation} sort the resulting entries.
330    *
331    * More specifically, this method depends on these invariants in an ACL:
332    * - The list must be sorted.
333    * - Each entry in the list must be unique by scope + type + name.
334    * - There is exactly one each of the unnamed user/group/other entries.
335    * - The mask entry must not have a name.
336    * - The other entry must not have a name.
337    * - Default entries may be present, but they are ignored during enforcement.
338    *
339    * @param inode INodeAttributes accessed inode
340    * @param snapshotId int snapshot ID
341    * @param access FsAction requested permission
342    * @param mode FsPermission mode from inode
343    * @param aclFeature AclFeature of inode
344    * @throws AccessControlException if the ACL denies permission
345    */
checkAccessAcl(INodeAttributes inode, String path, FsAction access, FsPermission mode, AclFeature aclFeature)346   private void checkAccessAcl(INodeAttributes inode, String path,
347       FsAction access, FsPermission mode, AclFeature aclFeature)
348       throws AccessControlException {
349     boolean foundMatch = false;
350 
351     // Use owner entry from permission bits if user is owner.
352     if (getUser().equals(inode.getUserName())) {
353       if (mode.getUserAction().implies(access)) {
354         return;
355       }
356       foundMatch = true;
357     }
358 
359     // Check named user and group entries if user was not denied by owner entry.
360     if (!foundMatch) {
361       for (int pos = 0, entry; pos < aclFeature.getEntriesSize(); pos++) {
362         entry = aclFeature.getEntryAt(pos);
363         if (AclEntryStatusFormat.getScope(entry) == AclEntryScope.DEFAULT) {
364           break;
365         }
366         AclEntryType type = AclEntryStatusFormat.getType(entry);
367         String name = AclEntryStatusFormat.getName(entry);
368         if (type == AclEntryType.USER) {
369           // Use named user entry with mask from permission bits applied if user
370           // matches name.
371           if (getUser().equals(name)) {
372             FsAction masked = AclEntryStatusFormat.getPermission(entry).and(
373                 mode.getGroupAction());
374             if (masked.implies(access)) {
375               return;
376             }
377             foundMatch = true;
378             break;
379           }
380         } else if (type == AclEntryType.GROUP) {
381           // Use group entry (unnamed or named) with mask from permission bits
382           // applied if user is a member and entry grants access.  If user is a
383           // member of multiple groups that have entries that grant access, then
384           // it doesn't matter which is chosen, so exit early after first match.
385           String group = name == null ? inode.getGroupName() : name;
386           if (getGroups().contains(group)) {
387             FsAction masked = AclEntryStatusFormat.getPermission(entry).and(
388                 mode.getGroupAction());
389             if (masked.implies(access)) {
390               return;
391             }
392             foundMatch = true;
393           }
394         }
395       }
396     }
397 
398     // Use other entry if user was not denied by an earlier match.
399     if (!foundMatch && mode.getOtherAction().implies(access)) {
400       return;
401     }
402 
403     throw new AccessControlException(
404         toAccessControlString(inode, path, access, mode));
405   }
406 
407   /** Guarded by {@link FSNamesystem#readLock()} */
checkStickyBit(INodeAttributes parent, INodeAttributes inode )408   private void checkStickyBit(INodeAttributes parent, INodeAttributes inode
409       ) throws AccessControlException {
410     if (!parent.getFsPermission().getStickyBit()) {
411       return;
412     }
413 
414     // If this user is the directory owner, return
415     if (parent.getUserName().equals(getUser())) {
416       return;
417     }
418 
419     // if this user is the file owner, return
420     if (inode.getUserName().equals(getUser())) {
421       return;
422     }
423 
424     throw new AccessControlException("Permission denied by sticky bit setting:" +
425       " user=" + getUser() + ", inode=" + inode);
426   }
427 
428   /**
429    * Whether a cache pool can be accessed by the current context
430    *
431    * @param pool CachePool being accessed
432    * @param access type of action being performed on the cache pool
433    * @throws AccessControlException if pool cannot be accessed
434    */
checkPermission(CachePool pool, FsAction access)435   public void checkPermission(CachePool pool, FsAction access)
436       throws AccessControlException {
437     FsPermission mode = pool.getMode();
438     if (isSuperUser()) {
439       return;
440     }
441     if (getUser().equals(pool.getOwnerName())
442         && mode.getUserAction().implies(access)) {
443       return;
444     }
445     if (getGroups().contains(pool.getGroupName())
446         && mode.getGroupAction().implies(access)) {
447       return;
448     }
449     if (mode.getOtherAction().implies(access)) {
450       return;
451     }
452     throw new AccessControlException("Permission denied while accessing pool "
453         + pool.getPoolName() + ": user " + getUser() + " does not have "
454         + access.toString() + " permissions.");
455   }
456 }
457