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