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