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