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