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