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