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