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