001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018
019package org.apache.hadoop.hbase.security.access;
020
021import java.io.IOException;
022import java.util.HashMap;
023import java.util.HashSet;
024import java.util.List;
025import java.util.Map;
026import java.util.Set;
027import java.util.concurrent.ConcurrentHashMap;
028import java.util.concurrent.atomic.AtomicLong;
029
030import org.apache.hadoop.conf.Configuration;
031import org.apache.hadoop.hbase.AuthUtil;
032import org.apache.hadoop.hbase.Cell;
033import org.apache.hadoop.hbase.TableName;
034import org.apache.hadoop.hbase.exceptions.DeserializationException;
035import org.apache.hadoop.hbase.security.Superusers;
036import org.apache.hadoop.hbase.security.User;
037import org.apache.hadoop.hbase.util.Bytes;
038import org.apache.yetus.audience.InterfaceAudience;
039import org.slf4j.Logger;
040import org.slf4j.LoggerFactory;
041
042import org.apache.hbase.thirdparty.com.google.common.collect.ListMultimap;
043
044/**
045 * Performs authorization checks for a given user's assigned permissions.
046 * <p>
047 *   There're following scopes: <b>Global</b>, <b>Namespace</b>, <b>Table</b>, <b>Family</b>,
048 *   <b>Qualifier</b>, <b>Cell</b>.
049 *   Generally speaking, higher scopes can overrides lower scopes,
050 *   except for Cell permission can be granted even a user has not permission on specified table,
051 *   which means the user can get/scan only those granted cells parts.
052 * </p>
053 * e.g, if user A has global permission R(ead), he can
054 * read table T without checking table scope permission, so authorization checks alway starts from
055 * Global scope.
056 * <p>
057 *   For each scope, not only user but also groups he belongs to will be checked.
058 * </p>
059 */
060@InterfaceAudience.Private
061public final class AuthManager {
062
063  /**
064   * Cache of permissions, it is thread safe.
065   * @param <T> T extends Permission
066   */
067  private static class PermissionCache<T extends Permission> {
068    private final Object mutex = new Object();
069    private Map<String, Set<T>> cache = new HashMap<>();
070
071    void put(String name, T perm) {
072      synchronized (mutex) {
073        Set<T> perms = cache.getOrDefault(name, new HashSet<>());
074        perms.add(perm);
075        cache.put(name, perms);
076      }
077    }
078
079    Set<T> get(String name) {
080      synchronized (mutex) {
081        return cache.get(name);
082      }
083    }
084
085    void clear() {
086      synchronized (mutex) {
087        for (Map.Entry<String, Set<T>> entry : cache.entrySet()) {
088          entry.getValue().clear();
089        }
090        cache.clear();
091      }
092    }
093  }
094  PermissionCache<NamespacePermission> NS_NO_PERMISSION = new PermissionCache<>();
095  PermissionCache<TablePermission> TBL_NO_PERMISSION = new PermissionCache<>();
096
097  /**
098   * Cache for global permission excluding superuser and supergroup.
099   * Since every user/group can only have one global permission, no need to use PermissionCache.
100   */
101  private Map<String, GlobalPermission> globalCache = new ConcurrentHashMap<>();
102  /** Cache for namespace permission. */
103  private ConcurrentHashMap<String, PermissionCache<NamespacePermission>> namespaceCache =
104    new ConcurrentHashMap<>();
105  /** Cache for table permission. */
106  private ConcurrentHashMap<TableName, PermissionCache<TablePermission>> tableCache =
107    new ConcurrentHashMap<>();
108
109  private static final Logger LOG = LoggerFactory.getLogger(AuthManager.class);
110
111  private Configuration conf;
112  private final AtomicLong mtime = new AtomicLong(0L);
113
114  AuthManager(Configuration conf) {
115    this.conf = conf;
116  }
117
118  /**
119   * Update acl info for table.
120   * @param table name of table
121   * @param data updated acl data
122   * @throws IOException exception when deserialize data
123   */
124  public void refreshTableCacheFromWritable(TableName table, byte[] data) throws IOException {
125    if (data != null && data.length > 0) {
126      try {
127        ListMultimap<String, Permission> perms = PermissionStorage.readPermissions(data, conf);
128        if (perms != null) {
129          if (Bytes.equals(table.getName(), PermissionStorage.ACL_GLOBAL_NAME)) {
130            updateGlobalCache(perms);
131          } else {
132            updateTableCache(table, perms);
133          }
134        }
135      } catch (DeserializationException e) {
136        throw new IOException(e);
137      }
138    } else {
139      LOG.info("Skipping permission cache refresh because writable data is empty");
140    }
141  }
142
143  /**
144   * Update acl info for namespace.
145   * @param namespace namespace
146   * @param data updated acl data
147   * @throws IOException exception when deserialize data
148   */
149  public void refreshNamespaceCacheFromWritable(String namespace, byte[] data) throws IOException {
150    if (data != null && data.length > 0) {
151      try {
152        ListMultimap<String, Permission> perms = PermissionStorage.readPermissions(data, conf);
153        if (perms != null) {
154          updateNamespaceCache(namespace, perms);
155        }
156      } catch (DeserializationException e) {
157        throw new IOException(e);
158      }
159    } else {
160      LOG.debug("Skipping permission cache refresh because writable data is empty");
161    }
162  }
163
164  /**
165   * Updates the internal global permissions cache.
166   * @param globalPerms new global permissions
167   */
168  private void updateGlobalCache(ListMultimap<String, Permission> globalPerms) {
169    globalCache.clear();
170    for (String name : globalPerms.keySet()) {
171      for (Permission permission : globalPerms.get(name)) {
172        // Before 2.2, the global permission which storage in zk is not right. It was saved as a
173        // table permission. So here need to handle this for compatibility. See HBASE-22503.
174        if (permission instanceof TablePermission) {
175          globalCache.put(name, new GlobalPermission(permission.getActions()));
176        } else {
177          globalCache.put(name, (GlobalPermission) permission);
178        }
179      }
180    }
181    mtime.incrementAndGet();
182  }
183
184  /**
185   * Updates the internal table permissions cache for specified table.
186   * @param table updated table name
187   * @param tablePerms new table permissions
188   */
189  private void updateTableCache(TableName table, ListMultimap<String, Permission> tablePerms) {
190    PermissionCache<TablePermission> cacheToUpdate =
191      tableCache.getOrDefault(table, new PermissionCache<>());
192    clearCache(cacheToUpdate);
193    updateCache(tablePerms, cacheToUpdate);
194    tableCache.put(table, cacheToUpdate);
195    mtime.incrementAndGet();
196  }
197
198  /**
199   * Updates the internal namespace permissions cache for specified namespace.
200   * @param namespace updated namespace
201   * @param nsPerms new namespace permissions
202   */
203  private void updateNamespaceCache(String namespace,
204      ListMultimap<String, Permission> nsPerms) {
205    PermissionCache<NamespacePermission> cacheToUpdate =
206      namespaceCache.getOrDefault(namespace, new PermissionCache<>());
207    clearCache(cacheToUpdate);
208    updateCache(nsPerms, cacheToUpdate);
209    namespaceCache.put(namespace, cacheToUpdate);
210    mtime.incrementAndGet();
211  }
212
213  private void clearCache(PermissionCache cacheToUpdate) {
214    cacheToUpdate.clear();
215  }
216
217  @SuppressWarnings("unchecked")
218  private void updateCache(ListMultimap<String, ? extends Permission> newPermissions,
219      PermissionCache cacheToUpdate) {
220    for (String name : newPermissions.keySet()) {
221      for (Permission permission : newPermissions.get(name)) {
222        cacheToUpdate.put(name, permission);
223      }
224    }
225  }
226
227  /**
228   * Check if user has given action privilige in global scope.
229   * @param user user name
230   * @param action one of action in [Read, Write, Create, Exec, Admin]
231   * @return true if user has, false otherwise
232   */
233  public boolean authorizeUserGlobal(User user, Permission.Action action) {
234    if (user == null) {
235      return false;
236    }
237    if (Superusers.isSuperUser(user)) {
238      return true;
239    }
240    if (authorizeGlobal(globalCache.get(user.getShortName()), action)) {
241      return true;
242    }
243    for (String group : user.getGroupNames()) {
244      if (authorizeGlobal(globalCache.get(AuthUtil.toGroupEntry(group)), action)) {
245        return true;
246      }
247    }
248    return false;
249  }
250
251  private boolean authorizeGlobal(GlobalPermission permissions, Permission.Action action) {
252    return permissions != null && permissions.implies(action);
253  }
254
255  /**
256   * Check if user has given action privilige in namespace scope.
257   * @param user user name
258   * @param namespace namespace
259   * @param action one of action in [Read, Write, Create, Exec, Admin]
260   * @return true if user has, false otherwise
261   */
262  public boolean authorizeUserNamespace(User user, String namespace, Permission.Action action) {
263    if (user == null) {
264      return false;
265    }
266    if (authorizeUserGlobal(user, action)) {
267      return true;
268    }
269    PermissionCache<NamespacePermission> nsPermissions = namespaceCache.getOrDefault(namespace,
270      NS_NO_PERMISSION);
271    if (authorizeNamespace(nsPermissions.get(user.getShortName()), namespace, action)) {
272      return true;
273    }
274    for (String group : user.getGroupNames()) {
275      if (authorizeNamespace(nsPermissions.get(AuthUtil.toGroupEntry(group)), namespace, action)) {
276        return true;
277      }
278    }
279    return false;
280  }
281
282  private boolean authorizeNamespace(Set<NamespacePermission> permissions,
283      String namespace, Permission.Action action) {
284    if (permissions == null) {
285      return false;
286    }
287    for (NamespacePermission permission : permissions) {
288      if (permission.implies(namespace, action)) {
289        return true;
290      }
291    }
292    return false;
293  }
294
295  /**
296   * Checks if the user has access to the full table or at least a family/qualifier
297   * for the specified action.
298   * @param user user name
299   * @param table table name
300   * @param action action in one of [Read, Write, Create, Exec, Admin]
301   * @return true if the user has access to the table, false otherwise
302   */
303  public boolean accessUserTable(User user, TableName table, Permission.Action action) {
304    if (user == null) {
305      return false;
306    }
307    if (table == null) {
308      table = PermissionStorage.ACL_TABLE_NAME;
309    }
310    if (authorizeUserNamespace(user, table.getNamespaceAsString(), action)) {
311      return true;
312    }
313    PermissionCache<TablePermission> tblPermissions = tableCache.getOrDefault(table,
314      TBL_NO_PERMISSION);
315    if (hasAccessTable(tblPermissions.get(user.getShortName()), action)) {
316      return true;
317    }
318    for (String group : user.getGroupNames()) {
319      if (hasAccessTable(tblPermissions.get(AuthUtil.toGroupEntry(group)), action)) {
320        return true;
321      }
322    }
323    return false;
324  }
325
326  private boolean hasAccessTable(Set<TablePermission> permissions, Permission.Action action) {
327    if (permissions == null) {
328      return false;
329    }
330    for (TablePermission permission : permissions) {
331      if (permission.implies(action)) {
332        return true;
333      }
334    }
335    return false;
336  }
337
338  /**
339   * Check if user has given action privilige in table scope.
340   * @param user user name
341   * @param table table name
342   * @param action one of action in [Read, Write, Create, Exec, Admin]
343   * @return true if user has, false otherwise
344   */
345  public boolean authorizeUserTable(User user, TableName table, Permission.Action action) {
346    return authorizeUserTable(user, table, null, null, action);
347  }
348
349  /**
350   * Check if user has given action privilige in table:family scope.
351   * @param user user name
352   * @param table table name
353   * @param family family name
354   * @param action one of action in [Read, Write, Create, Exec, Admin]
355   * @return true if user has, false otherwise
356   */
357  public boolean authorizeUserTable(User user, TableName table, byte[] family,
358      Permission.Action action) {
359    return authorizeUserTable(user, table, family, null, action);
360  }
361
362  /**
363   * Check if user has given action privilige in table:family:qualifier scope.
364   * @param user user name
365   * @param table table name
366   * @param family family name
367   * @param qualifier qualifier name
368   * @param action one of action in [Read, Write, Create, Exec, Admin]
369   * @return true if user has, false otherwise
370   */
371  public boolean authorizeUserTable(User user, TableName table, byte[] family,
372      byte[] qualifier, Permission.Action action) {
373    if (user == null) {
374      return false;
375    }
376    if (table == null) {
377      table = PermissionStorage.ACL_TABLE_NAME;
378    }
379    if (authorizeUserNamespace(user, table.getNamespaceAsString(), action)) {
380      return true;
381    }
382    PermissionCache<TablePermission> tblPermissions = tableCache.getOrDefault(table,
383      TBL_NO_PERMISSION);
384    if (authorizeTable(tblPermissions.get(user.getShortName()), table, family, qualifier, action)) {
385      return true;
386    }
387    for (String group : user.getGroupNames()) {
388      if (authorizeTable(tblPermissions.get(AuthUtil.toGroupEntry(group)),
389          table, family, qualifier, action)) {
390        return true;
391      }
392    }
393    return false;
394  }
395
396  private boolean authorizeTable(Set<TablePermission> permissions,
397      TableName table, byte[] family, byte[] qualifier, Permission.Action action) {
398    if (permissions == null) {
399      return false;
400    }
401    for (TablePermission permission : permissions) {
402      if (permission.implies(table, family, qualifier, action)) {
403        return true;
404      }
405    }
406    return false;
407  }
408
409  /**
410   * Check if user has given action privilige in table:family scope.
411   * This method is for backward compatibility.
412   * @param user user name
413   * @param table table name
414   * @param family family names
415   * @param action one of action in [Read, Write, Create, Exec, Admin]
416   * @return true if user has, false otherwise
417   */
418  public boolean authorizeUserFamily(User user, TableName table,
419      byte[] family, Permission.Action action) {
420    PermissionCache<TablePermission> tblPermissions = tableCache.getOrDefault(table,
421      TBL_NO_PERMISSION);
422    if (authorizeFamily(tblPermissions.get(user.getShortName()), table, family, action)) {
423      return true;
424    }
425    for (String group : user.getGroupNames()) {
426      if (authorizeFamily(tblPermissions.get(AuthUtil.toGroupEntry(group)),
427          table, family, action)) {
428        return true;
429      }
430    }
431    return false;
432  }
433
434  private boolean authorizeFamily(Set<TablePermission> permissions,
435      TableName table, byte[] family, Permission.Action action) {
436    if (permissions == null) {
437      return false;
438    }
439    for (TablePermission permission : permissions) {
440      if (permission.implies(table, family, action)) {
441        return true;
442      }
443    }
444    return false;
445  }
446
447  /**
448   * Check if user has given action privilige in cell scope.
449   * @param user user name
450   * @param table table name
451   * @param cell cell to be checked
452   * @param action one of action in [Read, Write, Create, Exec, Admin]
453   * @return true if user has, false otherwise
454   */
455  public boolean authorizeCell(User user, TableName table, Cell cell, Permission.Action action) {
456    try {
457      List<Permission> perms = PermissionStorage.getCellPermissionsForUser(user, cell);
458      if (LOG.isTraceEnabled()) {
459        LOG.trace("Perms for user {} in table {} in cell {}: {}",
460          user.getShortName(), table, cell, (perms != null ? perms : ""));
461      }
462      if (perms != null) {
463        for (Permission p: perms) {
464          if (p.implies(action)) {
465            return true;
466          }
467        }
468      }
469    } catch (IOException e) {
470      // We failed to parse the KV tag
471      LOG.error("Failed parse of ACL tag in cell " + cell);
472      // Fall through to check with the table and CF perms we were able
473      // to collect regardless
474    }
475    return false;
476  }
477
478  /**
479   * Remove given namespace from AuthManager's namespace cache.
480   * @param ns namespace
481   */
482  public void removeNamespace(byte[] ns) {
483    namespaceCache.remove(Bytes.toString(ns));
484  }
485
486  /**
487   * Remove given table from AuthManager's table cache.
488   * @param table table name
489   */
490  public void removeTable(TableName table) {
491    tableCache.remove(table);
492  }
493
494  /**
495   * Last modification logical time
496   * @return time
497   */
498  public long getMTime() {
499    return mtime.get();
500  }
501}