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