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,
013 * software distributed under the License is distributed on an
014 * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
015 * KIND, either express or implied.  See the License for the
016 * specific language governing permissions and limitations
017 * under the License.
018 */
019package org.apache.hadoop.hbase.security.access;
020
021import java.io.IOException;
022import java.net.InetAddress;
023import java.security.PrivilegedAction;
024import java.security.PrivilegedExceptionAction;
025import java.util.ArrayList;
026import java.util.Collection;
027import java.util.List;
028import java.util.Map;
029import java.util.Set;
030import java.util.TreeMap;
031
032import org.apache.hadoop.conf.Configuration;
033import org.apache.hadoop.hbase.AuthUtil;
034import org.apache.hadoop.hbase.Cell;
035import org.apache.hadoop.hbase.CellUtil;
036import org.apache.hadoop.hbase.DoNotRetryIOException;
037import org.apache.hadoop.hbase.HBaseInterfaceAudience;
038import org.apache.hadoop.hbase.NamespaceDescriptor;
039import org.apache.hadoop.hbase.TableName;
040import org.apache.hadoop.hbase.client.RegionInfo;
041import org.apache.hadoop.hbase.ipc.RpcServer;
042import org.apache.hadoop.hbase.security.AccessDeniedException;
043import org.apache.hadoop.hbase.security.Superusers;
044import org.apache.hadoop.hbase.security.User;
045import org.apache.hadoop.hbase.security.UserProvider;
046import org.apache.hadoop.hbase.security.access.Permission.Action;
047import org.apache.hadoop.hbase.util.Bytes;
048import org.apache.hadoop.security.Groups;
049import org.apache.hadoop.security.HadoopKerberosName;
050import org.apache.yetus.audience.InterfaceAudience;
051import org.apache.yetus.audience.InterfaceStability;
052import org.slf4j.Logger;
053import org.slf4j.LoggerFactory;
054import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSet;
055
056@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
057@InterfaceStability.Evolving
058public class AccessChecker {
059  private static final Logger LOG = LoggerFactory.getLogger(AccessChecker.class);
060  private static final Logger AUDITLOG =
061      LoggerFactory.getLogger("SecurityLogger." + AccessChecker.class.getName());
062  private final AuthManager authManager;
063
064  /** Group service to retrieve the user group information */
065  private static Groups groupService;
066
067  public static boolean isAuthorizationSupported(Configuration conf) {
068    return conf.getBoolean(User.HBASE_SECURITY_AUTHORIZATION_CONF_KEY, false);
069  }
070
071  /**
072   * Constructor with existing configuration
073   *
074   * @param conf Existing configuration to use
075   */
076  public AccessChecker(final Configuration conf) {
077    this.authManager = new AuthManager(conf);
078    initGroupService(conf);
079  }
080
081  public AuthManager getAuthManager() {
082    return authManager;
083  }
084
085  /**
086   * Authorizes that the current user has any of the given permissions to access the table.
087   *
088   * @param user Active user to which authorization checks should be applied
089   * @param request Request type.
090   * @param tableName   Table requested
091   * @param permissions Actions being requested
092   * @throws IOException if obtaining the current user fails
093   * @throws AccessDeniedException if user has no authorization
094   */
095  public void requireAccess(User user, String request, TableName tableName,
096      Action... permissions) throws IOException {
097    AuthResult result = null;
098
099    for (Action permission : permissions) {
100      if (authManager.accessUserTable(user, tableName, permission)) {
101        result = AuthResult.allow(request, "Table permission granted",
102            user, permission, tableName, null, null);
103        break;
104      } else {
105        // rest of the world
106        result = AuthResult.deny(request, "Insufficient permissions",
107            user, permission, tableName, null, null);
108      }
109    }
110    logResult(result);
111    if (!result.isAllowed()) {
112      throw new AccessDeniedException("Insufficient permissions " + result.toContextString());
113    }
114  }
115
116  /**
117   * Authorizes that the current user has global privileges for the given action.
118   * @param user Active user to which authorization checks should be applied
119   * @param request Request type
120   * @param filterUser User name to be filtered from permission as requested
121   * @param perm The action being requested
122   * @throws IOException if obtaining the current user fails
123   * @throws AccessDeniedException if authorization is denied
124   */
125  public void requirePermission(User user, String request, String filterUser, Action perm)
126      throws IOException {
127    requireGlobalPermission(user, request, perm, null, null, filterUser);
128  }
129
130  /**
131   * Checks that the user has the given global permission. The generated
132   * audit log message will contain context information for the operation
133   * being authorized, based on the given parameters.
134   *
135   * @param user Active user to which authorization checks should be applied
136   * @param request Request type
137   * @param perm      Action being requested
138   * @param tableName Affected table name.
139   * @param familyMap Affected column families.
140   * @param filterUser User name to be filtered from permission as requested
141   */
142  public void requireGlobalPermission(User user, String request,
143      Action perm, TableName tableName,
144      Map<byte[], ? extends Collection<byte[]>> familyMap, String filterUser) throws IOException {
145    AuthResult result;
146    if (authManager.authorizeUserGlobal(user, perm)) {
147      result = AuthResult.allow(request, "Global check allowed", user, perm, tableName, familyMap);
148    } else {
149      result = AuthResult.deny(request, "Global check failed", user, perm, tableName, familyMap);
150    }
151    result.getParams().setTableName(tableName).setFamilies(familyMap);
152    result.getParams().addExtraParam("filterUser", filterUser);
153    logResult(result);
154    if (!result.isAllowed()) {
155      throw new AccessDeniedException(
156          "Insufficient permissions for user '" + (user != null ? user.getShortName() : "null")
157              + "' (global, action=" + perm.toString() + ")");
158    }
159  }
160
161  /**
162   * Checks that the user has the given global permission. The generated
163   * audit log message will contain context information for the operation
164   * being authorized, based on the given parameters.
165   *
166   * @param user Active user to which authorization checks should be applied
167   * @param request Request type
168   * @param perm      Action being requested
169   * @param namespace The given namespace
170   */
171  public void requireGlobalPermission(User user, String request, Action perm,
172      String namespace) throws IOException {
173    AuthResult authResult;
174    if (authManager.authorizeUserGlobal(user, perm)) {
175      authResult = AuthResult.allow(request, "Global check allowed", user, perm, null);
176      authResult.getParams().setNamespace(namespace);
177      logResult(authResult);
178    } else {
179      authResult = AuthResult.deny(request, "Global check failed", user, perm, null);
180      authResult.getParams().setNamespace(namespace);
181      logResult(authResult);
182      throw new AccessDeniedException(
183          "Insufficient permissions for user '" + (user != null ? user.getShortName() : "null")
184              + "' (global, action=" + perm.toString() + ")");
185    }
186  }
187
188  /**
189   * Checks that the user has the given global or namespace permission.
190   * @param user Active user to which authorization checks should be applied
191   * @param request Request type
192   * @param namespace Name space as requested
193   * @param filterUser User name to be filtered from permission as requested
194   * @param permissions Actions being requested
195   */
196  public void requireNamespacePermission(User user, String request, String namespace,
197      String filterUser, Action... permissions) throws IOException {
198    AuthResult result = null;
199
200    for (Action permission : permissions) {
201      if (authManager.authorizeUserNamespace(user, namespace, permission)) {
202        result =
203            AuthResult.allow(request, "Namespace permission granted", user, permission, namespace);
204        break;
205      } else {
206        // rest of the world
207        result = AuthResult.deny(request, "Insufficient permissions", user, permission, namespace);
208      }
209    }
210    result.getParams().addExtraParam("filterUser", filterUser);
211    logResult(result);
212    if (!result.isAllowed()) {
213      throw new AccessDeniedException("Insufficient permissions " + result.toContextString());
214    }
215  }
216
217  /**
218   * Checks that the user has the given global or namespace permission.
219   *
220   * @param user Active user to which authorization checks should be applied
221   * @param request Request type
222   * @param namespace  The given namespace
223   * @param tableName Table requested
224   * @param familyMap    Column family map requested
225   * @param permissions Actions being requested
226   */
227  public void requireNamespacePermission(User user, String request, String namespace,
228      TableName tableName, Map<byte[], ? extends Collection<byte[]>> familyMap,
229      Action... permissions) throws IOException {
230    AuthResult result = null;
231
232    for (Action permission : permissions) {
233      if (authManager.authorizeUserNamespace(user, namespace, permission)) {
234        result =
235            AuthResult.allow(request, "Namespace permission granted", user, permission, namespace);
236        result.getParams().setTableName(tableName).setFamilies(familyMap);
237        break;
238      } else {
239        // rest of the world
240        result = AuthResult.deny(request, "Insufficient permissions", user, permission, namespace);
241        result.getParams().setTableName(tableName).setFamilies(familyMap);
242      }
243    }
244    logResult(result);
245    if (!result.isAllowed()) {
246      throw new AccessDeniedException("Insufficient permissions " + result.toContextString());
247    }
248  }
249
250  /**
251   * Authorizes that the current user has any of the given permissions for the
252   * given table, column family and column qualifier.
253   *
254   * @param user Active user to which authorization checks should be applied
255   * @param request Request type
256   * @param tableName Table requested
257   * @param family    Column family requested
258   * @param qualifier Column qualifier requested
259   * @param filterUser User name to be filtered from permission as requested
260   * @param permissions Actions being requested
261   * @throws IOException if obtaining the current user fails
262   * @throws AccessDeniedException if user has no authorization
263   */
264  public void requirePermission(User user, String request, TableName tableName, byte[] family,
265      byte[] qualifier, String filterUser, Action... permissions) throws IOException {
266    AuthResult result = null;
267
268    for (Action permission : permissions) {
269      if (authManager.authorizeUserTable(user, tableName, family, qualifier, permission)) {
270        result = AuthResult.allow(request, "Table permission granted",
271            user, permission, tableName, family, qualifier);
272        break;
273      } else {
274        // rest of the world
275        result = AuthResult.deny(request, "Insufficient permissions",
276          user, permission, tableName, family, qualifier);
277      }
278    }
279    result.getParams().addExtraParam("filterUser", filterUser);
280    logResult(result);
281    if (!result.isAllowed()) {
282      throw new AccessDeniedException("Insufficient permissions " + result.toContextString());
283    }
284  }
285
286  /**
287   * Authorizes that the current user has any of the given permissions for the
288   * given table, column family and column qualifier.
289   *
290   * @param user Active user to which authorization checks should be applied
291   * @param request Request type
292   * @param tableName Table requested
293   * @param family    Column family param
294   * @param qualifier Column qualifier param
295   * @throws IOException           if obtaining the current user fails
296   * @throws AccessDeniedException if user has no authorization
297   */
298  public void requireTablePermission(User user, String request,
299      TableName tableName,byte[] family, byte[] qualifier,
300      Action... permissions) throws IOException {
301    AuthResult result = null;
302
303    for (Action permission : permissions) {
304      if (authManager.authorizeUserTable(user, tableName, permission)) {
305        result = AuthResult.allow(request, "Table permission granted",
306            user, permission, tableName, null, null);
307        result.getParams().setFamily(family).setQualifier(qualifier);
308        break;
309      } else {
310        // rest of the world
311        result = AuthResult.deny(request, "Insufficient permissions",
312                user, permission, tableName, family, qualifier);
313        result.getParams().setFamily(family).setQualifier(qualifier);
314      }
315    }
316    logResult(result);
317    if (!result.isAllowed()) {
318      throw new AccessDeniedException("Insufficient permissions " + result.toContextString());
319    }
320  }
321
322  /**
323   * Check if caller is granting or revoking superusers's or supergroups's permissions.
324   * @param request request name
325   * @param caller caller
326   * @param userToBeChecked target user or group
327   * @throws IOException AccessDeniedException if target user is superuser
328   */
329  public void performOnSuperuser(String request, User caller, String userToBeChecked)
330      throws IOException {
331    List<String> userGroups = new ArrayList<>();
332    userGroups.add(userToBeChecked);
333    if (!AuthUtil.isGroupPrincipal(userToBeChecked)) {
334      for (String group : getUserGroups(userToBeChecked)) {
335        userGroups.add(AuthUtil.toGroupEntry(group));
336      }
337    }
338    for (String name : userGroups) {
339      if (Superusers.isSuperUser(name)) {
340        AuthResult result = AuthResult.deny(
341          request,
342          "Granting or revoking superusers's or supergroups's permissions is not allowed",
343          caller,
344          Action.ADMIN,
345          NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR);
346        logResult(result);
347        throw new AccessDeniedException(result.getReason());
348      }
349    }
350  }
351
352  public void checkLockPermissions(User user, String namespace,
353      TableName tableName, RegionInfo[] regionInfos, String reason)
354      throws IOException {
355    if (namespace != null && !namespace.isEmpty()) {
356      requireNamespacePermission(user, reason, namespace, null, Action.ADMIN, Action.CREATE);
357    } else if (tableName != null || (regionInfos != null && regionInfos.length > 0)) {
358      // So, either a table or regions op. If latter, check perms ons table.
359      TableName tn = tableName != null? tableName: regionInfos[0].getTable();
360      requireTablePermission(user, reason, tn, null, null,
361          Action.ADMIN, Action.CREATE);
362    } else {
363      throw new DoNotRetryIOException("Invalid lock level when requesting permissions.");
364    }
365  }
366
367  public static void logResult(AuthResult result) {
368    if (AUDITLOG.isTraceEnabled()) {
369      AUDITLOG.trace(
370        "Access {} for user {}; reason: {}; remote address: {}; request: {}; context: {}",
371        (result.isAllowed() ? "allowed" : "denied"),
372        (result.getUser() != null ? result.getUser().getShortName() : "UNKNOWN"),
373        result.getReason(), RpcServer.getRemoteAddress().map(InetAddress::toString).orElse(""),
374        result.getRequest(), result.toContextString());
375    }
376  }
377
378  /*
379   * Validate the hasPermission operation caller with the filter user. Self check doesn't require
380   * any privilege but for others caller must have ADMIN privilege.
381   */
382  public User validateCallerWithFilterUser(User caller, TablePermission tPerm, String inputUserName)
383      throws IOException {
384    User filterUser = null;
385    if (!caller.getShortName().equals(inputUserName)) {
386      // User should have admin privilege if checking permission for other users
387      requirePermission(caller, "hasPermission", tPerm.getTableName(), tPerm.getFamily(),
388        tPerm.getQualifier(), inputUserName, Action.ADMIN);
389      // Initialize user instance for the input user name
390      List<String> groups = getUserGroups(inputUserName);
391      filterUser = new InputUser(inputUserName, groups.toArray(new String[groups.size()]));
392    } else {
393      // User don't need ADMIN privilege for self check.
394      // Setting action as null in AuthResult to display empty action in audit log
395      AuthResult result = AuthResult.allow("hasPermission", "Self user validation allowed", caller,
396        null, tPerm.getTableName(), tPerm.getFamily(), tPerm.getQualifier());
397      logResult(result);
398      filterUser = caller;
399    }
400    return filterUser;
401  }
402
403  /**
404   * A temporary user class to instantiate User instance based on the name and groups.
405   */
406  public static class InputUser extends User {
407    private String name;
408    private String shortName = null;
409    private String[] groups;
410
411    public InputUser(String name, String[] groups) {
412      this.name = name;
413      this.groups = groups;
414    }
415
416    @Override
417    public String getShortName() {
418      if (this.shortName == null) {
419        try {
420          this.shortName = new HadoopKerberosName(this.name).getShortName();
421        } catch (IOException ioe) {
422          throw new IllegalArgumentException(
423              "Illegal principal name " + this.name + ": " + ioe.toString(), ioe);
424        }
425      }
426      return shortName;
427    }
428
429    @Override
430    public String getName() {
431      return this.name;
432    }
433
434    @Override
435    public String[] getGroupNames() {
436      return this.groups;
437    }
438
439    @Override
440    public <T> T runAs(PrivilegedAction<T> action) {
441      throw new UnsupportedOperationException(
442          "Method not supported, this class has limited implementation");
443    }
444
445    @Override
446    public <T> T runAs(PrivilegedExceptionAction<T> action)
447        throws IOException, InterruptedException {
448      throw new UnsupportedOperationException(
449          "Method not supported, this class has limited implementation");
450    }
451
452    @Override
453    public String toString() {
454      return this.name;
455    }
456  }
457
458  /*
459   * Initialize the group service.
460   */
461  private void initGroupService(Configuration conf) {
462    if (groupService == null) {
463      if (conf.getBoolean(User.TestingGroups.TEST_CONF, false)) {
464        UserProvider.setGroups(new User.TestingGroups(UserProvider.getGroups()));
465        groupService = UserProvider.getGroups();
466      } else {
467        groupService = Groups.getUserToGroupsMappingService(conf);
468      }
469    }
470  }
471
472  /**
473   * Retrieve the groups of the given user.
474   * @param user User name
475   * @return Groups
476   */
477  public static List<String> getUserGroups(String user) {
478    try {
479      return groupService.getGroups(user);
480    } catch (IOException e) {
481      LOG.error("Error occurred while retrieving group for " + user, e);
482      return new ArrayList<>();
483    }
484  }
485
486  /**
487   * Authorizes that if the current user has the given permissions.
488   * @param user Active user to which authorization checks should be applied
489   * @param request Request type
490   * @param permission Actions being requested
491   * @return True if the user has the specific permission
492   */
493  public boolean hasUserPermission(User user, String request, Permission permission) {
494    if (permission instanceof TablePermission) {
495      TablePermission tPerm = (TablePermission) permission;
496      for (Permission.Action action : permission.getActions()) {
497        AuthResult authResult = permissionGranted(request, user, action, tPerm.getTableName(),
498          tPerm.getFamily(), tPerm.getQualifier());
499        AccessChecker.logResult(authResult);
500        if (!authResult.isAllowed()) {
501          return false;
502        }
503      }
504    } else if (permission instanceof NamespacePermission) {
505      NamespacePermission nsPerm = (NamespacePermission) permission;
506      AuthResult authResult;
507      for (Action action : nsPerm.getActions()) {
508        if (getAuthManager().authorizeUserNamespace(user, nsPerm.getNamespace(), action)) {
509          authResult =
510              AuthResult.allow(request, "Namespace action allowed", user, action, null, null);
511        } else {
512          authResult =
513              AuthResult.deny(request, "Namespace action denied", user, action, null, null);
514        }
515        AccessChecker.logResult(authResult);
516        if (!authResult.isAllowed()) {
517          return false;
518        }
519      }
520    } else {
521      AuthResult authResult;
522      for (Permission.Action action : permission.getActions()) {
523        if (getAuthManager().authorizeUserGlobal(user, action)) {
524          authResult = AuthResult.allow(request, "Global action allowed", user, action, null, null);
525        } else {
526          authResult = AuthResult.deny(request, "Global action denied", user, action, null, null);
527        }
528        AccessChecker.logResult(authResult);
529        if (!authResult.isAllowed()) {
530          return false;
531        }
532      }
533    }
534    return true;
535  }
536
537  private AuthResult permissionGranted(String request, User user, Action permRequest,
538      TableName tableName, byte[] family, byte[] qualifier) {
539    Map<byte[], ? extends Collection<byte[]>> map = makeFamilyMap(family, qualifier);
540    return permissionGranted(request, user, permRequest, tableName, map);
541  }
542
543  /**
544   * Check the current user for authorization to perform a specific action against the given set of
545   * row data.
546   * <p>
547   * Note: Ordering of the authorization checks has been carefully optimized to short-circuit the
548   * most common requests and minimize the amount of processing required.
549   * </p>
550   * @param request User request
551   * @param user User name
552   * @param permRequest the action being requested
553   * @param tableName Table name
554   * @param families the map of column families to qualifiers present in the request
555   * @return an authorization result
556   */
557  public AuthResult permissionGranted(String request, User user, Action permRequest,
558      TableName tableName, Map<byte[], ? extends Collection<?>> families) {
559    // 1. All users need read access to hbase:meta table.
560    // this is a very common operation, so deal with it quickly.
561    if (TableName.META_TABLE_NAME.equals(tableName)) {
562      if (permRequest == Action.READ) {
563        return AuthResult.allow(request, "All users allowed", user, permRequest, tableName,
564          families);
565      }
566    }
567
568    if (user == null) {
569      return AuthResult.deny(request, "No user associated with request!", null, permRequest,
570        tableName, families);
571    }
572
573    // 2. check for the table-level, if successful we can short-circuit
574    if (getAuthManager().authorizeUserTable(user, tableName, permRequest)) {
575      return AuthResult.allow(request, "Table permission granted", user, permRequest, tableName,
576        families);
577    }
578
579    // 3. check permissions against the requested families
580    if (families != null && families.size() > 0) {
581      // all families must pass
582      for (Map.Entry<byte[], ? extends Collection<?>> family : families.entrySet()) {
583        // a) check for family level access
584        if (getAuthManager().authorizeUserTable(user, tableName, family.getKey(), permRequest)) {
585          continue; // family-level permission overrides per-qualifier
586        }
587
588        // b) qualifier level access can still succeed
589        if ((family.getValue() != null) && (family.getValue().size() > 0)) {
590          if (family.getValue() instanceof Set) {
591            // for each qualifier of the family
592            Set<byte[]> familySet = (Set<byte[]>) family.getValue();
593            for (byte[] qualifier : familySet) {
594              if (!getAuthManager().authorizeUserTable(user, tableName, family.getKey(), qualifier,
595                permRequest)) {
596                return AuthResult.deny(request, "Failed qualifier check", user, permRequest,
597                  tableName, makeFamilyMap(family.getKey(), qualifier));
598              }
599            }
600          } else if (family.getValue() instanceof List) { // List<Cell>
601            List<Cell> cellList = (List<Cell>) family.getValue();
602            for (Cell cell : cellList) {
603              if (!getAuthManager().authorizeUserTable(user, tableName, family.getKey(),
604                CellUtil.cloneQualifier(cell), permRequest)) {
605                return AuthResult.deny(request, "Failed qualifier check", user, permRequest,
606                  tableName, makeFamilyMap(family.getKey(), CellUtil.cloneQualifier(cell)));
607              }
608            }
609          }
610        } else {
611          // no qualifiers and family-level check already failed
612          return AuthResult.deny(request, "Failed family check", user, permRequest, tableName,
613            makeFamilyMap(family.getKey(), null));
614        }
615      }
616
617      // all family checks passed
618      return AuthResult.allow(request, "All family checks passed", user, permRequest, tableName,
619        families);
620    }
621
622    // 4. no families to check and table level access failed
623    return AuthResult.deny(request, "No families to check and table permission failed", user,
624      permRequest, tableName, families);
625  }
626
627  private Map<byte[], ? extends Collection<byte[]>> makeFamilyMap(byte[] family, byte[] qualifier) {
628    if (family == null) {
629      return null;
630    }
631
632    Map<byte[], Collection<byte[]>> familyMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
633    familyMap.put(family, qualifier != null ? ImmutableSet.of(qualifier) : null);
634    return familyMap;
635  }
636}