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