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.security.PrivilegedExceptionAction;
022import java.util.ArrayList;
023import java.util.Collection;
024import java.util.Collections;
025import java.util.HashMap;
026import java.util.Iterator;
027import java.util.List;
028import java.util.Map;
029import java.util.Map.Entry;
030import java.util.Optional;
031import java.util.Set;
032import java.util.TreeMap;
033import java.util.TreeSet;
034import java.util.stream.Collectors;
035import org.apache.hadoop.conf.Configuration;
036import org.apache.hadoop.hbase.ArrayBackedTag;
037import org.apache.hadoop.hbase.Cell;
038import org.apache.hadoop.hbase.CellScanner;
039import org.apache.hadoop.hbase.CellUtil;
040import org.apache.hadoop.hbase.CompareOperator;
041import org.apache.hadoop.hbase.CompoundConfiguration;
042import org.apache.hadoop.hbase.CoprocessorEnvironment;
043import org.apache.hadoop.hbase.DoNotRetryIOException;
044import org.apache.hadoop.hbase.HBaseInterfaceAudience;
045import org.apache.hadoop.hbase.HConstants;
046import org.apache.hadoop.hbase.KeyValue;
047import org.apache.hadoop.hbase.KeyValue.Type;
048import org.apache.hadoop.hbase.NamespaceDescriptor;
049import org.apache.hadoop.hbase.PrivateCellUtil;
050import org.apache.hadoop.hbase.ServerName;
051import org.apache.hadoop.hbase.TableName;
052import org.apache.hadoop.hbase.Tag;
053import org.apache.hadoop.hbase.client.Admin;
054import org.apache.hadoop.hbase.client.Append;
055import org.apache.hadoop.hbase.client.BalanceRequest;
056import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
057import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
058import org.apache.hadoop.hbase.client.Delete;
059import org.apache.hadoop.hbase.client.Durability;
060import org.apache.hadoop.hbase.client.Get;
061import org.apache.hadoop.hbase.client.Increment;
062import org.apache.hadoop.hbase.client.MasterSwitchType;
063import org.apache.hadoop.hbase.client.Mutation;
064import org.apache.hadoop.hbase.client.Put;
065import org.apache.hadoop.hbase.client.Query;
066import org.apache.hadoop.hbase.client.RegionInfo;
067import org.apache.hadoop.hbase.client.Result;
068import org.apache.hadoop.hbase.client.Scan;
069import org.apache.hadoop.hbase.client.SnapshotDescription;
070import org.apache.hadoop.hbase.client.Table;
071import org.apache.hadoop.hbase.client.TableDescriptor;
072import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
073import org.apache.hadoop.hbase.coprocessor.BulkLoadObserver;
074import org.apache.hadoop.hbase.coprocessor.CoprocessorException;
075import org.apache.hadoop.hbase.coprocessor.CoreCoprocessor;
076import org.apache.hadoop.hbase.coprocessor.EndpointObserver;
077import org.apache.hadoop.hbase.coprocessor.HasMasterServices;
078import org.apache.hadoop.hbase.coprocessor.HasRegionServerServices;
079import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor;
080import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
081import org.apache.hadoop.hbase.coprocessor.MasterObserver;
082import org.apache.hadoop.hbase.coprocessor.ObserverContext;
083import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
084import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
085import org.apache.hadoop.hbase.coprocessor.RegionObserver;
086import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessor;
087import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment;
088import org.apache.hadoop.hbase.coprocessor.RegionServerObserver;
089import org.apache.hadoop.hbase.filter.ByteArrayComparable;
090import org.apache.hadoop.hbase.filter.Filter;
091import org.apache.hadoop.hbase.filter.FilterList;
092import org.apache.hadoop.hbase.io.hfile.HFile;
093import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
094import org.apache.hadoop.hbase.ipc.RpcServer;
095import org.apache.hadoop.hbase.master.MasterServices;
096import org.apache.hadoop.hbase.net.Address;
097import org.apache.hadoop.hbase.quotas.GlobalQuotaSettings;
098import org.apache.hadoop.hbase.regionserver.BloomType;
099import org.apache.hadoop.hbase.regionserver.FlushLifeCycleTracker;
100import org.apache.hadoop.hbase.regionserver.InternalScanner;
101import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
102import org.apache.hadoop.hbase.regionserver.Region;
103import org.apache.hadoop.hbase.regionserver.RegionScanner;
104import org.apache.hadoop.hbase.regionserver.RegionServerServices;
105import org.apache.hadoop.hbase.regionserver.ScanType;
106import org.apache.hadoop.hbase.regionserver.ScannerContext;
107import org.apache.hadoop.hbase.regionserver.Store;
108import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
109import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
110import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
111import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
112import org.apache.hadoop.hbase.replication.SyncReplicationState;
113import org.apache.hadoop.hbase.security.AccessDeniedException;
114import org.apache.hadoop.hbase.security.Superusers;
115import org.apache.hadoop.hbase.security.User;
116import org.apache.hadoop.hbase.security.UserProvider;
117import org.apache.hadoop.hbase.security.access.Permission.Action;
118import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
119import org.apache.hadoop.hbase.util.ByteRange;
120import org.apache.hadoop.hbase.util.Bytes;
121import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
122import org.apache.hadoop.hbase.util.Pair;
123import org.apache.hadoop.hbase.util.SimpleMutableByteRange;
124import org.apache.hadoop.hbase.wal.WALEdit;
125import org.apache.yetus.audience.InterfaceAudience;
126import org.slf4j.Logger;
127import org.slf4j.LoggerFactory;
128
129import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
130import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSet;
131import org.apache.hbase.thirdparty.com.google.common.collect.ListMultimap;
132import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
133import org.apache.hbase.thirdparty.com.google.common.collect.MapMaker;
134import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
135import org.apache.hbase.thirdparty.com.google.protobuf.Message;
136import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
137import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
138import org.apache.hbase.thirdparty.com.google.protobuf.Service;
139
140import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
141import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
142import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos;
143import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.AccessControlService;
144import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.HasPermissionRequest;
145import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.HasPermissionResponse;
146
147/**
148 * Provides basic authorization checks for data access and administrative operations.
149 * <p>
150 * {@code AccessController} performs authorization checks for HBase operations based on:
151 * </p>
152 * <ul>
153 * <li>the identity of the user performing the operation</li>
154 * <li>the scope over which the operation is performed, in increasing specificity: global, table,
155 * column family, or qualifier</li>
156 * <li>the type of action being performed (as mapped to {@link Permission.Action} values)</li>
157 * </ul>
158 * <p>
159 * If the authorization check fails, an {@link AccessDeniedException} will be thrown for the
160 * operation.
161 * </p>
162 * <p>
163 * To perform authorization checks, {@code AccessController} relies on the RpcServerEngine being
164 * loaded to provide the user identities for remote requests.
165 * </p>
166 * <p>
167 * The access control lists used for authorization can be manipulated via the exposed
168 * {@link AccessControlService} Interface implementation, and the associated {@code grant},
169 * {@code revoke}, and {@code user_permission} HBase shell commands.
170 * </p>
171 */
172@CoreCoprocessor
173@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
174public class AccessController implements MasterCoprocessor, RegionCoprocessor,
175  RegionServerCoprocessor, AccessControlService.Interface, MasterObserver, RegionObserver,
176  RegionServerObserver, EndpointObserver, BulkLoadObserver {
177  // TODO: encapsulate observer functions into separate class/sub-class.
178
179  private static final Logger LOG = LoggerFactory.getLogger(AccessController.class);
180
181  private static final Logger AUDITLOG =
182    LoggerFactory.getLogger("SecurityLogger." + AccessController.class.getName());
183  private static final String CHECK_COVERING_PERM = "check_covering_perm";
184  private static final String TAG_CHECK_PASSED = "tag_check_passed";
185  private static final byte[] TRUE = Bytes.toBytes(true);
186
187  private AccessChecker accessChecker;
188  private ZKPermissionWatcher zkPermissionWatcher;
189
190  /** flags if we are running on a region of the _acl_ table */
191  private boolean aclRegion = false;
192
193  /**
194   * defined only for Endpoint implementation, so it can have way to access region services
195   */
196  private RegionCoprocessorEnvironment regionEnv;
197
198  /** Mapping of scanner instances to the user who created them */
199  private Map<InternalScanner, String> scannerOwners = new MapMaker().weakKeys().makeMap();
200
201  private Map<TableName, List<UserPermission>> tableAcls;
202
203  /** Provider for mapping principal names to Users */
204  private UserProvider userProvider;
205
206  /**
207   * if we are active, usually false, only true if "hbase.security.authorization" has been set to
208   * true in site configuration
209   */
210  private boolean authorizationEnabled;
211
212  /** if we are able to support cell ACLs */
213  private boolean cellFeaturesEnabled;
214
215  /** if we should check EXEC permissions */
216  private boolean shouldCheckExecPermission;
217
218  /**
219   * if we should terminate access checks early as soon as table or CF grants allow access; pre-0.98
220   * compatible behavior
221   */
222  private boolean compatibleEarlyTermination;
223
224  /** if we have been successfully initialized */
225  private volatile boolean initialized = false;
226
227  /** if the ACL table is available, only relevant in the master */
228  private volatile boolean aclTabAvailable = false;
229
230  public static boolean isCellAuthorizationSupported(Configuration conf) {
231    return AccessChecker.isAuthorizationSupported(conf)
232      && (HFile.getFormatVersion(conf) >= HFile.MIN_FORMAT_VERSION_WITH_TAGS);
233  }
234
235  public Region getRegion() {
236    return regionEnv != null ? regionEnv.getRegion() : null;
237  }
238
239  public AuthManager getAuthManager() {
240    return accessChecker.getAuthManager();
241  }
242
243  private void initialize(RegionCoprocessorEnvironment e) throws IOException {
244    final Region region = e.getRegion();
245    Configuration conf = e.getConfiguration();
246    Map<byte[], ListMultimap<String, UserPermission>> tables = PermissionStorage.loadAll(region);
247    // For each table, write out the table's permissions to the respective
248    // znode for that table.
249    for (Map.Entry<byte[], ListMultimap<String, UserPermission>> t : tables.entrySet()) {
250      byte[] entry = t.getKey();
251      ListMultimap<String, UserPermission> perms = t.getValue();
252      byte[] serialized = PermissionStorage.writePermissionsAsBytes(perms, conf);
253      zkPermissionWatcher.writeToZookeeper(entry, serialized);
254    }
255    initialized = true;
256  }
257
258  /**
259   * Writes all table ACLs for the tables in the given Map up into ZooKeeper znodes. This is called
260   * to synchronize ACL changes following {@code _acl_} table updates.
261   */
262  private void updateACL(RegionCoprocessorEnvironment e, final Map<byte[], List<Cell>> familyMap) {
263    Set<byte[]> entries = new TreeSet<>(Bytes.BYTES_RAWCOMPARATOR);
264    for (Map.Entry<byte[], List<Cell>> f : familyMap.entrySet()) {
265      List<Cell> cells = f.getValue();
266      for (Cell cell : cells) {
267        if (CellUtil.matchingFamily(cell, PermissionStorage.ACL_LIST_FAMILY)) {
268          entries.add(CellUtil.cloneRow(cell));
269        }
270      }
271    }
272    Configuration conf = regionEnv.getConfiguration();
273    byte[] currentEntry = null;
274    // TODO: Here we are already on the ACL region. (And it is single
275    // region) We can even just get the region from the env and do get
276    // directly. The short circuit connection would avoid the RPC overhead
277    // so no socket communication, req write/read .. But we have the PB
278    // to and fro conversion overhead. get req is converted to PB req
279    // and results are converted to PB results 1st and then to POJOs
280    // again. We could have avoided such at least in ACL table context..
281    try (Table t = e.getConnection().getTable(PermissionStorage.ACL_TABLE_NAME)) {
282      for (byte[] entry : entries) {
283        currentEntry = entry;
284        ListMultimap<String, UserPermission> perms =
285          PermissionStorage.getPermissions(conf, entry, t, null, null, null, false);
286        byte[] serialized = PermissionStorage.writePermissionsAsBytes(perms, conf);
287        zkPermissionWatcher.writeToZookeeper(entry, serialized);
288      }
289    } catch (IOException ex) {
290      LOG.error("Failed updating permissions mirror for '"
291        + (currentEntry == null ? "null" : Bytes.toString(currentEntry)) + "'", ex);
292    }
293  }
294
295  /**
296   * Check the current user for authorization to perform a specific action against the given set of
297   * row data.
298   * @param opType   the operation type
299   * @param user     the user
300   * @param e        the coprocessor environment
301   * @param families the map of column families to qualifiers present in the request
302   * @param actions  the desired actions
303   * @return an authorization result
304   */
305  private AuthResult permissionGranted(OpType opType, User user, RegionCoprocessorEnvironment e,
306    Map<byte[], ? extends Collection<?>> families, Action... actions) {
307    AuthResult result = null;
308    for (Action action : actions) {
309      result = accessChecker.permissionGranted(opType.toString(), user, action,
310        e.getRegion().getRegionInfo().getTable(), families);
311      if (!result.isAllowed()) {
312        return result;
313      }
314    }
315    return result;
316  }
317
318  public void requireAccess(ObserverContext<?> ctx, String request, TableName tableName,
319    Action... permissions) throws IOException {
320    accessChecker.requireAccess(getActiveUser(ctx), request, tableName, permissions);
321  }
322
323  public void requirePermission(ObserverContext<?> ctx, String request, Action perm)
324    throws IOException {
325    accessChecker.requirePermission(getActiveUser(ctx), request, null, perm);
326  }
327
328  public void requireGlobalPermission(ObserverContext<?> ctx, String request, Action perm,
329    TableName tableName, Map<byte[], ? extends Collection<byte[]>> familyMap) throws IOException {
330    accessChecker.requireGlobalPermission(getActiveUser(ctx), request, perm, tableName, familyMap,
331      null);
332  }
333
334  public void requireGlobalPermission(ObserverContext<?> ctx, String request, Action perm,
335    String namespace) throws IOException {
336    accessChecker.requireGlobalPermission(getActiveUser(ctx), request, perm, namespace);
337  }
338
339  public void requireNamespacePermission(ObserverContext<?> ctx, String request, String namespace,
340    Action... permissions) throws IOException {
341    accessChecker.requireNamespacePermission(getActiveUser(ctx), request, namespace, null,
342      permissions);
343  }
344
345  public void requireNamespacePermission(ObserverContext<?> ctx, String request, String namespace,
346    TableName tableName, Map<byte[], ? extends Collection<byte[]>> familyMap, Action... permissions)
347    throws IOException {
348    accessChecker.requireNamespacePermission(getActiveUser(ctx), request, namespace, tableName,
349      familyMap, permissions);
350  }
351
352  public void requirePermission(ObserverContext<?> ctx, String request, TableName tableName,
353    byte[] family, byte[] qualifier, Action... permissions) throws IOException {
354    accessChecker.requirePermission(getActiveUser(ctx), request, tableName, family, qualifier, null,
355      permissions);
356  }
357
358  public void requireTablePermission(ObserverContext<?> ctx, String request, TableName tableName,
359    byte[] family, byte[] qualifier, Action... permissions) throws IOException {
360    accessChecker.requireTablePermission(getActiveUser(ctx), request, tableName, family, qualifier,
361      permissions);
362  }
363
364  public void checkLockPermissions(ObserverContext<?> ctx, String namespace, TableName tableName,
365    RegionInfo[] regionInfos, String reason) throws IOException {
366    accessChecker.checkLockPermissions(getActiveUser(ctx), namespace, tableName, regionInfos,
367      reason);
368  }
369
370  /**
371   * Returns <code>true</code> if the current user is allowed the given action over at least one of
372   * the column qualifiers in the given column families.
373   */
374  private boolean hasFamilyQualifierPermission(User user, Action perm,
375    RegionCoprocessorEnvironment env, Map<byte[], ? extends Collection<byte[]>> familyMap)
376    throws IOException {
377    RegionInfo hri = env.getRegion().getRegionInfo();
378    TableName tableName = hri.getTable();
379
380    if (user == null) {
381      return false;
382    }
383
384    if (familyMap != null && familyMap.size() > 0) {
385      // at least one family must be allowed
386      for (Map.Entry<byte[], ? extends Collection<byte[]>> family : familyMap.entrySet()) {
387        if (family.getValue() != null && !family.getValue().isEmpty()) {
388          for (byte[] qualifier : family.getValue()) {
389            if (
390              getAuthManager().authorizeUserTable(user, tableName, family.getKey(), qualifier, perm)
391            ) {
392              return true;
393            }
394          }
395        } else {
396          if (getAuthManager().authorizeUserFamily(user, tableName, family.getKey(), perm)) {
397            return true;
398          }
399        }
400      }
401    } else if (LOG.isDebugEnabled()) {
402      LOG.debug("Empty family map passed for permission check");
403    }
404
405    return false;
406  }
407
408  private enum OpType {
409    GET("get"),
410    EXISTS("exists"),
411    SCAN("scan"),
412    PUT("put"),
413    DELETE("delete"),
414    CHECK_AND_PUT("checkAndPut"),
415    CHECK_AND_DELETE("checkAndDelete"),
416    APPEND("append"),
417    INCREMENT("increment");
418
419    private String type;
420
421    private OpType(String type) {
422      this.type = type;
423    }
424
425    @Override
426    public String toString() {
427      return type;
428    }
429  }
430
431  /**
432   * Determine if cell ACLs covered by the operation grant access. This is expensive.
433   * @return false if cell ACLs failed to grant access, true otherwise n
434   */
435  private boolean checkCoveringPermission(User user, OpType request, RegionCoprocessorEnvironment e,
436    byte[] row, Map<byte[], ? extends Collection<?>> familyMap, long opTs, Action... actions)
437    throws IOException {
438    if (!cellFeaturesEnabled) {
439      return false;
440    }
441    long cellGrants = 0;
442    long latestCellTs = 0;
443    Get get = new Get(row);
444    // Only in case of Put/Delete op, consider TS within cell (if set for individual cells).
445    // When every cell, within a Mutation, can be linked with diff TS we can not rely on only one
446    // version. We have to get every cell version and check its TS against the TS asked for in
447    // Mutation and skip those Cells which is outside this Mutation TS.In case of Put, we have to
448    // consider only one such passing cell. In case of Delete we have to consider all the cell
449    // versions under this passing version. When Delete Mutation contains columns which are a
450    // version delete just consider only one version for those column cells.
451    boolean considerCellTs = (request == OpType.PUT || request == OpType.DELETE);
452    if (considerCellTs) {
453      get.readAllVersions();
454    } else {
455      get.readVersions(1);
456    }
457    boolean diffCellTsFromOpTs = false;
458    for (Map.Entry<byte[], ? extends Collection<?>> entry : familyMap.entrySet()) {
459      byte[] col = entry.getKey();
460      // TODO: HBASE-7114 could possibly unify the collection type in family
461      // maps so we would not need to do this
462      if (entry.getValue() instanceof Set) {
463        Set<byte[]> set = (Set<byte[]>) entry.getValue();
464        if (set == null || set.isEmpty()) {
465          get.addFamily(col);
466        } else {
467          for (byte[] qual : set) {
468            get.addColumn(col, qual);
469          }
470        }
471      } else if (entry.getValue() instanceof List) {
472        List<Cell> list = (List<Cell>) entry.getValue();
473        if (list == null || list.isEmpty()) {
474          get.addFamily(col);
475        } else {
476          // In case of family delete, a Cell will be added into the list with Qualifier as null.
477          for (Cell cell : list) {
478            if (
479              cell.getQualifierLength() == 0 && (cell.getTypeByte() == Type.DeleteFamily.getCode()
480                || cell.getTypeByte() == Type.DeleteFamilyVersion.getCode())
481            ) {
482              get.addFamily(col);
483            } else {
484              get.addColumn(col, CellUtil.cloneQualifier(cell));
485            }
486            if (considerCellTs) {
487              long cellTs = cell.getTimestamp();
488              latestCellTs = Math.max(latestCellTs, cellTs);
489              diffCellTsFromOpTs = diffCellTsFromOpTs || (opTs != cellTs);
490            }
491          }
492        }
493      } else if (entry.getValue() == null) {
494        get.addFamily(col);
495      } else {
496        throw new RuntimeException(
497          "Unhandled collection type " + entry.getValue().getClass().getName());
498      }
499    }
500    // We want to avoid looking into the future. So, if the cells of the
501    // operation specify a timestamp, or the operation itself specifies a
502    // timestamp, then we use the maximum ts found. Otherwise, we bound
503    // the Get to the current server time. We add 1 to the timerange since
504    // the upper bound of a timerange is exclusive yet we need to examine
505    // any cells found there inclusively.
506    long latestTs = Math.max(opTs, latestCellTs);
507    if (latestTs == 0 || latestTs == HConstants.LATEST_TIMESTAMP) {
508      latestTs = EnvironmentEdgeManager.currentTime();
509    }
510    get.setTimeRange(0, latestTs + 1);
511    // In case of Put operation we set to read all versions. This was done to consider the case
512    // where columns are added with TS other than the Mutation TS. But normally this wont be the
513    // case with Put. There no need to get all versions but get latest version only.
514    if (!diffCellTsFromOpTs && request == OpType.PUT) {
515      get.readVersions(1);
516    }
517    if (LOG.isTraceEnabled()) {
518      LOG.trace("Scanning for cells with " + get);
519    }
520    // This Map is identical to familyMap. The key is a BR rather than byte[].
521    // It will be easy to do gets over this new Map as we can create get keys over the Cell cf by
522    // new SimpleByteRange(cell.familyArray, cell.familyOffset, cell.familyLen)
523    Map<ByteRange, List<Cell>> familyMap1 = new HashMap<>();
524    for (Entry<byte[], ? extends Collection<?>> entry : familyMap.entrySet()) {
525      if (entry.getValue() instanceof List) {
526        familyMap1.put(new SimpleMutableByteRange(entry.getKey()), (List<Cell>) entry.getValue());
527      }
528    }
529    RegionScanner scanner = getRegion(e).getScanner(new Scan(get));
530    List<Cell> cells = Lists.newArrayList();
531    Cell prevCell = null;
532    ByteRange curFam = new SimpleMutableByteRange();
533    boolean curColAllVersions = (request == OpType.DELETE);
534    long curColCheckTs = opTs;
535    boolean foundColumn = false;
536    try {
537      boolean more = false;
538      ScannerContext scannerContext = ScannerContext.newBuilder().setBatchLimit(1).build();
539
540      do {
541        cells.clear();
542        // scan with limit as 1 to hold down memory use on wide rows
543        more = scanner.next(cells, scannerContext);
544        for (Cell cell : cells) {
545          if (LOG.isTraceEnabled()) {
546            LOG.trace("Found cell " + cell);
547          }
548          boolean colChange = prevCell == null || !CellUtil.matchingColumn(prevCell, cell);
549          if (colChange) foundColumn = false;
550          prevCell = cell;
551          if (!curColAllVersions && foundColumn) {
552            continue;
553          }
554          if (colChange && considerCellTs) {
555            curFam.set(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength());
556            List<Cell> cols = familyMap1.get(curFam);
557            for (Cell col : cols) {
558              // null/empty qualifier is used to denote a Family delete. The TS and delete type
559              // associated with this is applicable for all columns within the family. That is
560              // why the below (col.getQualifierLength() == 0) check.
561              if (
562                (col.getQualifierLength() == 0 && request == OpType.DELETE)
563                  || CellUtil.matchingQualifier(cell, col)
564              ) {
565                byte type = col.getTypeByte();
566                if (considerCellTs) {
567                  curColCheckTs = col.getTimestamp();
568                }
569                // For a Delete op we pass allVersions as true. When a Delete Mutation contains
570                // a version delete for a column no need to check all the covering cells within
571                // that column. Check all versions when Type is DeleteColumn or DeleteFamily
572                // One version delete types are Delete/DeleteFamilyVersion
573                curColAllVersions = (KeyValue.Type.DeleteColumn.getCode() == type)
574                  || (KeyValue.Type.DeleteFamily.getCode() == type);
575                break;
576              }
577            }
578          }
579          if (cell.getTimestamp() > curColCheckTs) {
580            // Just ignore this cell. This is not a covering cell.
581            continue;
582          }
583          foundColumn = true;
584          for (Action action : actions) {
585            // Are there permissions for this user for the cell?
586            if (!getAuthManager().authorizeCell(user, getTableName(e), cell, action)) {
587              // We can stop if the cell ACL denies access
588              return false;
589            }
590          }
591          cellGrants++;
592        }
593      } while (more);
594    } catch (AccessDeniedException ex) {
595      throw ex;
596    } catch (IOException ex) {
597      LOG.error("Exception while getting cells to calculate covering permission", ex);
598    } finally {
599      scanner.close();
600    }
601    // We should not authorize unless we have found one or more cell ACLs that
602    // grant access. This code is used to check for additional permissions
603    // after no table or CF grants are found.
604    return cellGrants > 0;
605  }
606
607  private static void addCellPermissions(final byte[] perms, Map<byte[], List<Cell>> familyMap) {
608    // Iterate over the entries in the familyMap, replacing the cells therein
609    // with new cells including the ACL data
610    for (Map.Entry<byte[], List<Cell>> e : familyMap.entrySet()) {
611      List<Cell> newCells = Lists.newArrayList();
612      for (Cell cell : e.getValue()) {
613        // Prepend the supplied perms in a new ACL tag to an update list of tags for the cell
614        List<Tag> tags = new ArrayList<>();
615        tags.add(new ArrayBackedTag(PermissionStorage.ACL_TAG_TYPE, perms));
616        Iterator<Tag> tagIterator = PrivateCellUtil.tagsIterator(cell);
617        while (tagIterator.hasNext()) {
618          tags.add(tagIterator.next());
619        }
620        newCells.add(PrivateCellUtil.createCell(cell, tags));
621      }
622      // This is supposed to be safe, won't CME
623      e.setValue(newCells);
624    }
625  }
626
627  // Checks whether incoming cells contain any tag with type as ACL_TAG_TYPE. This tag
628  // type is reserved and should not be explicitly set by user.
629  private void checkForReservedTagPresence(User user, Mutation m) throws IOException {
630    // No need to check if we're not going to throw
631    if (!authorizationEnabled) {
632      m.setAttribute(TAG_CHECK_PASSED, TRUE);
633      return;
634    }
635    // Superusers are allowed to store cells unconditionally.
636    if (Superusers.isSuperUser(user)) {
637      m.setAttribute(TAG_CHECK_PASSED, TRUE);
638      return;
639    }
640    // We already checked (prePut vs preBatchMutation)
641    if (m.getAttribute(TAG_CHECK_PASSED) != null) {
642      return;
643    }
644    for (CellScanner cellScanner = m.cellScanner(); cellScanner.advance();) {
645      Iterator<Tag> tagsItr = PrivateCellUtil.tagsIterator(cellScanner.current());
646      while (tagsItr.hasNext()) {
647        if (tagsItr.next().getType() == PermissionStorage.ACL_TAG_TYPE) {
648          throw new AccessDeniedException("Mutation contains cell with reserved type tag");
649        }
650      }
651    }
652    m.setAttribute(TAG_CHECK_PASSED, TRUE);
653  }
654
655  /* ---- MasterObserver implementation ---- */
656  @Override
657  public void start(CoprocessorEnvironment env) throws IOException {
658    CompoundConfiguration conf = new CompoundConfiguration();
659    conf.add(env.getConfiguration());
660
661    authorizationEnabled = AccessChecker.isAuthorizationSupported(conf);
662    if (!authorizationEnabled) {
663      LOG.warn("AccessController has been loaded with authorization checks DISABLED!");
664    }
665
666    shouldCheckExecPermission = conf.getBoolean(AccessControlConstants.EXEC_PERMISSION_CHECKS_KEY,
667      AccessControlConstants.DEFAULT_EXEC_PERMISSION_CHECKS);
668
669    cellFeaturesEnabled = (HFile.getFormatVersion(conf) >= HFile.MIN_FORMAT_VERSION_WITH_TAGS);
670    if (!cellFeaturesEnabled) {
671      LOG.info("A minimum HFile version of " + HFile.MIN_FORMAT_VERSION_WITH_TAGS
672        + " is required to persist cell ACLs. Consider setting " + HFile.FORMAT_VERSION_KEY
673        + " accordingly.");
674    }
675
676    if (env instanceof MasterCoprocessorEnvironment) {
677      // if running on HMaster
678      MasterCoprocessorEnvironment mEnv = (MasterCoprocessorEnvironment) env;
679      if (mEnv instanceof HasMasterServices) {
680        MasterServices masterServices = ((HasMasterServices) mEnv).getMasterServices();
681        zkPermissionWatcher = masterServices.getZKPermissionWatcher();
682        accessChecker = masterServices.getAccessChecker();
683      }
684    } else if (env instanceof RegionServerCoprocessorEnvironment) {
685      RegionServerCoprocessorEnvironment rsEnv = (RegionServerCoprocessorEnvironment) env;
686      if (rsEnv instanceof HasRegionServerServices) {
687        RegionServerServices rsServices =
688          ((HasRegionServerServices) rsEnv).getRegionServerServices();
689        zkPermissionWatcher = rsServices.getZKPermissionWatcher();
690        accessChecker = rsServices.getAccessChecker();
691      }
692    } else if (env instanceof RegionCoprocessorEnvironment) {
693      // if running at region
694      regionEnv = (RegionCoprocessorEnvironment) env;
695      conf.addBytesMap(regionEnv.getRegion().getTableDescriptor().getValues());
696      compatibleEarlyTermination = conf.getBoolean(AccessControlConstants.CF_ATTRIBUTE_EARLY_OUT,
697        AccessControlConstants.DEFAULT_ATTRIBUTE_EARLY_OUT);
698      if (regionEnv instanceof HasRegionServerServices) {
699        RegionServerServices rsServices =
700          ((HasRegionServerServices) regionEnv).getRegionServerServices();
701        zkPermissionWatcher = rsServices.getZKPermissionWatcher();
702        accessChecker = rsServices.getAccessChecker();
703      }
704    }
705
706    Preconditions.checkState(zkPermissionWatcher != null, "ZKPermissionWatcher is null");
707    Preconditions.checkState(accessChecker != null, "AccessChecker is null");
708
709    // set the user-provider.
710    this.userProvider = UserProvider.instantiate(env.getConfiguration());
711    tableAcls = new MapMaker().weakValues().makeMap();
712  }
713
714  @Override
715  public void stop(CoprocessorEnvironment env) {
716  }
717
718  /*********************************** Observer/Service Getters ***********************************/
719  @Override
720  public Optional<RegionObserver> getRegionObserver() {
721    return Optional.of(this);
722  }
723
724  @Override
725  public Optional<MasterObserver> getMasterObserver() {
726    return Optional.of(this);
727  }
728
729  @Override
730  public Optional<EndpointObserver> getEndpointObserver() {
731    return Optional.of(this);
732  }
733
734  @Override
735  public Optional<BulkLoadObserver> getBulkLoadObserver() {
736    return Optional.of(this);
737  }
738
739  @Override
740  public Optional<RegionServerObserver> getRegionServerObserver() {
741    return Optional.of(this);
742  }
743
744  @Override
745  public Iterable<Service> getServices() {
746    return Collections
747      .singleton(AccessControlProtos.AccessControlService.newReflectiveService(this));
748  }
749
750  /*********************************** Observer implementations ***********************************/
751
752  @Override
753  public void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> c, TableDescriptor desc,
754    RegionInfo[] regions) throws IOException {
755    Set<byte[]> families = desc.getColumnFamilyNames();
756    Map<byte[], Set<byte[]>> familyMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
757    for (byte[] family : families) {
758      familyMap.put(family, null);
759    }
760    requireNamespacePermission(c, "createTable", desc.getTableName().getNamespaceAsString(),
761      desc.getTableName(), familyMap, Action.ADMIN, Action.CREATE);
762  }
763
764  @Override
765  public void postCompletedCreateTableAction(final ObserverContext<MasterCoprocessorEnvironment> c,
766    final TableDescriptor desc, final RegionInfo[] regions) throws IOException {
767    // When AC is used, it should be configured as the 1st CP.
768    // In Master, the table operations like create, are handled by a Thread pool but the max size
769    // for this pool is 1. So if multiple CPs create tables on startup, these creations will happen
770    // sequentially only.
771    // Related code in HMaster#startServiceThreads
772    // {code}
773    // // We depend on there being only one instance of this executor running
774    // // at a time. To do concurrency, would need fencing of enable/disable of
775    // // tables.
776    // this.service.startExecutorService(ExecutorType.MASTER_TABLE_OPERATIONS, 1);
777    // {code}
778    // In future if we change this pool to have more threads, then there is a chance for thread,
779    // creating acl table, getting delayed and by that time another table creation got over and
780    // this hook is getting called. In such a case, we will need a wait logic here which will
781    // wait till the acl table is created.
782    if (PermissionStorage.isAclTable(desc)) {
783      this.aclTabAvailable = true;
784    } else {
785      if (!aclTabAvailable) {
786        LOG.warn("Not adding owner permission for table " + desc.getTableName() + ". "
787          + PermissionStorage.ACL_TABLE_NAME + " is not yet created. " + getClass().getSimpleName()
788          + " should be configured as the first Coprocessor");
789      } else {
790        String owner = getActiveUser(c).getShortName();
791        final UserPermission userPermission = new UserPermission(owner,
792          Permission.newBuilder(desc.getTableName()).withActions(Action.values()).build());
793        // switch to the real hbase master user for doing the RPC on the ACL table
794        User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
795          @Override
796          public Void run() throws Exception {
797            try (Table table =
798              c.getEnvironment().getConnection().getTable(PermissionStorage.ACL_TABLE_NAME)) {
799              PermissionStorage.addUserPermission(c.getEnvironment().getConfiguration(),
800                userPermission, table);
801            }
802            return null;
803          }
804        });
805      }
806    }
807  }
808
809  @Override
810  public void preDeleteTable(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName)
811    throws IOException {
812    requirePermission(c, "deleteTable", tableName, null, null, Action.ADMIN, Action.CREATE);
813  }
814
815  @Override
816  public void postDeleteTable(ObserverContext<MasterCoprocessorEnvironment> c,
817    final TableName tableName) throws IOException {
818    final Configuration conf = c.getEnvironment().getConfiguration();
819    User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
820      @Override
821      public Void run() throws Exception {
822        try (Table table =
823          c.getEnvironment().getConnection().getTable(PermissionStorage.ACL_TABLE_NAME)) {
824          PermissionStorage.removeTablePermissions(conf, tableName, table);
825        }
826        return null;
827      }
828    });
829    zkPermissionWatcher.deleteTableACLNode(tableName);
830  }
831
832  @Override
833  public void preTruncateTable(ObserverContext<MasterCoprocessorEnvironment> c,
834    final TableName tableName) throws IOException {
835    requirePermission(c, "truncateTable", tableName, null, null, Action.ADMIN, Action.CREATE);
836
837    final Configuration conf = c.getEnvironment().getConfiguration();
838    User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
839      @Override
840      public Void run() throws Exception {
841        List<UserPermission> acls =
842          PermissionStorage.getUserTablePermissions(conf, tableName, null, null, null, false);
843        if (acls != null) {
844          tableAcls.put(tableName, acls);
845        }
846        return null;
847      }
848    });
849  }
850
851  @Override
852  public void postTruncateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
853    final TableName tableName) throws IOException {
854    final Configuration conf = ctx.getEnvironment().getConfiguration();
855    User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
856      @Override
857      public Void run() throws Exception {
858        List<UserPermission> perms = tableAcls.get(tableName);
859        if (perms != null) {
860          for (UserPermission perm : perms) {
861            try (Table table =
862              ctx.getEnvironment().getConnection().getTable(PermissionStorage.ACL_TABLE_NAME)) {
863              PermissionStorage.addUserPermission(conf, perm, table);
864            }
865          }
866        }
867        tableAcls.remove(tableName);
868        return null;
869      }
870    });
871  }
872
873  @Override
874  public TableDescriptor preModifyTable(ObserverContext<MasterCoprocessorEnvironment> c,
875    TableName tableName, TableDescriptor currentDesc, TableDescriptor newDesc) throws IOException {
876    // TODO: potentially check if this is a add/modify/delete column operation
877    requirePermission(c, "modifyTable", tableName, null, null, Action.ADMIN, Action.CREATE);
878    return newDesc;
879  }
880
881  @Override
882  public String preModifyTableStoreFileTracker(ObserverContext<MasterCoprocessorEnvironment> c,
883    TableName tableName, String dstSFT) throws IOException {
884    requirePermission(c, "modifyTableStoreFileTracker", tableName, null, null, Action.ADMIN,
885      Action.CREATE);
886    return dstSFT;
887  }
888
889  @Override
890  public String preModifyColumnFamilyStoreFileTracker(
891    ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName, byte[] family,
892    String dstSFT) throws IOException {
893    requirePermission(c, "modifyColumnFamilyStoreFileTracker", tableName, family, null,
894      Action.ADMIN, Action.CREATE);
895    return dstSFT;
896  }
897
898  @Override
899  public void postModifyTable(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName,
900    TableDescriptor oldDesc, TableDescriptor currentDesc) throws IOException {
901    final Configuration conf = c.getEnvironment().getConfiguration();
902    // default the table owner to current user, if not specified.
903    final String owner = getActiveUser(c).getShortName();
904    User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
905      @Override
906      public Void run() throws Exception {
907        UserPermission userperm = new UserPermission(owner,
908          Permission.newBuilder(currentDesc.getTableName()).withActions(Action.values()).build());
909        try (Table table =
910          c.getEnvironment().getConnection().getTable(PermissionStorage.ACL_TABLE_NAME)) {
911          PermissionStorage.addUserPermission(conf, userperm, table);
912        }
913        return null;
914      }
915    });
916  }
917
918  @Override
919  public void preEnableTable(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName)
920    throws IOException {
921    requirePermission(c, "enableTable", tableName, null, null, Action.ADMIN, Action.CREATE);
922  }
923
924  @Override
925  public void preDisableTable(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName)
926    throws IOException {
927    if (Bytes.equals(tableName.getName(), PermissionStorage.ACL_GLOBAL_NAME)) {
928      // We have to unconditionally disallow disable of the ACL table when we are installed,
929      // even if not enforcing authorizations. We are still allowing grants and revocations,
930      // checking permissions and logging audit messages, etc. If the ACL table is not
931      // available we will fail random actions all over the place.
932      throw new AccessDeniedException("Not allowed to disable " + PermissionStorage.ACL_TABLE_NAME
933        + " table with AccessController installed");
934    }
935    requirePermission(c, "disableTable", tableName, null, null, Action.ADMIN, Action.CREATE);
936  }
937
938  @Override
939  public void preAbortProcedure(ObserverContext<MasterCoprocessorEnvironment> ctx,
940    final long procId) throws IOException {
941    requirePermission(ctx, "abortProcedure", Action.ADMIN);
942  }
943
944  @Override
945  public void postAbortProcedure(ObserverContext<MasterCoprocessorEnvironment> ctx)
946    throws IOException {
947    // There is nothing to do at this time after the procedure abort request was sent.
948  }
949
950  @Override
951  public void preGetProcedures(ObserverContext<MasterCoprocessorEnvironment> ctx)
952    throws IOException {
953    requirePermission(ctx, "getProcedure", Action.ADMIN);
954  }
955
956  @Override
957  public void preGetLocks(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
958    User user = getActiveUser(ctx);
959    accessChecker.requirePermission(user, "getLocks", null, Action.ADMIN);
960  }
961
962  @Override
963  public void preMove(ObserverContext<MasterCoprocessorEnvironment> c, RegionInfo region,
964    ServerName srcServer, ServerName destServer) throws IOException {
965    requirePermission(c, "move", region.getTable(), null, null, Action.ADMIN);
966  }
967
968  @Override
969  public void preAssign(ObserverContext<MasterCoprocessorEnvironment> c, RegionInfo regionInfo)
970    throws IOException {
971    requirePermission(c, "assign", regionInfo.getTable(), null, null, Action.ADMIN);
972  }
973
974  @Override
975  public void preUnassign(ObserverContext<MasterCoprocessorEnvironment> c, RegionInfo regionInfo)
976    throws IOException {
977    requirePermission(c, "unassign", regionInfo.getTable(), null, null, Action.ADMIN);
978  }
979
980  @Override
981  public void preRegionOffline(ObserverContext<MasterCoprocessorEnvironment> c,
982    RegionInfo regionInfo) throws IOException {
983    requirePermission(c, "regionOffline", regionInfo.getTable(), null, null, Action.ADMIN);
984  }
985
986  @Override
987  public void preSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx,
988    final boolean newValue, final MasterSwitchType switchType) throws IOException {
989    requirePermission(ctx, "setSplitOrMergeEnabled", Action.ADMIN);
990  }
991
992  @Override
993  public void preBalance(ObserverContext<MasterCoprocessorEnvironment> c, BalanceRequest request)
994    throws IOException {
995    requirePermission(c, "balance", Action.ADMIN);
996  }
997
998  @Override
999  public void preBalanceSwitch(ObserverContext<MasterCoprocessorEnvironment> c, boolean newValue)
1000    throws IOException {
1001    requirePermission(c, "balanceSwitch", Action.ADMIN);
1002  }
1003
1004  @Override
1005  public void preShutdown(ObserverContext<MasterCoprocessorEnvironment> c) throws IOException {
1006    requirePermission(c, "shutdown", Action.ADMIN);
1007  }
1008
1009  @Override
1010  public void preStopMaster(ObserverContext<MasterCoprocessorEnvironment> c) throws IOException {
1011    requirePermission(c, "stopMaster", Action.ADMIN);
1012  }
1013
1014  @Override
1015  public void postStartMaster(ObserverContext<MasterCoprocessorEnvironment> ctx)
1016    throws IOException {
1017    try (Admin admin = ctx.getEnvironment().getConnection().getAdmin()) {
1018      if (!admin.tableExists(PermissionStorage.ACL_TABLE_NAME)) {
1019        createACLTable(admin);
1020      } else {
1021        this.aclTabAvailable = true;
1022      }
1023    }
1024  }
1025
1026  /**
1027   * Create the ACL table n
1028   */
1029  private static void createACLTable(Admin admin) throws IOException {
1030    /** Table descriptor for ACL table */
1031    ColumnFamilyDescriptor cfd =
1032      ColumnFamilyDescriptorBuilder.newBuilder(PermissionStorage.ACL_LIST_FAMILY).setMaxVersions(1)
1033        .setInMemory(true).setBlockCacheEnabled(true).setBlocksize(8 * 1024)
1034        .setBloomFilterType(BloomType.NONE).setScope(HConstants.REPLICATION_SCOPE_LOCAL).build();
1035    TableDescriptor td = TableDescriptorBuilder.newBuilder(PermissionStorage.ACL_TABLE_NAME)
1036      .setColumnFamily(cfd).build();
1037    admin.createTable(td);
1038  }
1039
1040  @Override
1041  public void preSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1042    final SnapshotDescription snapshot, final TableDescriptor hTableDescriptor) throws IOException {
1043    // Move this ACL check to SnapshotManager#checkPermissions as part of AC deprecation.
1044    requirePermission(ctx, "snapshot " + snapshot.getName(), hTableDescriptor.getTableName(), null,
1045      null, Permission.Action.ADMIN);
1046  }
1047
1048  @Override
1049  public void preListSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
1050    final SnapshotDescription snapshot) throws IOException {
1051    User user = getActiveUser(ctx);
1052    if (SnapshotDescriptionUtils.isSnapshotOwner(snapshot, user)) {
1053      // list it, if user is the owner of snapshot
1054      AuthResult result = AuthResult.allow("listSnapshot " + snapshot.getName(),
1055        "Snapshot owner check allowed", user, null, null, null);
1056      AccessChecker.logResult(result);
1057    } else {
1058      accessChecker.requirePermission(user, "listSnapshot " + snapshot.getName(), null,
1059        Action.ADMIN);
1060    }
1061  }
1062
1063  @Override
1064  public void preCloneSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1065    final SnapshotDescription snapshot, final TableDescriptor hTableDescriptor) throws IOException {
1066    User user = getActiveUser(ctx);
1067    if (
1068      SnapshotDescriptionUtils.isSnapshotOwner(snapshot, user)
1069        && hTableDescriptor.getTableName().getNameAsString().equals(snapshot.getTableNameAsString())
1070    ) {
1071      // Snapshot owner is allowed to create a table with the same name as the snapshot he took
1072      AuthResult result = AuthResult.allow("cloneSnapshot " + snapshot.getName(),
1073        "Snapshot owner check allowed", user, null, hTableDescriptor.getTableName(), null);
1074      AccessChecker.logResult(result);
1075    } else {
1076      accessChecker.requirePermission(user, "cloneSnapshot " + snapshot.getName(), null,
1077        Action.ADMIN);
1078    }
1079  }
1080
1081  @Override
1082  public void preRestoreSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1083    final SnapshotDescription snapshot, final TableDescriptor hTableDescriptor) throws IOException {
1084    User user = getActiveUser(ctx);
1085    if (SnapshotDescriptionUtils.isSnapshotOwner(snapshot, user)) {
1086      accessChecker.requirePermission(user, "restoreSnapshot " + snapshot.getName(),
1087        hTableDescriptor.getTableName(), null, null, null, Permission.Action.ADMIN);
1088    } else {
1089      accessChecker.requirePermission(user, "restoreSnapshot " + snapshot.getName(), null,
1090        Action.ADMIN);
1091    }
1092  }
1093
1094  @Override
1095  public void preDeleteSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1096    final SnapshotDescription snapshot) throws IOException {
1097    User user = getActiveUser(ctx);
1098    if (SnapshotDescriptionUtils.isSnapshotOwner(snapshot, user)) {
1099      // Snapshot owner is allowed to delete the snapshot
1100      AuthResult result = AuthResult.allow("deleteSnapshot " + snapshot.getName(),
1101        "Snapshot owner check allowed", user, null, null, null);
1102      AccessChecker.logResult(result);
1103    } else {
1104      accessChecker.requirePermission(user, "deleteSnapshot " + snapshot.getName(), null,
1105        Action.ADMIN);
1106    }
1107  }
1108
1109  @Override
1110  public void preCreateNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
1111    NamespaceDescriptor ns) throws IOException {
1112    requireGlobalPermission(ctx, "createNamespace", Action.ADMIN, ns.getName());
1113  }
1114
1115  @Override
1116  public void preDeleteNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
1117    String namespace) throws IOException {
1118    requireGlobalPermission(ctx, "deleteNamespace", Action.ADMIN, namespace);
1119  }
1120
1121  @Override
1122  public void postDeleteNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
1123    final String namespace) throws IOException {
1124    final Configuration conf = ctx.getEnvironment().getConfiguration();
1125    User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
1126      @Override
1127      public Void run() throws Exception {
1128        try (Table table =
1129          ctx.getEnvironment().getConnection().getTable(PermissionStorage.ACL_TABLE_NAME)) {
1130          PermissionStorage.removeNamespacePermissions(conf, namespace, table);
1131        }
1132        return null;
1133      }
1134    });
1135    zkPermissionWatcher.deleteNamespaceACLNode(namespace);
1136    LOG.info(namespace + " entry deleted in " + PermissionStorage.ACL_TABLE_NAME + " table.");
1137  }
1138
1139  @Override
1140  public void preModifyNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
1141    NamespaceDescriptor currentNsDesc, NamespaceDescriptor newNsDesc) throws IOException {
1142    // We require only global permission so that
1143    // a user with NS admin cannot altering namespace configurations. i.e. namespace quota
1144    requireGlobalPermission(ctx, "modifyNamespace", Action.ADMIN, newNsDesc.getName());
1145  }
1146
1147  @Override
1148  public void preGetNamespaceDescriptor(ObserverContext<MasterCoprocessorEnvironment> ctx,
1149    String namespace) throws IOException {
1150    requireNamespacePermission(ctx, "getNamespaceDescriptor", namespace, Action.ADMIN);
1151  }
1152
1153  @Override
1154  public void postListNamespaces(ObserverContext<MasterCoprocessorEnvironment> ctx,
1155    List<String> namespaces) throws IOException {
1156    /* always allow namespace listing */
1157  }
1158
1159  @Override
1160  public void postListNamespaceDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
1161    List<NamespaceDescriptor> descriptors) throws IOException {
1162    // Retains only those which passes authorization checks, as the checks weren't done as part
1163    // of preGetTableDescriptors.
1164    Iterator<NamespaceDescriptor> itr = descriptors.iterator();
1165    User user = getActiveUser(ctx);
1166    while (itr.hasNext()) {
1167      NamespaceDescriptor desc = itr.next();
1168      try {
1169        accessChecker.requireNamespacePermission(user, "listNamespaces", desc.getName(), null,
1170          Action.ADMIN);
1171      } catch (AccessDeniedException e) {
1172        itr.remove();
1173      }
1174    }
1175  }
1176
1177  @Override
1178  public void preTableFlush(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1179    final TableName tableName) throws IOException {
1180    // Move this ACL check to MasterFlushTableProcedureManager#checkPermissions as part of AC
1181    // deprecation.
1182    requirePermission(ctx, "flushTable", tableName, null, null, Action.ADMIN, Action.CREATE);
1183  }
1184
1185  @Override
1186  public void preSplitRegion(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1187    final TableName tableName, final byte[] splitRow) throws IOException {
1188    requirePermission(ctx, "split", tableName, null, null, Action.ADMIN);
1189  }
1190
1191  @Override
1192  public void preClearDeadServers(ObserverContext<MasterCoprocessorEnvironment> ctx)
1193    throws IOException {
1194    requirePermission(ctx, "clearDeadServers", Action.ADMIN);
1195  }
1196
1197  @Override
1198  public void preDecommissionRegionServers(ObserverContext<MasterCoprocessorEnvironment> ctx,
1199    List<ServerName> servers, boolean offload) throws IOException {
1200    requirePermission(ctx, "decommissionRegionServers", Action.ADMIN);
1201  }
1202
1203  @Override
1204  public void preListDecommissionedRegionServers(ObserverContext<MasterCoprocessorEnvironment> ctx)
1205    throws IOException {
1206    requirePermission(ctx, "listDecommissionedRegionServers", Action.ADMIN);
1207  }
1208
1209  @Override
1210  public void preRecommissionRegionServer(ObserverContext<MasterCoprocessorEnvironment> ctx,
1211    ServerName server, List<byte[]> encodedRegionNames) throws IOException {
1212    requirePermission(ctx, "recommissionRegionServers", Action.ADMIN);
1213  }
1214
1215  /* ---- RegionObserver implementation ---- */
1216
1217  @Override
1218  public void preOpen(ObserverContext<RegionCoprocessorEnvironment> c) throws IOException {
1219    RegionCoprocessorEnvironment env = c.getEnvironment();
1220    final Region region = env.getRegion();
1221    if (region == null) {
1222      LOG.error("NULL region from RegionCoprocessorEnvironment in preOpen()");
1223    } else {
1224      RegionInfo regionInfo = region.getRegionInfo();
1225      if (regionInfo.getTable().isSystemTable()) {
1226        checkSystemOrSuperUser(getActiveUser(c));
1227      } else {
1228        requirePermission(c, "preOpen", Action.ADMIN);
1229      }
1230    }
1231  }
1232
1233  @Override
1234  public void postOpen(ObserverContext<RegionCoprocessorEnvironment> c) {
1235    RegionCoprocessorEnvironment env = c.getEnvironment();
1236    final Region region = env.getRegion();
1237    if (region == null) {
1238      LOG.error("NULL region from RegionCoprocessorEnvironment in postOpen()");
1239      return;
1240    }
1241    if (PermissionStorage.isAclRegion(region)) {
1242      aclRegion = true;
1243      try {
1244        initialize(env);
1245      } catch (IOException ex) {
1246        // if we can't obtain permissions, it's better to fail
1247        // than perform checks incorrectly
1248        throw new RuntimeException("Failed to initialize permissions cache", ex);
1249      }
1250    } else {
1251      initialized = true;
1252    }
1253  }
1254
1255  @Override
1256  public void preFlush(ObserverContext<RegionCoprocessorEnvironment> c,
1257    FlushLifeCycleTracker tracker) throws IOException {
1258    requirePermission(c, "flush", getTableName(c.getEnvironment()), null, null, Action.ADMIN,
1259      Action.CREATE);
1260  }
1261
1262  @Override
1263  public InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
1264    InternalScanner scanner, ScanType scanType, CompactionLifeCycleTracker tracker,
1265    CompactionRequest request) throws IOException {
1266    requirePermission(c, "compact", getTableName(c.getEnvironment()), null, null, Action.ADMIN,
1267      Action.CREATE);
1268    return scanner;
1269  }
1270
1271  private void internalPreRead(final ObserverContext<RegionCoprocessorEnvironment> c,
1272    final Query query, OpType opType) throws IOException {
1273    Filter filter = query.getFilter();
1274    // Don't wrap an AccessControlFilter
1275    if (filter != null && filter instanceof AccessControlFilter) {
1276      return;
1277    }
1278    User user = getActiveUser(c);
1279    RegionCoprocessorEnvironment env = c.getEnvironment();
1280    Map<byte[], ? extends Collection<byte[]>> families = null;
1281    switch (opType) {
1282      case GET:
1283      case EXISTS:
1284        families = ((Get) query).getFamilyMap();
1285        break;
1286      case SCAN:
1287        families = ((Scan) query).getFamilyMap();
1288        break;
1289      default:
1290        throw new RuntimeException("Unhandled operation " + opType);
1291    }
1292    AuthResult authResult = permissionGranted(opType, user, env, families, Action.READ);
1293    Region region = getRegion(env);
1294    TableName table = getTableName(region);
1295    Map<ByteRange, Integer> cfVsMaxVersions = Maps.newHashMap();
1296    for (ColumnFamilyDescriptor hcd : region.getTableDescriptor().getColumnFamilies()) {
1297      cfVsMaxVersions.put(new SimpleMutableByteRange(hcd.getName()), hcd.getMaxVersions());
1298    }
1299    if (!authResult.isAllowed()) {
1300      if (!cellFeaturesEnabled || compatibleEarlyTermination) {
1301        // Old behavior: Scan with only qualifier checks if we have partial
1302        // permission. Backwards compatible behavior is to throw an
1303        // AccessDeniedException immediately if there are no grants for table
1304        // or CF or CF+qual. Only proceed with an injected filter if there are
1305        // grants for qualifiers. Otherwise we will fall through below and log
1306        // the result and throw an ADE. We may end up checking qualifier
1307        // grants three times (permissionGranted above, here, and in the
1308        // filter) but that's the price of backwards compatibility.
1309        if (hasFamilyQualifierPermission(user, Action.READ, env, families)) {
1310          authResult.setAllowed(true);
1311          authResult.setReason("Access allowed with filter");
1312          // Only wrap the filter if we are enforcing authorizations
1313          if (authorizationEnabled) {
1314            Filter ourFilter = new AccessControlFilter(getAuthManager(), user, table,
1315              AccessControlFilter.Strategy.CHECK_TABLE_AND_CF_ONLY, cfVsMaxVersions);
1316            // wrap any existing filter
1317            if (filter != null) {
1318              ourFilter = new FilterList(FilterList.Operator.MUST_PASS_ALL,
1319                Lists.newArrayList(ourFilter, filter));
1320            }
1321            switch (opType) {
1322              case GET:
1323              case EXISTS:
1324                ((Get) query).setFilter(ourFilter);
1325                break;
1326              case SCAN:
1327                ((Scan) query).setFilter(ourFilter);
1328                break;
1329              default:
1330                throw new RuntimeException("Unhandled operation " + opType);
1331            }
1332          }
1333        }
1334      } else {
1335        // New behavior: Any access we might be granted is more fine-grained
1336        // than whole table or CF. Simply inject a filter and return what is
1337        // allowed. We will not throw an AccessDeniedException. This is a
1338        // behavioral change since 0.96.
1339        authResult.setAllowed(true);
1340        authResult.setReason("Access allowed with filter");
1341        // Only wrap the filter if we are enforcing authorizations
1342        if (authorizationEnabled) {
1343          Filter ourFilter = new AccessControlFilter(getAuthManager(), user, table,
1344            AccessControlFilter.Strategy.CHECK_CELL_DEFAULT, cfVsMaxVersions);
1345          // wrap any existing filter
1346          if (filter != null) {
1347            ourFilter = new FilterList(FilterList.Operator.MUST_PASS_ALL,
1348              Lists.newArrayList(ourFilter, filter));
1349          }
1350          switch (opType) {
1351            case GET:
1352            case EXISTS:
1353              ((Get) query).setFilter(ourFilter);
1354              break;
1355            case SCAN:
1356              ((Scan) query).setFilter(ourFilter);
1357              break;
1358            default:
1359              throw new RuntimeException("Unhandled operation " + opType);
1360          }
1361        }
1362      }
1363    }
1364
1365    AccessChecker.logResult(authResult);
1366    if (authorizationEnabled && !authResult.isAllowed()) {
1367      throw new AccessDeniedException("Insufficient permissions for user '"
1368        + (user != null ? user.getShortName() : "null") + "' (table=" + table + ", action=READ)");
1369    }
1370  }
1371
1372  @Override
1373  public void preGetOp(final ObserverContext<RegionCoprocessorEnvironment> c, final Get get,
1374    final List<Cell> result) throws IOException {
1375    internalPreRead(c, get, OpType.GET);
1376  }
1377
1378  @Override
1379  public boolean preExists(final ObserverContext<RegionCoprocessorEnvironment> c, final Get get,
1380    final boolean exists) throws IOException {
1381    internalPreRead(c, get, OpType.EXISTS);
1382    return exists;
1383  }
1384
1385  @Override
1386  public void prePut(final ObserverContext<RegionCoprocessorEnvironment> c, final Put put,
1387    final WALEdit edit, final Durability durability) throws IOException {
1388    User user = getActiveUser(c);
1389    checkForReservedTagPresence(user, put);
1390
1391    // Require WRITE permission to the table, CF, or top visible value, if any.
1392    // NOTE: We don't need to check the permissions for any earlier Puts
1393    // because we treat the ACLs in each Put as timestamped like any other
1394    // HBase value. A new ACL in a new Put applies to that Put. It doesn't
1395    // change the ACL of any previous Put. This allows simple evolution of
1396    // security policy over time without requiring expensive updates.
1397    RegionCoprocessorEnvironment env = c.getEnvironment();
1398    Map<byte[], ? extends Collection<Cell>> families = put.getFamilyCellMap();
1399    AuthResult authResult = permissionGranted(OpType.PUT, user, env, families, Action.WRITE);
1400    AccessChecker.logResult(authResult);
1401    if (!authResult.isAllowed()) {
1402      if (cellFeaturesEnabled && !compatibleEarlyTermination) {
1403        put.setAttribute(CHECK_COVERING_PERM, TRUE);
1404      } else if (authorizationEnabled) {
1405        throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
1406      }
1407    }
1408
1409    // Add cell ACLs from the operation to the cells themselves
1410    byte[] bytes = put.getAttribute(AccessControlConstants.OP_ATTRIBUTE_ACL);
1411    if (bytes != null) {
1412      if (cellFeaturesEnabled) {
1413        addCellPermissions(bytes, put.getFamilyCellMap());
1414      } else {
1415        throw new DoNotRetryIOException("Cell ACLs cannot be persisted");
1416      }
1417    }
1418  }
1419
1420  @Override
1421  public void postPut(final ObserverContext<RegionCoprocessorEnvironment> c, final Put put,
1422    final WALEdit edit, final Durability durability) {
1423    if (aclRegion) {
1424      updateACL(c.getEnvironment(), put.getFamilyCellMap());
1425    }
1426  }
1427
1428  @Override
1429  public void preDelete(final ObserverContext<RegionCoprocessorEnvironment> c, final Delete delete,
1430    final WALEdit edit, final Durability durability) throws IOException {
1431    // An ACL on a delete is useless, we shouldn't allow it
1432    if (delete.getAttribute(AccessControlConstants.OP_ATTRIBUTE_ACL) != null) {
1433      throw new DoNotRetryIOException("ACL on delete has no effect: " + delete.toString());
1434    }
1435    // Require WRITE permissions on all cells covered by the delete. Unlike
1436    // for Puts we need to check all visible prior versions, because a major
1437    // compaction could remove them. If the user doesn't have permission to
1438    // overwrite any of the visible versions ('visible' defined as not covered
1439    // by a tombstone already) then we have to disallow this operation.
1440    RegionCoprocessorEnvironment env = c.getEnvironment();
1441    Map<byte[], ? extends Collection<Cell>> families = delete.getFamilyCellMap();
1442    User user = getActiveUser(c);
1443    AuthResult authResult = permissionGranted(OpType.DELETE, user, env, families, Action.WRITE);
1444    AccessChecker.logResult(authResult);
1445    if (!authResult.isAllowed()) {
1446      if (cellFeaturesEnabled && !compatibleEarlyTermination) {
1447        delete.setAttribute(CHECK_COVERING_PERM, TRUE);
1448      } else if (authorizationEnabled) {
1449        throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
1450      }
1451    }
1452  }
1453
1454  @Override
1455  public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
1456    MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
1457    if (cellFeaturesEnabled && !compatibleEarlyTermination) {
1458      TableName table = c.getEnvironment().getRegion().getRegionInfo().getTable();
1459      User user = getActiveUser(c);
1460      for (int i = 0; i < miniBatchOp.size(); i++) {
1461        Mutation m = miniBatchOp.getOperation(i);
1462        if (m.getAttribute(CHECK_COVERING_PERM) != null) {
1463          // We have a failure with table, cf and q perm checks and now giving a chance for cell
1464          // perm check
1465          OpType opType;
1466          long timestamp;
1467          if (m instanceof Put) {
1468            checkForReservedTagPresence(user, m);
1469            opType = OpType.PUT;
1470            timestamp = m.getTimestamp();
1471          } else if (m instanceof Delete) {
1472            opType = OpType.DELETE;
1473            timestamp = m.getTimestamp();
1474          } else if (m instanceof Increment) {
1475            opType = OpType.INCREMENT;
1476            timestamp = ((Increment) m).getTimeRange().getMax();
1477          } else if (m instanceof Append) {
1478            opType = OpType.APPEND;
1479            timestamp = ((Append) m).getTimeRange().getMax();
1480          } else {
1481            // If the operation type is not Put/Delete/Increment/Append, do nothing
1482            continue;
1483          }
1484          AuthResult authResult = null;
1485          if (
1486            checkCoveringPermission(user, opType, c.getEnvironment(), m.getRow(),
1487              m.getFamilyCellMap(), timestamp, Action.WRITE)
1488          ) {
1489            authResult = AuthResult.allow(opType.toString(), "Covering cell set", user,
1490              Action.WRITE, table, m.getFamilyCellMap());
1491          } else {
1492            authResult = AuthResult.deny(opType.toString(), "Covering cell set", user, Action.WRITE,
1493              table, m.getFamilyCellMap());
1494          }
1495          AccessChecker.logResult(authResult);
1496          if (authorizationEnabled && !authResult.isAllowed()) {
1497            throw new AccessDeniedException(
1498              "Insufficient permissions " + authResult.toContextString());
1499          }
1500        }
1501      }
1502    }
1503  }
1504
1505  @Override
1506  public void postDelete(final ObserverContext<RegionCoprocessorEnvironment> c, final Delete delete,
1507    final WALEdit edit, final Durability durability) throws IOException {
1508    if (aclRegion) {
1509      updateACL(c.getEnvironment(), delete.getFamilyCellMap());
1510    }
1511  }
1512
1513  @Override
1514  public boolean preCheckAndPut(final ObserverContext<RegionCoprocessorEnvironment> c,
1515    final byte[] row, final byte[] family, final byte[] qualifier, final CompareOperator op,
1516    final ByteArrayComparable comparator, final Put put, final boolean result) throws IOException {
1517    User user = getActiveUser(c);
1518    checkForReservedTagPresence(user, put);
1519
1520    // Require READ and WRITE permissions on the table, CF, and KV to update
1521    RegionCoprocessorEnvironment env = c.getEnvironment();
1522    Map<byte[], ? extends Collection<byte[]>> families = makeFamilyMap(family, qualifier);
1523    AuthResult authResult =
1524      permissionGranted(OpType.CHECK_AND_PUT, user, env, families, Action.READ, Action.WRITE);
1525    AccessChecker.logResult(authResult);
1526    if (!authResult.isAllowed()) {
1527      if (cellFeaturesEnabled && !compatibleEarlyTermination) {
1528        put.setAttribute(CHECK_COVERING_PERM, TRUE);
1529      } else if (authorizationEnabled) {
1530        throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
1531      }
1532    }
1533
1534    byte[] bytes = put.getAttribute(AccessControlConstants.OP_ATTRIBUTE_ACL);
1535    if (bytes != null) {
1536      if (cellFeaturesEnabled) {
1537        addCellPermissions(bytes, put.getFamilyCellMap());
1538      } else {
1539        throw new DoNotRetryIOException("Cell ACLs cannot be persisted");
1540      }
1541    }
1542    return result;
1543  }
1544
1545  @Override
1546  public boolean preCheckAndPutAfterRowLock(final ObserverContext<RegionCoprocessorEnvironment> c,
1547    final byte[] row, final byte[] family, final byte[] qualifier, final CompareOperator opp,
1548    final ByteArrayComparable comparator, final Put put, final boolean result) throws IOException {
1549    if (put.getAttribute(CHECK_COVERING_PERM) != null) {
1550      // We had failure with table, cf and q perm checks and now giving a chance for cell
1551      // perm check
1552      TableName table = c.getEnvironment().getRegion().getRegionInfo().getTable();
1553      Map<byte[], ? extends Collection<byte[]>> families = makeFamilyMap(family, qualifier);
1554      AuthResult authResult = null;
1555      User user = getActiveUser(c);
1556      if (
1557        checkCoveringPermission(user, OpType.CHECK_AND_PUT, c.getEnvironment(), row, families,
1558          HConstants.LATEST_TIMESTAMP, Action.READ)
1559      ) {
1560        authResult = AuthResult.allow(OpType.CHECK_AND_PUT.toString(), "Covering cell set", user,
1561          Action.READ, table, families);
1562      } else {
1563        authResult = AuthResult.deny(OpType.CHECK_AND_PUT.toString(), "Covering cell set", user,
1564          Action.READ, table, families);
1565      }
1566      AccessChecker.logResult(authResult);
1567      if (authorizationEnabled && !authResult.isAllowed()) {
1568        throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
1569      }
1570    }
1571    return result;
1572  }
1573
1574  @Override
1575  public boolean preCheckAndDelete(final ObserverContext<RegionCoprocessorEnvironment> c,
1576    final byte[] row, final byte[] family, final byte[] qualifier, final CompareOperator op,
1577    final ByteArrayComparable comparator, final Delete delete, final boolean result)
1578    throws IOException {
1579    // An ACL on a delete is useless, we shouldn't allow it
1580    if (delete.getAttribute(AccessControlConstants.OP_ATTRIBUTE_ACL) != null) {
1581      throw new DoNotRetryIOException("ACL on checkAndDelete has no effect: " + delete.toString());
1582    }
1583    // Require READ and WRITE permissions on the table, CF, and the KV covered
1584    // by the delete
1585    RegionCoprocessorEnvironment env = c.getEnvironment();
1586    Map<byte[], ? extends Collection<byte[]>> families = makeFamilyMap(family, qualifier);
1587    User user = getActiveUser(c);
1588    AuthResult authResult =
1589      permissionGranted(OpType.CHECK_AND_DELETE, user, env, families, Action.READ, Action.WRITE);
1590    AccessChecker.logResult(authResult);
1591    if (!authResult.isAllowed()) {
1592      if (cellFeaturesEnabled && !compatibleEarlyTermination) {
1593        delete.setAttribute(CHECK_COVERING_PERM, TRUE);
1594      } else if (authorizationEnabled) {
1595        throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
1596      }
1597    }
1598    return result;
1599  }
1600
1601  @Override
1602  public boolean preCheckAndDeleteAfterRowLock(
1603    final ObserverContext<RegionCoprocessorEnvironment> c, final byte[] row, final byte[] family,
1604    final byte[] qualifier, final CompareOperator op, final ByteArrayComparable comparator,
1605    final Delete delete, final boolean result) throws IOException {
1606    if (delete.getAttribute(CHECK_COVERING_PERM) != null) {
1607      // We had failure with table, cf and q perm checks and now giving a chance for cell
1608      // perm check
1609      TableName table = c.getEnvironment().getRegion().getRegionInfo().getTable();
1610      Map<byte[], ? extends Collection<byte[]>> families = makeFamilyMap(family, qualifier);
1611      AuthResult authResult = null;
1612      User user = getActiveUser(c);
1613      if (
1614        checkCoveringPermission(user, OpType.CHECK_AND_DELETE, c.getEnvironment(), row, families,
1615          HConstants.LATEST_TIMESTAMP, Action.READ)
1616      ) {
1617        authResult = AuthResult.allow(OpType.CHECK_AND_DELETE.toString(), "Covering cell set", user,
1618          Action.READ, table, families);
1619      } else {
1620        authResult = AuthResult.deny(OpType.CHECK_AND_DELETE.toString(), "Covering cell set", user,
1621          Action.READ, table, families);
1622      }
1623      AccessChecker.logResult(authResult);
1624      if (authorizationEnabled && !authResult.isAllowed()) {
1625        throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
1626      }
1627    }
1628    return result;
1629  }
1630
1631  @Override
1632  public Result preAppend(ObserverContext<RegionCoprocessorEnvironment> c, Append append)
1633    throws IOException {
1634    User user = getActiveUser(c);
1635    checkForReservedTagPresence(user, append);
1636
1637    // Require WRITE permission to the table, CF, and the KV to be appended
1638    RegionCoprocessorEnvironment env = c.getEnvironment();
1639    Map<byte[], ? extends Collection<Cell>> families = append.getFamilyCellMap();
1640    AuthResult authResult = permissionGranted(OpType.APPEND, user, env, families, Action.WRITE);
1641    AccessChecker.logResult(authResult);
1642    if (!authResult.isAllowed()) {
1643      if (cellFeaturesEnabled && !compatibleEarlyTermination) {
1644        append.setAttribute(CHECK_COVERING_PERM, TRUE);
1645      } else if (authorizationEnabled) {
1646        throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
1647      }
1648    }
1649
1650    byte[] bytes = append.getAttribute(AccessControlConstants.OP_ATTRIBUTE_ACL);
1651    if (bytes != null) {
1652      if (cellFeaturesEnabled) {
1653        addCellPermissions(bytes, append.getFamilyCellMap());
1654      } else {
1655        throw new DoNotRetryIOException("Cell ACLs cannot be persisted");
1656      }
1657    }
1658
1659    return null;
1660  }
1661
1662  @Override
1663  public Result preIncrement(final ObserverContext<RegionCoprocessorEnvironment> c,
1664    final Increment increment) throws IOException {
1665    User user = getActiveUser(c);
1666    checkForReservedTagPresence(user, increment);
1667
1668    // Require WRITE permission to the table, CF, and the KV to be replaced by
1669    // the incremented value
1670    RegionCoprocessorEnvironment env = c.getEnvironment();
1671    Map<byte[], ? extends Collection<Cell>> families = increment.getFamilyCellMap();
1672    AuthResult authResult = permissionGranted(OpType.INCREMENT, user, env, families, Action.WRITE);
1673    AccessChecker.logResult(authResult);
1674    if (!authResult.isAllowed()) {
1675      if (cellFeaturesEnabled && !compatibleEarlyTermination) {
1676        increment.setAttribute(CHECK_COVERING_PERM, TRUE);
1677      } else if (authorizationEnabled) {
1678        throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
1679      }
1680    }
1681
1682    byte[] bytes = increment.getAttribute(AccessControlConstants.OP_ATTRIBUTE_ACL);
1683    if (bytes != null) {
1684      if (cellFeaturesEnabled) {
1685        addCellPermissions(bytes, increment.getFamilyCellMap());
1686      } else {
1687        throw new DoNotRetryIOException("Cell ACLs cannot be persisted");
1688      }
1689    }
1690
1691    return null;
1692  }
1693
1694  @Override
1695  public List<Pair<Cell, Cell>> postIncrementBeforeWAL(
1696    ObserverContext<RegionCoprocessorEnvironment> ctx, Mutation mutation,
1697    List<Pair<Cell, Cell>> cellPairs) throws IOException {
1698    // If the HFile version is insufficient to persist tags, we won't have any
1699    // work to do here
1700    if (!cellFeaturesEnabled || mutation.getACL() == null) {
1701      return cellPairs;
1702    }
1703    return cellPairs.stream()
1704      .map(pair -> new Pair<>(pair.getFirst(),
1705        createNewCellWithTags(mutation, pair.getFirst(), pair.getSecond())))
1706      .collect(Collectors.toList());
1707  }
1708
1709  @Override
1710  public List<Pair<Cell, Cell>> postAppendBeforeWAL(
1711    ObserverContext<RegionCoprocessorEnvironment> ctx, Mutation mutation,
1712    List<Pair<Cell, Cell>> cellPairs) throws IOException {
1713    // If the HFile version is insufficient to persist tags, we won't have any
1714    // work to do here
1715    if (!cellFeaturesEnabled || mutation.getACL() == null) {
1716      return cellPairs;
1717    }
1718    return cellPairs.stream()
1719      .map(pair -> new Pair<>(pair.getFirst(),
1720        createNewCellWithTags(mutation, pair.getFirst(), pair.getSecond())))
1721      .collect(Collectors.toList());
1722  }
1723
1724  private Cell createNewCellWithTags(Mutation mutation, Cell oldCell, Cell newCell) {
1725    // As Increment and Append operations have already copied the tags of oldCell to the newCell,
1726    // there is no need to rewrite them again. Just extract non-acl tags of newCell if we need to
1727    // add a new acl tag for the cell. Actually, oldCell is useless here.
1728    List<Tag> tags = Lists.newArrayList();
1729    if (newCell != null) {
1730      Iterator<Tag> tagIterator = PrivateCellUtil.tagsIterator(newCell);
1731      while (tagIterator.hasNext()) {
1732        Tag tag = tagIterator.next();
1733        if (tag.getType() != PermissionStorage.ACL_TAG_TYPE) {
1734          // Not an ACL tag, just carry it through
1735          if (LOG.isTraceEnabled()) {
1736            LOG.trace("Carrying forward tag from " + newCell + ": type " + tag.getType()
1737              + " length " + tag.getValueLength());
1738          }
1739          tags.add(tag);
1740        }
1741      }
1742    }
1743
1744    // We have checked the ACL tag of mutation is not null.
1745    // So that the tags could not be empty.
1746    tags.add(new ArrayBackedTag(PermissionStorage.ACL_TAG_TYPE, mutation.getACL()));
1747    return PrivateCellUtil.createCell(newCell, tags);
1748  }
1749
1750  @Override
1751  public void preScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c, final Scan scan)
1752    throws IOException {
1753    internalPreRead(c, scan, OpType.SCAN);
1754  }
1755
1756  @Override
1757  public RegionScanner postScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
1758    final Scan scan, final RegionScanner s) throws IOException {
1759    User user = getActiveUser(c);
1760    if (user != null && user.getShortName() != null) {
1761      // store reference to scanner owner for later checks
1762      scannerOwners.put(s, user.getShortName());
1763    }
1764    return s;
1765  }
1766
1767  @Override
1768  public boolean preScannerNext(final ObserverContext<RegionCoprocessorEnvironment> c,
1769    final InternalScanner s, final List<Result> result, final int limit, final boolean hasNext)
1770    throws IOException {
1771    requireScannerOwner(s);
1772    return hasNext;
1773  }
1774
1775  @Override
1776  public void preScannerClose(final ObserverContext<RegionCoprocessorEnvironment> c,
1777    final InternalScanner s) throws IOException {
1778    requireScannerOwner(s);
1779  }
1780
1781  @Override
1782  public void postScannerClose(final ObserverContext<RegionCoprocessorEnvironment> c,
1783    final InternalScanner s) throws IOException {
1784    // clean up any associated owner mapping
1785    scannerOwners.remove(s);
1786  }
1787
1788  /**
1789   * Verify, when servicing an RPC, that the caller is the scanner owner. If so, we assume that
1790   * access control is correctly enforced based on the checks performed in preScannerOpen()
1791   */
1792  private void requireScannerOwner(InternalScanner s) throws AccessDeniedException {
1793    if (!RpcServer.isInRpcCallContext()) {
1794      return;
1795    }
1796    String requestUserName = RpcServer.getRequestUserName().orElse(null);
1797    String owner = scannerOwners.get(s);
1798    if (authorizationEnabled && owner != null && !owner.equals(requestUserName)) {
1799      throw new AccessDeniedException("User '" + requestUserName + "' is not the scanner owner!");
1800    }
1801  }
1802
1803  /**
1804   * Verifies user has CREATE or ADMIN privileges on the Column Families involved in the
1805   * bulkLoadHFile request. Specific Column Write privileges are presently ignored.
1806   */
1807  @Override
1808  public void preBulkLoadHFile(ObserverContext<RegionCoprocessorEnvironment> ctx,
1809    List<Pair<byte[], String>> familyPaths) throws IOException {
1810    User user = getActiveUser(ctx);
1811    for (Pair<byte[], String> el : familyPaths) {
1812      accessChecker.requirePermission(user, "preBulkLoadHFile",
1813        ctx.getEnvironment().getRegion().getTableDescriptor().getTableName(), el.getFirst(), null,
1814        null, Action.ADMIN, Action.CREATE);
1815    }
1816  }
1817
1818  /**
1819   * Authorization check for SecureBulkLoadProtocol.prepareBulkLoad()
1820   * @param ctx the context n
1821   */
1822  @Override
1823  public void prePrepareBulkLoad(ObserverContext<RegionCoprocessorEnvironment> ctx)
1824    throws IOException {
1825    requireAccess(ctx, "prePrepareBulkLoad",
1826      ctx.getEnvironment().getRegion().getTableDescriptor().getTableName(), Action.ADMIN,
1827      Action.CREATE);
1828  }
1829
1830  /**
1831   * Authorization security check for SecureBulkLoadProtocol.cleanupBulkLoad()
1832   * @param ctx the context n
1833   */
1834  @Override
1835  public void preCleanupBulkLoad(ObserverContext<RegionCoprocessorEnvironment> ctx)
1836    throws IOException {
1837    requireAccess(ctx, "preCleanupBulkLoad",
1838      ctx.getEnvironment().getRegion().getTableDescriptor().getTableName(), Action.ADMIN,
1839      Action.CREATE);
1840  }
1841
1842  /* ---- EndpointObserver implementation ---- */
1843
1844  @Override
1845  public Message preEndpointInvocation(ObserverContext<RegionCoprocessorEnvironment> ctx,
1846    Service service, String methodName, Message request) throws IOException {
1847    // Don't intercept calls to our own AccessControlService, we check for
1848    // appropriate permissions in the service handlers
1849    if (shouldCheckExecPermission && !(service instanceof AccessControlService)) {
1850      requirePermission(ctx,
1851        "invoke(" + service.getDescriptorForType().getName() + "." + methodName + ")",
1852        getTableName(ctx.getEnvironment()), null, null, Action.EXEC);
1853    }
1854    return request;
1855  }
1856
1857  @Override
1858  public void postEndpointInvocation(ObserverContext<RegionCoprocessorEnvironment> ctx,
1859    Service service, String methodName, Message request, Message.Builder responseBuilder)
1860    throws IOException {
1861  }
1862
1863  /* ---- Protobuf AccessControlService implementation ---- */
1864
1865  /**
1866   * @deprecated since 2.2.0 and will be removed in 4.0.0. Use
1867   *             {@link Admin#grant(UserPermission, boolean)} instead.
1868   * @see Admin#grant(UserPermission, boolean)
1869   * @see <a href="https://issues.apache.org/jira/browse/HBASE-21739">HBASE-21739</a>
1870   */
1871  @Deprecated
1872  @Override
1873  public void grant(RpcController controller, AccessControlProtos.GrantRequest request,
1874    RpcCallback<AccessControlProtos.GrantResponse> done) {
1875    final UserPermission perm = AccessControlUtil.toUserPermission(request.getUserPermission());
1876    AccessControlProtos.GrantResponse response = null;
1877    try {
1878      // verify it's only running at .acl.
1879      if (aclRegion) {
1880        if (!initialized) {
1881          throw new CoprocessorException("AccessController not yet initialized");
1882        }
1883        User caller = RpcServer.getRequestUser().orElse(null);
1884        if (LOG.isDebugEnabled()) {
1885          LOG.debug("Received request from {} to grant access permission {}", caller.getName(),
1886            perm.toString());
1887        }
1888        preGrantOrRevoke(caller, "grant", perm);
1889
1890        // regionEnv is set at #start. Hopefully not null at this point.
1891        regionEnv.getConnection().getAdmin().grant(
1892          new UserPermission(perm.getUser(), perm.getPermission()),
1893          request.getMergeExistingPermissions());
1894        if (AUDITLOG.isTraceEnabled()) {
1895          // audit log should store permission changes in addition to auth results
1896          AUDITLOG.trace("Granted permission " + perm.toString());
1897        }
1898      } else {
1899        throw new CoprocessorException(AccessController.class,
1900          "This method " + "can only execute at " + PermissionStorage.ACL_TABLE_NAME + " table.");
1901      }
1902      response = AccessControlProtos.GrantResponse.getDefaultInstance();
1903    } catch (IOException ioe) {
1904      // pass exception back up
1905      CoprocessorRpcUtils.setControllerException(controller, ioe);
1906    }
1907    done.run(response);
1908  }
1909
1910  /**
1911   * @deprecated since 2.2.0 and will be removed in 4.0.0. Use {@link Admin#revoke(UserPermission)}
1912   *             instead.
1913   * @see Admin#revoke(UserPermission)
1914   * @see <a href="https://issues.apache.org/jira/browse/HBASE-21739">HBASE-21739</a>
1915   */
1916  @Deprecated
1917  @Override
1918  public void revoke(RpcController controller, AccessControlProtos.RevokeRequest request,
1919    RpcCallback<AccessControlProtos.RevokeResponse> done) {
1920    final UserPermission perm = AccessControlUtil.toUserPermission(request.getUserPermission());
1921    AccessControlProtos.RevokeResponse response = null;
1922    try {
1923      // only allowed to be called on _acl_ region
1924      if (aclRegion) {
1925        if (!initialized) {
1926          throw new CoprocessorException("AccessController not yet initialized");
1927        }
1928        User caller = RpcServer.getRequestUser().orElse(null);
1929        if (LOG.isDebugEnabled()) {
1930          LOG.debug("Received request from {} to revoke access permission {}",
1931            caller.getShortName(), perm.toString());
1932        }
1933        preGrantOrRevoke(caller, "revoke", perm);
1934        // regionEnv is set at #start. Hopefully not null here.
1935        regionEnv.getConnection().getAdmin()
1936          .revoke(new UserPermission(perm.getUser(), perm.getPermission()));
1937        if (AUDITLOG.isTraceEnabled()) {
1938          // audit log should record all permission changes
1939          AUDITLOG.trace("Revoked permission " + perm.toString());
1940        }
1941      } else {
1942        throw new CoprocessorException(AccessController.class,
1943          "This method " + "can only execute at " + PermissionStorage.ACL_TABLE_NAME + " table.");
1944      }
1945      response = AccessControlProtos.RevokeResponse.getDefaultInstance();
1946    } catch (IOException ioe) {
1947      // pass exception back up
1948      CoprocessorRpcUtils.setControllerException(controller, ioe);
1949    }
1950    done.run(response);
1951  }
1952
1953  /**
1954   * @deprecated since 2.2.0 and will be removed in 4.0.0. Use
1955   *             {@link Admin#getUserPermissions(GetUserPermissionsRequest)} instead.
1956   * @see Admin#getUserPermissions(GetUserPermissionsRequest)
1957   * @see <a href="https://issues.apache.org/jira/browse/HBASE-21911">HBASE-21911</a>
1958   */
1959  @Deprecated
1960  @Override
1961  public void getUserPermissions(RpcController controller,
1962    AccessControlProtos.GetUserPermissionsRequest request,
1963    RpcCallback<AccessControlProtos.GetUserPermissionsResponse> done) {
1964    AccessControlProtos.GetUserPermissionsResponse response = null;
1965    try {
1966      // only allowed to be called on _acl_ region
1967      if (aclRegion) {
1968        if (!initialized) {
1969          throw new CoprocessorException("AccessController not yet initialized");
1970        }
1971        User caller = RpcServer.getRequestUser().orElse(null);
1972        final String userName = request.hasUserName() ? request.getUserName().toStringUtf8() : null;
1973        final String namespace =
1974          request.hasNamespaceName() ? request.getNamespaceName().toStringUtf8() : null;
1975        final TableName table =
1976          request.hasTableName() ? ProtobufUtil.toTableName(request.getTableName()) : null;
1977        final byte[] cf =
1978          request.hasColumnFamily() ? request.getColumnFamily().toByteArray() : null;
1979        final byte[] cq =
1980          request.hasColumnQualifier() ? request.getColumnQualifier().toByteArray() : null;
1981        preGetUserPermissions(caller, userName, namespace, table, cf, cq);
1982        GetUserPermissionsRequest getUserPermissionsRequest = null;
1983        if (request.getType() == AccessControlProtos.Permission.Type.Table) {
1984          getUserPermissionsRequest = GetUserPermissionsRequest.newBuilder(table).withFamily(cf)
1985            .withQualifier(cq).withUserName(userName).build();
1986        } else if (request.getType() == AccessControlProtos.Permission.Type.Namespace) {
1987          getUserPermissionsRequest =
1988            GetUserPermissionsRequest.newBuilder(namespace).withUserName(userName).build();
1989        } else {
1990          getUserPermissionsRequest =
1991            GetUserPermissionsRequest.newBuilder().withUserName(userName).build();
1992        }
1993        List<UserPermission> perms =
1994          regionEnv.getConnection().getAdmin().getUserPermissions(getUserPermissionsRequest);
1995        response = AccessControlUtil.buildGetUserPermissionsResponse(perms);
1996      } else {
1997        throw new CoprocessorException(AccessController.class,
1998          "This method " + "can only execute at " + PermissionStorage.ACL_TABLE_NAME + " table.");
1999      }
2000    } catch (IOException ioe) {
2001      // pass exception back up
2002      CoprocessorRpcUtils.setControllerException(controller, ioe);
2003    }
2004    done.run(response);
2005  }
2006
2007  /**
2008   * @deprecated since 2.2.0 and will be removed 4.0.0. Use {@link Admin#hasUserPermissions(List)}
2009   *             instead.
2010   * @see Admin#hasUserPermissions(List)
2011   * @see <a href="https://issues.apache.org/jira/browse/HBASE-22117">HBASE-22117</a>
2012   */
2013  @Deprecated
2014  @Override
2015  public void checkPermissions(RpcController controller,
2016    AccessControlProtos.CheckPermissionsRequest request,
2017    RpcCallback<AccessControlProtos.CheckPermissionsResponse> done) {
2018    AccessControlProtos.CheckPermissionsResponse response = null;
2019    try {
2020      User user = RpcServer.getRequestUser().orElse(null);
2021      TableName tableName = regionEnv.getRegion().getTableDescriptor().getTableName();
2022      List<Permission> permissions = new ArrayList<>();
2023      for (int i = 0; i < request.getPermissionCount(); i++) {
2024        Permission permission = AccessControlUtil.toPermission(request.getPermission(i));
2025        permissions.add(permission);
2026        if (permission instanceof TablePermission) {
2027          TablePermission tperm = (TablePermission) permission;
2028          if (!tperm.getTableName().equals(tableName)) {
2029            throw new CoprocessorException(AccessController.class,
2030              String.format(
2031                "This method can only execute at the table specified in "
2032                  + "TablePermission. Table of the region:%s , requested table:%s",
2033                tableName, tperm.getTableName()));
2034          }
2035        }
2036      }
2037      for (Permission permission : permissions) {
2038        boolean hasPermission =
2039          accessChecker.hasUserPermission(user, "checkPermissions", permission);
2040        if (!hasPermission) {
2041          throw new AccessDeniedException("Insufficient permissions " + permission.toString());
2042        }
2043      }
2044      response = AccessControlProtos.CheckPermissionsResponse.getDefaultInstance();
2045    } catch (IOException ioe) {
2046      CoprocessorRpcUtils.setControllerException(controller, ioe);
2047    }
2048    done.run(response);
2049  }
2050
2051  private Region getRegion(RegionCoprocessorEnvironment e) {
2052    return e.getRegion();
2053  }
2054
2055  private TableName getTableName(RegionCoprocessorEnvironment e) {
2056    Region region = e.getRegion();
2057    if (region != null) {
2058      return getTableName(region);
2059    }
2060    return null;
2061  }
2062
2063  private TableName getTableName(Region region) {
2064    RegionInfo regionInfo = region.getRegionInfo();
2065    if (regionInfo != null) {
2066      return regionInfo.getTable();
2067    }
2068    return null;
2069  }
2070
2071  @Override
2072  public void preClose(ObserverContext<RegionCoprocessorEnvironment> c, boolean abortRequested)
2073    throws IOException {
2074    requirePermission(c, "preClose", Action.ADMIN);
2075  }
2076
2077  private void checkSystemOrSuperUser(User activeUser) throws IOException {
2078    // No need to check if we're not going to throw
2079    if (!authorizationEnabled) {
2080      return;
2081    }
2082    if (!Superusers.isSuperUser(activeUser)) {
2083      throw new AccessDeniedException(
2084        "User '" + (activeUser != null ? activeUser.getShortName() : "null")
2085          + "' is not system or super user.");
2086    }
2087  }
2088
2089  @Override
2090  public void preStopRegionServer(ObserverContext<RegionServerCoprocessorEnvironment> ctx)
2091    throws IOException {
2092    requirePermission(ctx, "preStopRegionServer", Action.ADMIN);
2093  }
2094
2095  private Map<byte[], ? extends Collection<byte[]>> makeFamilyMap(byte[] family, byte[] qualifier) {
2096    if (family == null) {
2097      return null;
2098    }
2099
2100    Map<byte[], Collection<byte[]>> familyMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
2101    familyMap.put(family, qualifier != null ? ImmutableSet.of(qualifier) : null);
2102    return familyMap;
2103  }
2104
2105  @Override
2106  public void preGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
2107    List<TableName> tableNamesList, List<TableDescriptor> descriptors, String regex)
2108    throws IOException {
2109    // We are delegating the authorization check to postGetTableDescriptors as we don't have
2110    // any concrete set of table names when a regex is present or the full list is requested.
2111    if (regex == null && tableNamesList != null && !tableNamesList.isEmpty()) {
2112      // Otherwise, if the requestor has ADMIN or CREATE privs for all listed tables, the
2113      // request can be granted.
2114      try (Admin admin = ctx.getEnvironment().getConnection().getAdmin()) {
2115        for (TableName tableName : tableNamesList) {
2116          // Skip checks for a table that does not exist
2117          if (!admin.tableExists(tableName)) {
2118            continue;
2119          }
2120          requirePermission(ctx, "getTableDescriptors", tableName, null, null, Action.ADMIN,
2121            Action.CREATE);
2122        }
2123      }
2124    }
2125  }
2126
2127  @Override
2128  public void postGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
2129    List<TableName> tableNamesList, List<TableDescriptor> descriptors, String regex)
2130    throws IOException {
2131    // Skipping as checks in this case are already done by preGetTableDescriptors.
2132    if (regex == null && tableNamesList != null && !tableNamesList.isEmpty()) {
2133      return;
2134    }
2135
2136    // Retains only those which passes authorization checks, as the checks weren't done as part
2137    // of preGetTableDescriptors.
2138    Iterator<TableDescriptor> itr = descriptors.iterator();
2139    while (itr.hasNext()) {
2140      TableDescriptor htd = itr.next();
2141      try {
2142        requirePermission(ctx, "getTableDescriptors", htd.getTableName(), null, null, Action.ADMIN,
2143          Action.CREATE);
2144      } catch (AccessDeniedException e) {
2145        itr.remove();
2146      }
2147    }
2148  }
2149
2150  @Override
2151  public void postGetTableNames(ObserverContext<MasterCoprocessorEnvironment> ctx,
2152    List<TableDescriptor> descriptors, String regex) throws IOException {
2153    // Retains only those which passes authorization checks.
2154    Iterator<TableDescriptor> itr = descriptors.iterator();
2155    while (itr.hasNext()) {
2156      TableDescriptor htd = itr.next();
2157      try {
2158        requireAccess(ctx, "getTableNames", htd.getTableName(), Action.values());
2159      } catch (AccessDeniedException e) {
2160        itr.remove();
2161      }
2162    }
2163  }
2164
2165  @Override
2166  public void preMergeRegions(final ObserverContext<MasterCoprocessorEnvironment> ctx,
2167    final RegionInfo[] regionsToMerge) throws IOException {
2168    requirePermission(ctx, "mergeRegions", regionsToMerge[0].getTable(), null, null, Action.ADMIN);
2169  }
2170
2171  @Override
2172  public void preRollWALWriterRequest(ObserverContext<RegionServerCoprocessorEnvironment> ctx)
2173    throws IOException {
2174    requirePermission(ctx, "preRollLogWriterRequest", Permission.Action.ADMIN);
2175  }
2176
2177  @Override
2178  public void postRollWALWriterRequest(ObserverContext<RegionServerCoprocessorEnvironment> ctx)
2179    throws IOException {
2180  }
2181
2182  @Override
2183  public void preSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
2184    final String userName, final GlobalQuotaSettings quotas) throws IOException {
2185    requirePermission(ctx, "setUserQuota", Action.ADMIN);
2186  }
2187
2188  @Override
2189  public void preSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
2190    final String userName, final TableName tableName, final GlobalQuotaSettings quotas)
2191    throws IOException {
2192    requirePermission(ctx, "setUserTableQuota", tableName, null, null, Action.ADMIN);
2193  }
2194
2195  @Override
2196  public void preSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
2197    final String userName, final String namespace, final GlobalQuotaSettings quotas)
2198    throws IOException {
2199    requirePermission(ctx, "setUserNamespaceQuota", Action.ADMIN);
2200  }
2201
2202  @Override
2203  public void preSetTableQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
2204    final TableName tableName, final GlobalQuotaSettings quotas) throws IOException {
2205    requirePermission(ctx, "setTableQuota", tableName, null, null, Action.ADMIN);
2206  }
2207
2208  @Override
2209  public void preSetNamespaceQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
2210    final String namespace, final GlobalQuotaSettings quotas) throws IOException {
2211    requirePermission(ctx, "setNamespaceQuota", Action.ADMIN);
2212  }
2213
2214  @Override
2215  public void preSetRegionServerQuota(ObserverContext<MasterCoprocessorEnvironment> ctx,
2216    final String regionServer, GlobalQuotaSettings quotas) throws IOException {
2217    requirePermission(ctx, "setRegionServerQuota", Action.ADMIN);
2218  }
2219
2220  @Override
2221  public ReplicationEndpoint postCreateReplicationEndPoint(
2222    ObserverContext<RegionServerCoprocessorEnvironment> ctx, ReplicationEndpoint endpoint) {
2223    return endpoint;
2224  }
2225
2226  @Override
2227  public void preReplicateLogEntries(ObserverContext<RegionServerCoprocessorEnvironment> ctx)
2228    throws IOException {
2229    requirePermission(ctx, "replicateLogEntries", Action.WRITE);
2230  }
2231
2232  @Override
2233  public void preClearCompactionQueues(ObserverContext<RegionServerCoprocessorEnvironment> ctx)
2234    throws IOException {
2235    requirePermission(ctx, "preClearCompactionQueues", Permission.Action.ADMIN);
2236  }
2237
2238  @Override
2239  public void preAddReplicationPeer(final ObserverContext<MasterCoprocessorEnvironment> ctx,
2240    String peerId, ReplicationPeerConfig peerConfig) throws IOException {
2241    requirePermission(ctx, "addReplicationPeer", Action.ADMIN);
2242  }
2243
2244  @Override
2245  public void preRemoveReplicationPeer(final ObserverContext<MasterCoprocessorEnvironment> ctx,
2246    String peerId) throws IOException {
2247    requirePermission(ctx, "removeReplicationPeer", Action.ADMIN);
2248  }
2249
2250  @Override
2251  public void preEnableReplicationPeer(final ObserverContext<MasterCoprocessorEnvironment> ctx,
2252    String peerId) throws IOException {
2253    requirePermission(ctx, "enableReplicationPeer", Action.ADMIN);
2254  }
2255
2256  @Override
2257  public void preDisableReplicationPeer(final ObserverContext<MasterCoprocessorEnvironment> ctx,
2258    String peerId) throws IOException {
2259    requirePermission(ctx, "disableReplicationPeer", Action.ADMIN);
2260  }
2261
2262  @Override
2263  public void preGetReplicationPeerConfig(final ObserverContext<MasterCoprocessorEnvironment> ctx,
2264    String peerId) throws IOException {
2265    requirePermission(ctx, "getReplicationPeerConfig", Action.ADMIN);
2266  }
2267
2268  @Override
2269  public void preUpdateReplicationPeerConfig(
2270    final ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId,
2271    ReplicationPeerConfig peerConfig) throws IOException {
2272    requirePermission(ctx, "updateReplicationPeerConfig", Action.ADMIN);
2273  }
2274
2275  @Override
2276  public void preTransitReplicationPeerSyncReplicationState(
2277    final ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId,
2278    SyncReplicationState clusterState) throws IOException {
2279    requirePermission(ctx, "transitSyncReplicationPeerState", Action.ADMIN);
2280  }
2281
2282  @Override
2283  public void preListReplicationPeers(final ObserverContext<MasterCoprocessorEnvironment> ctx,
2284    String regex) throws IOException {
2285    requirePermission(ctx, "listReplicationPeers", Action.ADMIN);
2286  }
2287
2288  @Override
2289  public void preRequestLock(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace,
2290    TableName tableName, RegionInfo[] regionInfos, String description) throws IOException {
2291    // There are operations in the CREATE and ADMIN domain which may require lock, READ
2292    // or WRITE. So for any lock request, we check for these two perms irrespective of lock type.
2293    String reason = String.format("Description=%s", description);
2294    checkLockPermissions(ctx, namespace, tableName, regionInfos, reason);
2295  }
2296
2297  @Override
2298  public void preLockHeartbeat(ObserverContext<MasterCoprocessorEnvironment> ctx,
2299    TableName tableName, String description) throws IOException {
2300    checkLockPermissions(ctx, null, tableName, null, description);
2301  }
2302
2303  @Override
2304  public void preExecuteProcedures(ObserverContext<RegionServerCoprocessorEnvironment> ctx)
2305    throws IOException {
2306    checkSystemOrSuperUser(getActiveUser(ctx));
2307  }
2308
2309  @Override
2310  public void preSwitchRpcThrottle(ObserverContext<MasterCoprocessorEnvironment> ctx,
2311    boolean enable) throws IOException {
2312    requirePermission(ctx, "switchRpcThrottle", Action.ADMIN);
2313  }
2314
2315  @Override
2316  public void preIsRpcThrottleEnabled(ObserverContext<MasterCoprocessorEnvironment> ctx)
2317    throws IOException {
2318    requirePermission(ctx, "isRpcThrottleEnabled", Action.ADMIN);
2319  }
2320
2321  @Override
2322  public void preSwitchExceedThrottleQuota(ObserverContext<MasterCoprocessorEnvironment> ctx,
2323    boolean enable) throws IOException {
2324    requirePermission(ctx, "switchExceedThrottleQuota", Action.ADMIN);
2325  }
2326
2327  /**
2328   * Returns the active user to which authorization checks should be applied. If we are in the
2329   * context of an RPC call, the remote user is used, otherwise the currently logged in user is
2330   * used.
2331   */
2332  private User getActiveUser(ObserverContext<?> ctx) throws IOException {
2333    // for non-rpc handling, fallback to system user
2334    Optional<User> optionalUser = ctx.getCaller();
2335    if (optionalUser.isPresent()) {
2336      return optionalUser.get();
2337    }
2338    return userProvider.getCurrent();
2339  }
2340
2341  /**
2342   * @deprecated since 2.2.0 and will be removed in 4.0.0. Use
2343   *             {@link Admin#hasUserPermissions(String, List)} instead.
2344   * @see Admin#hasUserPermissions(String, List)
2345   * @see <a href="https://issues.apache.org/jira/browse/HBASE-22117">HBASE-22117</a>
2346   */
2347  @Deprecated
2348  @Override
2349  public void hasPermission(RpcController controller, HasPermissionRequest request,
2350    RpcCallback<HasPermissionResponse> done) {
2351    // Converts proto to a TablePermission object.
2352    TablePermission tPerm = AccessControlUtil.toTablePermission(request.getTablePermission());
2353    // Check input user name
2354    if (!request.hasUserName()) {
2355      throw new IllegalStateException("Input username cannot be empty");
2356    }
2357    final String inputUserName = request.getUserName().toStringUtf8();
2358    AccessControlProtos.HasPermissionResponse response = null;
2359    try {
2360      User caller = RpcServer.getRequestUser().orElse(null);
2361      List<Permission> permissions = Lists.newArrayList(tPerm);
2362      preHasUserPermissions(caller, inputUserName, permissions);
2363      boolean hasPermission =
2364        regionEnv.getConnection().getAdmin().hasUserPermissions(inputUserName, permissions).get(0);
2365      response = ResponseConverter.buildHasPermissionResponse(hasPermission);
2366    } catch (IOException ioe) {
2367      ResponseConverter.setControllerException(controller, ioe);
2368    }
2369    done.run(response);
2370  }
2371
2372  @Override
2373  public void preGrant(ObserverContext<MasterCoprocessorEnvironment> ctx,
2374    UserPermission userPermission, boolean mergeExistingPermissions) throws IOException {
2375    preGrantOrRevoke(getActiveUser(ctx), "grant", userPermission);
2376  }
2377
2378  @Override
2379  public void preRevoke(ObserverContext<MasterCoprocessorEnvironment> ctx,
2380    UserPermission userPermission) throws IOException {
2381    preGrantOrRevoke(getActiveUser(ctx), "revoke", userPermission);
2382  }
2383
2384  private void preGrantOrRevoke(User caller, String request, UserPermission userPermission)
2385    throws IOException {
2386    switch (userPermission.getPermission().scope) {
2387      case GLOBAL:
2388        accessChecker.requireGlobalPermission(caller, request, Action.ADMIN, "");
2389        break;
2390      case NAMESPACE:
2391        NamespacePermission namespacePerm = (NamespacePermission) userPermission.getPermission();
2392        accessChecker.requireNamespacePermission(caller, request, namespacePerm.getNamespace(),
2393          null, Action.ADMIN);
2394        break;
2395      case TABLE:
2396        TablePermission tablePerm = (TablePermission) userPermission.getPermission();
2397        accessChecker.requirePermission(caller, request, tablePerm.getTableName(),
2398          tablePerm.getFamily(), tablePerm.getQualifier(), null, Action.ADMIN);
2399        break;
2400      default:
2401    }
2402    if (!Superusers.isSuperUser(caller)) {
2403      accessChecker.performOnSuperuser(request, caller, userPermission.getUser());
2404    }
2405  }
2406
2407  @Override
2408  public void preGetUserPermissions(ObserverContext<MasterCoprocessorEnvironment> ctx,
2409    String userName, String namespace, TableName tableName, byte[] family, byte[] qualifier)
2410    throws IOException {
2411    preGetUserPermissions(getActiveUser(ctx), userName, namespace, tableName, family, qualifier);
2412  }
2413
2414  private void preGetUserPermissions(User caller, String userName, String namespace,
2415    TableName tableName, byte[] family, byte[] qualifier) throws IOException {
2416    if (tableName != null) {
2417      accessChecker.requirePermission(caller, "getUserPermissions", tableName, family, qualifier,
2418        userName, Action.ADMIN);
2419    } else if (namespace != null) {
2420      accessChecker.requireNamespacePermission(caller, "getUserPermissions", namespace, userName,
2421        Action.ADMIN);
2422    } else {
2423      accessChecker.requirePermission(caller, "getUserPermissions", userName, Action.ADMIN);
2424    }
2425  }
2426
2427  @Override
2428  public void preHasUserPermissions(ObserverContext<MasterCoprocessorEnvironment> ctx,
2429    String userName, List<Permission> permissions) throws IOException {
2430    preHasUserPermissions(getActiveUser(ctx), userName, permissions);
2431  }
2432
2433  private void preHasUserPermissions(User caller, String userName, List<Permission> permissions)
2434    throws IOException {
2435    String request = "hasUserPermissions";
2436    for (Permission permission : permissions) {
2437      if (!caller.getShortName().equals(userName)) {
2438        // User should have admin privilege if checking permission for other users
2439        if (permission instanceof TablePermission) {
2440          TablePermission tPerm = (TablePermission) permission;
2441          accessChecker.requirePermission(caller, request, tPerm.getTableName(), tPerm.getFamily(),
2442            tPerm.getQualifier(), userName, Action.ADMIN);
2443        } else if (permission instanceof NamespacePermission) {
2444          NamespacePermission nsPerm = (NamespacePermission) permission;
2445          accessChecker.requireNamespacePermission(caller, request, nsPerm.getNamespace(), userName,
2446            Action.ADMIN);
2447        } else {
2448          accessChecker.requirePermission(caller, request, userName, Action.ADMIN);
2449        }
2450      } else {
2451        // User don't need ADMIN privilege for self check.
2452        // Setting action as null in AuthResult to display empty action in audit log
2453        AuthResult result;
2454        if (permission instanceof TablePermission) {
2455          TablePermission tPerm = (TablePermission) permission;
2456          result = AuthResult.allow(request, "Self user validation allowed", caller, null,
2457            tPerm.getTableName(), tPerm.getFamily(), tPerm.getQualifier());
2458        } else if (permission instanceof NamespacePermission) {
2459          NamespacePermission nsPerm = (NamespacePermission) permission;
2460          result = AuthResult.allow(request, "Self user validation allowed", caller, null,
2461            nsPerm.getNamespace());
2462        } else {
2463          result = AuthResult.allow(request, "Self user validation allowed", caller, null, null,
2464            null, null);
2465        }
2466        AccessChecker.logResult(result);
2467      }
2468    }
2469  }
2470
2471  @Override
2472  public void preMoveServersAndTables(ObserverContext<MasterCoprocessorEnvironment> ctx,
2473    Set<Address> servers, Set<TableName> tables, String targetGroup) throws IOException {
2474    accessChecker.requirePermission(getActiveUser(ctx), "moveServersAndTables", null,
2475      Permission.Action.ADMIN);
2476  }
2477
2478  @Override
2479  public void preMoveServers(final ObserverContext<MasterCoprocessorEnvironment> ctx,
2480    Set<Address> servers, String targetGroup) throws IOException {
2481    accessChecker.requirePermission(getActiveUser(ctx), "moveServers", null,
2482      Permission.Action.ADMIN);
2483  }
2484
2485  @Override
2486  public void preMoveTables(ObserverContext<MasterCoprocessorEnvironment> ctx,
2487    Set<TableName> tables, String targetGroup) throws IOException {
2488    accessChecker.requirePermission(getActiveUser(ctx), "moveTables", null,
2489      Permission.Action.ADMIN);
2490  }
2491
2492  @Override
2493  public void preAddRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String name)
2494    throws IOException {
2495    accessChecker.requirePermission(getActiveUser(ctx), "addRSGroup", null,
2496      Permission.Action.ADMIN);
2497  }
2498
2499  @Override
2500  public void preRemoveRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String name)
2501    throws IOException {
2502    accessChecker.requirePermission(getActiveUser(ctx), "removeRSGroup", null,
2503      Permission.Action.ADMIN);
2504  }
2505
2506  @Override
2507  public void preBalanceRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String groupName,
2508    BalanceRequest request) throws IOException {
2509    accessChecker.requirePermission(getActiveUser(ctx), "balanceRSGroup", null,
2510      Permission.Action.ADMIN);
2511  }
2512
2513  @Override
2514  public void preRemoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx,
2515    Set<Address> servers) throws IOException {
2516    accessChecker.requirePermission(getActiveUser(ctx), "removeServers", null,
2517      Permission.Action.ADMIN);
2518  }
2519
2520  @Override
2521  public void preGetRSGroupInfo(ObserverContext<MasterCoprocessorEnvironment> ctx, String groupName)
2522    throws IOException {
2523    accessChecker.requirePermission(getActiveUser(ctx), "getRSGroupInfo", null,
2524      Permission.Action.ADMIN);
2525  }
2526
2527  @Override
2528  public void preGetRSGroupInfoOfTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
2529    TableName tableName) throws IOException {
2530    accessChecker.requirePermission(getActiveUser(ctx), "getRSGroupInfoOfTable", null,
2531      Permission.Action.ADMIN);
2532    // todo: should add check for table existence
2533  }
2534
2535  @Override
2536  public void preListRSGroups(ObserverContext<MasterCoprocessorEnvironment> ctx)
2537    throws IOException {
2538    accessChecker.requirePermission(getActiveUser(ctx), "listRSGroups", null,
2539      Permission.Action.ADMIN);
2540  }
2541
2542  @Override
2543  public void preListTablesInRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
2544    String groupName) throws IOException {
2545    accessChecker.requirePermission(getActiveUser(ctx), "listTablesInRSGroup", null,
2546      Permission.Action.ADMIN);
2547  }
2548
2549  @Override
2550  public void preGetConfiguredNamespacesAndTablesInRSGroup(
2551    ObserverContext<MasterCoprocessorEnvironment> ctx, String groupName) throws IOException {
2552    accessChecker.requirePermission(getActiveUser(ctx), "getConfiguredNamespacesAndTablesInRSGroup",
2553      null, Permission.Action.ADMIN);
2554  }
2555
2556  @Override
2557  public void preGetRSGroupInfoOfServer(ObserverContext<MasterCoprocessorEnvironment> ctx,
2558    Address server) throws IOException {
2559    accessChecker.requirePermission(getActiveUser(ctx), "getRSGroupInfoOfServer", null,
2560      Permission.Action.ADMIN);
2561  }
2562
2563  @Override
2564  public void preRenameRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String oldName,
2565    String newName) throws IOException {
2566    accessChecker.requirePermission(getActiveUser(ctx), "renameRSGroup", null,
2567      Permission.Action.ADMIN);
2568  }
2569
2570  @Override
2571  public void preUpdateRSGroupConfig(final ObserverContext<MasterCoprocessorEnvironment> ctx,
2572    final String groupName, final Map<String, String> configuration) throws IOException {
2573    accessChecker.requirePermission(getActiveUser(ctx), "updateRSGroupConfig", null,
2574      Permission.Action.ADMIN);
2575  }
2576}