001/* 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.security.visibility; 019 020import static org.apache.hadoop.hbase.HConstants.OperationStatusCode.SANITY_CHECK_FAILURE; 021import static org.apache.hadoop.hbase.HConstants.OperationStatusCode.SUCCESS; 022import static org.apache.hadoop.hbase.security.visibility.VisibilityConstants.LABELS_TABLE_FAMILY; 023import static org.apache.hadoop.hbase.security.visibility.VisibilityConstants.LABELS_TABLE_NAME; 024 025import java.io.IOException; 026import java.net.InetAddress; 027import java.util.ArrayList; 028import java.util.Collections; 029import java.util.HashMap; 030import java.util.Iterator; 031import java.util.List; 032import java.util.Map; 033import java.util.Objects; 034import java.util.Optional; 035import org.apache.hadoop.conf.Configuration; 036import org.apache.hadoop.hbase.AuthUtil; 037import org.apache.hadoop.hbase.Cell; 038import org.apache.hadoop.hbase.CoprocessorEnvironment; 039import org.apache.hadoop.hbase.DoNotRetryIOException; 040import org.apache.hadoop.hbase.ExtendedCell; 041import org.apache.hadoop.hbase.ExtendedCellScanner; 042import org.apache.hadoop.hbase.HBaseInterfaceAudience; 043import org.apache.hadoop.hbase.PrivateCellUtil; 044import org.apache.hadoop.hbase.TableName; 045import org.apache.hadoop.hbase.Tag; 046import org.apache.hadoop.hbase.TagType; 047import org.apache.hadoop.hbase.client.Admin; 048import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; 049import org.apache.hadoop.hbase.client.Delete; 050import org.apache.hadoop.hbase.client.Get; 051import org.apache.hadoop.hbase.client.MasterSwitchType; 052import org.apache.hadoop.hbase.client.Mutation; 053import org.apache.hadoop.hbase.client.Put; 054import org.apache.hadoop.hbase.client.Result; 055import org.apache.hadoop.hbase.client.Scan; 056import org.apache.hadoop.hbase.client.TableDescriptor; 057import org.apache.hadoop.hbase.client.TableDescriptorBuilder; 058import org.apache.hadoop.hbase.constraint.ConstraintException; 059import org.apache.hadoop.hbase.coprocessor.CoprocessorException; 060import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; 061import org.apache.hadoop.hbase.coprocessor.CoreCoprocessor; 062import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor; 063import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment; 064import org.apache.hadoop.hbase.coprocessor.MasterObserver; 065import org.apache.hadoop.hbase.coprocessor.ObserverContext; 066import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor; 067import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; 068import org.apache.hadoop.hbase.coprocessor.RegionObserver; 069import org.apache.hadoop.hbase.exceptions.DeserializationException; 070import org.apache.hadoop.hbase.filter.Filter; 071import org.apache.hadoop.hbase.filter.FilterBase; 072import org.apache.hadoop.hbase.filter.FilterList; 073import org.apache.hadoop.hbase.io.hfile.HFile; 074import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils; 075import org.apache.hadoop.hbase.ipc.RpcServer; 076import org.apache.hadoop.hbase.regionserver.BloomType; 077import org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy; 078import org.apache.hadoop.hbase.regionserver.InternalScanner; 079import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress; 080import org.apache.hadoop.hbase.regionserver.OperationStatus; 081import org.apache.hadoop.hbase.regionserver.Region; 082import org.apache.hadoop.hbase.regionserver.RegionScanner; 083import org.apache.hadoop.hbase.regionserver.querymatcher.DeleteTracker; 084import org.apache.hadoop.hbase.security.AccessDeniedException; 085import org.apache.hadoop.hbase.security.Superusers; 086import org.apache.hadoop.hbase.security.User; 087import org.apache.hadoop.hbase.security.access.AccessChecker; 088import org.apache.hadoop.hbase.security.access.AccessController; 089import org.apache.hadoop.hbase.util.Bytes; 090import org.apache.hadoop.hbase.util.Pair; 091import org.apache.hadoop.util.StringUtils; 092import org.apache.yetus.audience.InterfaceAudience; 093import org.slf4j.Logger; 094import org.slf4j.LoggerFactory; 095 096import org.apache.hbase.thirdparty.com.google.common.collect.Lists; 097import org.apache.hbase.thirdparty.com.google.common.collect.MapMaker; 098import org.apache.hbase.thirdparty.com.google.protobuf.ByteString; 099import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback; 100import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; 101import org.apache.hbase.thirdparty.com.google.protobuf.Service; 102 103import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult; 104import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair; 105import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos; 106import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.GetAuthsRequest; 107import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.GetAuthsResponse; 108import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.ListLabelsRequest; 109import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.ListLabelsResponse; 110import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.SetAuthsRequest; 111import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.VisibilityLabel; 112import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsRequest; 113import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse; 114import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsService; 115 116/** 117 * Coprocessor that has both the MasterObserver and RegionObserver implemented that supports in 118 * visibility labels 119 */ 120@CoreCoprocessor 121@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG) 122// TODO: break out Observer functions into separate class/sub-class. 123public class VisibilityController implements MasterCoprocessor, RegionCoprocessor, 124 VisibilityLabelsService.Interface, MasterObserver, RegionObserver { 125 126 private static final Logger LOG = LoggerFactory.getLogger(VisibilityController.class); 127 private static final Logger AUDITLOG = 128 LoggerFactory.getLogger("SecurityLogger." + VisibilityController.class.getName()); 129 // flags if we are running on a region of the 'labels' table 130 private boolean labelsRegion = false; 131 // Flag denoting whether AcessController is available or not. 132 private boolean accessControllerAvailable = false; 133 private Configuration conf; 134 private volatile boolean initialized = false; 135 private boolean checkAuths = false; 136 /** Mapping of scanner instances to the user who created them */ 137 private Map<InternalScanner, String> scannerOwners = new MapMaker().weakKeys().makeMap(); 138 139 private VisibilityLabelService visibilityLabelService; 140 141 /** 142 * if we are active, usually false, only true if "hbase.security.authorization" has been set to 143 * true in site configuration 144 */ 145 boolean authorizationEnabled; 146 147 // Add to this list if there are any reserved tag types 148 private static ArrayList<Byte> RESERVED_VIS_TAG_TYPES = new ArrayList<>(); 149 static { 150 RESERVED_VIS_TAG_TYPES.add(TagType.VISIBILITY_TAG_TYPE); 151 RESERVED_VIS_TAG_TYPES.add(TagType.VISIBILITY_EXP_SERIALIZATION_FORMAT_TAG_TYPE); 152 RESERVED_VIS_TAG_TYPES.add(TagType.STRING_VIS_TAG_TYPE); 153 } 154 155 public static boolean isCellAuthorizationSupported(Configuration conf) { 156 return AccessChecker.isAuthorizationSupported(conf); 157 } 158 159 @Override 160 public void start(CoprocessorEnvironment env) throws IOException { 161 this.conf = env.getConfiguration(); 162 163 authorizationEnabled = AccessChecker.isAuthorizationSupported(conf); 164 if (!authorizationEnabled) { 165 LOG.warn("The VisibilityController has been loaded with authorization checks disabled."); 166 } 167 168 if (HFile.getFormatVersion(conf) < HFile.MIN_FORMAT_VERSION_WITH_TAGS) { 169 throw new RuntimeException("A minimum HFile version of " + HFile.MIN_FORMAT_VERSION_WITH_TAGS 170 + " is required to persist visibility labels. Consider setting " + HFile.FORMAT_VERSION_KEY 171 + " accordingly."); 172 } 173 174 // Do not create for master CPs 175 if (!(env instanceof MasterCoprocessorEnvironment)) { 176 visibilityLabelService = 177 VisibilityLabelServiceManager.getInstance().getVisibilityLabelService(this.conf); 178 } 179 } 180 181 @Override 182 public void stop(CoprocessorEnvironment env) throws IOException { 183 184 } 185 186 /**************************** Observer/Service Getters ************************************/ 187 @Override 188 public Optional<RegionObserver> getRegionObserver() { 189 return Optional.of(this); 190 } 191 192 @Override 193 public Optional<MasterObserver> getMasterObserver() { 194 return Optional.of(this); 195 } 196 197 @Override 198 public Iterable<Service> getServices() { 199 return Collections 200 .singleton(VisibilityLabelsProtos.VisibilityLabelsService.newReflectiveService(this)); 201 } 202 203 /********************************* Master related hooks **********************************/ 204 205 @Override 206 public void postStartMaster(ObserverContext<MasterCoprocessorEnvironment> ctx) 207 throws IOException { 208 // Need to create the new system table for labels here 209 try (Admin admin = ctx.getEnvironment().getConnection().getAdmin()) { 210 if (!admin.tableExists(LABELS_TABLE_NAME)) { 211 // We will cache all the labels. No need of normal table block cache. 212 // Let the "labels" table having only one region always. We are not expecting too many 213 // labels in the system. 214 TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(LABELS_TABLE_NAME) 215 .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(LABELS_TABLE_FAMILY) 216 .setBloomFilterType(BloomType.NONE).setBlockCacheEnabled(false).build()) 217 .setValue(TableDescriptorBuilder.SPLIT_POLICY, DisabledRegionSplitPolicy.class.getName()) 218 .build(); 219 220 admin.createTable(tableDescriptor); 221 } 222 } 223 } 224 225 @Override 226 public TableDescriptor preModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx, 227 TableName tableName, TableDescriptor currentDescriptor, TableDescriptor newDescriptor) 228 throws IOException { 229 if (authorizationEnabled) { 230 if (LABELS_TABLE_NAME.equals(tableName)) { 231 throw new ConstraintException("Cannot alter " + LABELS_TABLE_NAME); 232 } 233 } 234 return newDescriptor; 235 } 236 237 @Override 238 public void preDisableTable(ObserverContext<MasterCoprocessorEnvironment> ctx, 239 TableName tableName) throws IOException { 240 if (!authorizationEnabled) { 241 return; 242 } 243 if (LABELS_TABLE_NAME.equals(tableName)) { 244 throw new ConstraintException("Cannot disable " + LABELS_TABLE_NAME); 245 } 246 } 247 248 /****************************** Region related hooks ******************************/ 249 250 @Override 251 public void postOpen(ObserverContext<? extends RegionCoprocessorEnvironment> e) { 252 // Read the entire labels table and populate the zk 253 if (e.getEnvironment().getRegion().getRegionInfo().getTable().equals(LABELS_TABLE_NAME)) { 254 this.labelsRegion = true; 255 synchronized (this) { 256 this.accessControllerAvailable = 257 CoprocessorHost.getLoadedCoprocessors().contains(AccessController.class.getName()); 258 } 259 initVisibilityLabelService(e.getEnvironment()); 260 } else { 261 checkAuths = e.getEnvironment().getConfiguration() 262 .getBoolean(VisibilityConstants.CHECK_AUTHS_FOR_MUTATION, false); 263 initVisibilityLabelService(e.getEnvironment()); 264 } 265 } 266 267 private void initVisibilityLabelService(RegionCoprocessorEnvironment env) { 268 try { 269 this.visibilityLabelService.init(env); 270 this.initialized = true; 271 } catch (IOException ioe) { 272 LOG.error("Error while initializing VisibilityLabelService..", ioe); 273 throw new RuntimeException(ioe); 274 } 275 } 276 277 @Override 278 public void postSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx, 279 final boolean newValue, final MasterSwitchType switchType) throws IOException { 280 } 281 282 @Override 283 public void preBatchMutate(ObserverContext<? extends RegionCoprocessorEnvironment> c, 284 MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException { 285 if (c.getEnvironment().getRegion().getRegionInfo().getTable().isSystemTable()) { 286 return; 287 } 288 // TODO this can be made as a global LRU cache at HRS level? 289 Map<String, List<Tag>> labelCache = new HashMap<>(); 290 for (int i = 0; i < miniBatchOp.size(); i++) { 291 Mutation m = miniBatchOp.getOperation(i); 292 CellVisibility cellVisibility = null; 293 try { 294 cellVisibility = m.getCellVisibility(); 295 } catch (DeserializationException de) { 296 miniBatchOp.setOperationStatus(i, 297 new OperationStatus(SANITY_CHECK_FAILURE, de.getMessage())); 298 continue; 299 } 300 boolean sanityFailure = false; 301 boolean modifiedTagFound = false; 302 Pair<Boolean, Tag> pair = new Pair<>(false, null); 303 for (ExtendedCellScanner cellScanner = m.cellScanner(); cellScanner.advance();) { 304 ExtendedCell cell = cellScanner.current(); 305 pair = checkForReservedVisibilityTagPresence(cell, pair); 306 if (!pair.getFirst()) { 307 // Don't disallow reserved tags if authorization is disabled 308 if (authorizationEnabled) { 309 miniBatchOp.setOperationStatus(i, new OperationStatus(SANITY_CHECK_FAILURE, 310 "Mutation contains cell with reserved type tag")); 311 sanityFailure = true; 312 } 313 break; 314 } else { 315 // Indicates that the cell has a the tag which was modified in the src replication cluster 316 Tag tag = pair.getSecond(); 317 if (cellVisibility == null && tag != null) { 318 // May need to store only the first one 319 cellVisibility = new CellVisibility(Tag.getValueAsString(tag)); 320 modifiedTagFound = true; 321 } 322 } 323 } 324 if (!sanityFailure && (m instanceof Put || m instanceof Delete)) { 325 if (cellVisibility != null) { 326 String labelsExp = cellVisibility.getExpression(); 327 List<Tag> visibilityTags = labelCache.get(labelsExp); 328 if (visibilityTags == null) { 329 // Don't check user auths for labels with Mutations when the user is super user 330 boolean authCheck = authorizationEnabled && checkAuths && !(isSystemOrSuperUser()); 331 try { 332 visibilityTags = 333 this.visibilityLabelService.createVisibilityExpTags(labelsExp, true, authCheck); 334 } catch (InvalidLabelException e) { 335 miniBatchOp.setOperationStatus(i, 336 new OperationStatus(SANITY_CHECK_FAILURE, e.getMessage())); 337 } 338 if (visibilityTags != null) { 339 labelCache.put(labelsExp, visibilityTags); 340 } 341 } 342 if (visibilityTags != null) { 343 List<ExtendedCell> updatedCells = new ArrayList<>(); 344 for (ExtendedCellScanner cellScanner = m.cellScanner(); cellScanner.advance();) { 345 ExtendedCell cell = cellScanner.current(); 346 List<Tag> tags = PrivateCellUtil.getTags(cell); 347 if (modifiedTagFound) { 348 // Rewrite the tags by removing the modified tags. 349 removeReplicationVisibilityTag(tags); 350 } 351 tags.addAll(visibilityTags); 352 ExtendedCell updatedCell = PrivateCellUtil.createCell(cell, tags); 353 updatedCells.add(updatedCell); 354 } 355 m.getFamilyCellMap().clear(); 356 // Clear and add new Cells to the Mutation. 357 for (ExtendedCell cell : updatedCells) { 358 if (m instanceof Put) { 359 Put p = (Put) m; 360 p.add(cell); 361 } else { 362 Delete d = (Delete) m; 363 d.add(cell); 364 } 365 } 366 } 367 } 368 } 369 } 370 } 371 372 @Override 373 public void prePrepareTimeStampForDeleteVersion( 374 ObserverContext<? extends RegionCoprocessorEnvironment> ctx, Mutation delete, Cell cell, 375 byte[] byteNow, Get get) throws IOException { 376 // Nothing to do if we are not filtering by visibility 377 if (!authorizationEnabled) { 378 return; 379 } 380 381 CellVisibility cellVisibility = null; 382 try { 383 cellVisibility = delete.getCellVisibility(); 384 } catch (DeserializationException de) { 385 throw new IOException("Invalid cell visibility specified " + delete, de); 386 } 387 // The check for checkForReservedVisibilityTagPresence happens in preBatchMutate happens. 388 // It happens for every mutation and that would be enough. 389 List<Tag> visibilityTags = new ArrayList<>(); 390 if (cellVisibility != null) { 391 String labelsExp = cellVisibility.getExpression(); 392 try { 393 visibilityTags = 394 this.visibilityLabelService.createVisibilityExpTags(labelsExp, false, false); 395 } catch (InvalidLabelException e) { 396 throw new IOException("Invalid cell visibility specified " + labelsExp, e); 397 } 398 } 399 get.setFilter(new DeleteVersionVisibilityExpressionFilter(visibilityTags, 400 VisibilityConstants.SORTED_ORDINAL_SERIALIZATION_FORMAT)); 401 try (RegionScanner scanner = ctx.getEnvironment().getRegion().getScanner(new Scan(get))) { 402 // NOTE: Please don't use HRegion.get() instead, 403 // because it will copy cells to heap. See HBASE-26036 404 List<Cell> result = new ArrayList<>(); 405 scanner.next(result); 406 407 if (result.size() < get.getMaxVersions()) { 408 // Nothing to delete 409 PrivateCellUtil.updateLatestStamp(cell, byteNow); 410 return; 411 } 412 if (result.size() > get.getMaxVersions()) { 413 throw new RuntimeException( 414 "Unexpected size: " + result.size() + ". Results more than the max versions obtained."); 415 } 416 Cell getCell = result.get(get.getMaxVersions() - 1); 417 PrivateCellUtil.setTimestamp(cell, getCell.getTimestamp()); 418 } 419 // We are bypassing here because in the HRegion.updateDeleteLatestVersionTimeStamp we would 420 // update with the current timestamp after again doing a get. As the hook as already determined 421 // the needed timestamp we need to bypass here. 422 // TODO : See if HRegion.updateDeleteLatestVersionTimeStamp() could be 423 // called only if the hook is not called. 424 ctx.bypass(); 425 } 426 427 /** 428 * Checks whether cell contains any tag with type as VISIBILITY_TAG_TYPE. This tag type is 429 * reserved and should not be explicitly set by user. 430 * @param cell The cell under consideration 431 * @param pair An optional pair of type {@code <Boolean, Tag>} which would be reused if already 432 * set and new one will be created if NULL is passed 433 * @return If the boolean is false then it indicates that the cell has a RESERVERD_VIS_TAG and 434 * with boolean as true, not null tag indicates that a string modified tag was found. 435 */ 436 private Pair<Boolean, Tag> checkForReservedVisibilityTagPresence(ExtendedCell cell, 437 Pair<Boolean, Tag> pair) throws IOException { 438 if (pair == null) { 439 pair = new Pair<>(false, null); 440 } else { 441 pair.setFirst(false); 442 pair.setSecond(null); 443 } 444 // Bypass this check when the operation is done by a system/super user. 445 // This is done because, while Replication, the Cells coming to the peer cluster with reserved 446 // typed tags and this is fine and should get added to the peer cluster table 447 if (isSystemOrSuperUser()) { 448 // Does the cell contain special tag which indicates that the replicated 449 // cell visiblilty tags 450 // have been modified 451 Tag modifiedTag = null; 452 Iterator<Tag> tagsIterator = PrivateCellUtil.tagsIterator(cell); 453 while (tagsIterator.hasNext()) { 454 Tag tag = tagsIterator.next(); 455 if (tag.getType() == TagType.STRING_VIS_TAG_TYPE) { 456 modifiedTag = tag; 457 break; 458 } 459 } 460 pair.setFirst(true); 461 pair.setSecond(modifiedTag); 462 return pair; 463 } 464 Iterator<Tag> tagsItr = PrivateCellUtil.tagsIterator(cell); 465 while (tagsItr.hasNext()) { 466 if (RESERVED_VIS_TAG_TYPES.contains(tagsItr.next().getType())) { 467 return pair; 468 } 469 } 470 pair.setFirst(true); 471 return pair; 472 } 473 474 private void removeReplicationVisibilityTag(List<Tag> tags) throws IOException { 475 Iterator<Tag> iterator = tags.iterator(); 476 while (iterator.hasNext()) { 477 Tag tag = iterator.next(); 478 if (tag.getType() == TagType.STRING_VIS_TAG_TYPE) { 479 iterator.remove(); 480 break; 481 } 482 } 483 } 484 485 @Override 486 public void preScannerOpen(ObserverContext<? extends RegionCoprocessorEnvironment> e, Scan scan) 487 throws IOException { 488 if (!initialized) { 489 throw new VisibilityControllerNotReadyException("VisibilityController not yet initialized!"); 490 } 491 // Nothing to do if authorization is not enabled 492 if (!authorizationEnabled) { 493 return; 494 } 495 Region region = e.getEnvironment().getRegion(); 496 Authorizations authorizations = null; 497 try { 498 authorizations = scan.getAuthorizations(); 499 } catch (DeserializationException de) { 500 throw new IOException(de); 501 } 502 if (authorizations == null) { 503 // No Authorizations present for this scan/Get! 504 // In case of system tables other than "labels" just scan with out visibility check and 505 // filtering. Checking visibility labels for META and NAMESPACE table is not needed. 506 TableName table = region.getRegionInfo().getTable(); 507 if (table.isSystemTable() && !table.equals(LABELS_TABLE_NAME)) { 508 return; 509 } 510 } 511 512 Filter visibilityLabelFilter = 513 VisibilityUtils.createVisibilityLabelFilter(region, authorizations); 514 if (visibilityLabelFilter != null) { 515 Filter filter = scan.getFilter(); 516 if (filter != null) { 517 scan.setFilter(new FilterList(filter, visibilityLabelFilter)); 518 } else { 519 scan.setFilter(visibilityLabelFilter); 520 } 521 } 522 } 523 524 @Override 525 public DeleteTracker postInstantiateDeleteTracker( 526 ObserverContext<? extends RegionCoprocessorEnvironment> ctx, DeleteTracker delTracker) 527 throws IOException { 528 // Nothing to do if we are not filtering by visibility 529 if (!authorizationEnabled) { 530 return delTracker; 531 } 532 Region region = ctx.getEnvironment().getRegion(); 533 TableName table = region.getRegionInfo().getTable(); 534 if (table.isSystemTable()) { 535 return delTracker; 536 } 537 // We are creating a new type of delete tracker here which is able to track 538 // the timestamps and also the 539 // visibility tags per cell. The covering cells are determined not only 540 // based on the delete type and ts 541 // but also on the visibility expression matching. 542 return new VisibilityScanDeleteTracker(delTracker.getCellComparator()); 543 } 544 545 @Override 546 public RegionScanner postScannerOpen( 547 final ObserverContext<? extends RegionCoprocessorEnvironment> c, final Scan scan, 548 final RegionScanner s) throws IOException { 549 User user = VisibilityUtils.getActiveUser(); 550 if (user != null && user.getShortName() != null) { 551 scannerOwners.put(s, user.getShortName()); 552 } 553 return s; 554 } 555 556 @Override 557 public boolean preScannerNext(final ObserverContext<? extends RegionCoprocessorEnvironment> c, 558 final InternalScanner s, final List<Result> result, final int limit, final boolean hasNext) 559 throws IOException { 560 requireScannerOwner(s); 561 return hasNext; 562 } 563 564 @Override 565 public void preScannerClose(final ObserverContext<? extends RegionCoprocessorEnvironment> c, 566 final InternalScanner s) throws IOException { 567 requireScannerOwner(s); 568 } 569 570 @Override 571 public void postScannerClose(final ObserverContext<? extends RegionCoprocessorEnvironment> c, 572 final InternalScanner s) throws IOException { 573 // clean up any associated owner mapping 574 scannerOwners.remove(s); 575 } 576 577 /** 578 * Verify, when servicing an RPC, that the caller is the scanner owner. If so, we assume that 579 * access control is correctly enforced based on the checks performed in preScannerOpen() 580 */ 581 private void requireScannerOwner(InternalScanner s) throws AccessDeniedException { 582 if (!RpcServer.isInRpcCallContext()) return; 583 String requestUName = RpcServer.getRequestUserName().orElse(null); 584 String owner = scannerOwners.get(s); 585 if (authorizationEnabled && owner != null && !owner.equals(requestUName)) { 586 throw new AccessDeniedException("User '" + requestUName + "' is not the scanner owner!"); 587 } 588 } 589 590 @Override 591 public void preGetOp(ObserverContext<? extends RegionCoprocessorEnvironment> e, Get get, 592 List<Cell> results) throws IOException { 593 if (!initialized) { 594 throw new VisibilityControllerNotReadyException("VisibilityController not yet initialized"); 595 } 596 // Nothing useful to do if authorization is not enabled 597 if (!authorizationEnabled) { 598 return; 599 } 600 Region region = e.getEnvironment().getRegion(); 601 Authorizations authorizations = null; 602 try { 603 authorizations = get.getAuthorizations(); 604 } catch (DeserializationException de) { 605 throw new IOException(de); 606 } 607 if (authorizations == null) { 608 // No Authorizations present for this scan/Get! 609 // In case of system tables other than "labels" just scan with out visibility check and 610 // filtering. Checking visibility labels for META and NAMESPACE table is not needed. 611 TableName table = region.getRegionInfo().getTable(); 612 if (table.isSystemTable() && !table.equals(LABELS_TABLE_NAME)) { 613 return; 614 } 615 } 616 Filter visibilityLabelFilter = 617 VisibilityUtils.createVisibilityLabelFilter(e.getEnvironment().getRegion(), authorizations); 618 if (visibilityLabelFilter != null) { 619 Filter filter = get.getFilter(); 620 if (filter != null) { 621 get.setFilter(new FilterList(filter, visibilityLabelFilter)); 622 } else { 623 get.setFilter(visibilityLabelFilter); 624 } 625 } 626 } 627 628 private boolean isSystemOrSuperUser() throws IOException { 629 return Superusers.isSuperUser(VisibilityUtils.getActiveUser()); 630 } 631 632 @Override 633 public List<Pair<Cell, Cell>> postIncrementBeforeWAL( 634 ObserverContext<? extends RegionCoprocessorEnvironment> ctx, Mutation mutation, 635 List<Pair<Cell, Cell>> cellPairs) throws IOException { 636 List<Pair<Cell, Cell>> resultPairs = new ArrayList<>(cellPairs.size()); 637 for (Pair<Cell, Cell> pair : cellPairs) { 638 resultPairs.add(new Pair<>(pair.getFirst(), 639 createNewCellWithTags(mutation, (ExtendedCell) pair.getSecond()))); 640 } 641 return resultPairs; 642 } 643 644 @Override 645 public List<Pair<Cell, Cell>> postAppendBeforeWAL( 646 ObserverContext<? extends RegionCoprocessorEnvironment> ctx, Mutation mutation, 647 List<Pair<Cell, Cell>> cellPairs) throws IOException { 648 List<Pair<Cell, Cell>> resultPairs = new ArrayList<>(cellPairs.size()); 649 for (Pair<Cell, Cell> pair : cellPairs) { 650 resultPairs.add(new Pair<>(pair.getFirst(), 651 createNewCellWithTags(mutation, (ExtendedCell) pair.getSecond()))); 652 } 653 return resultPairs; 654 } 655 656 private Cell createNewCellWithTags(Mutation mutation, ExtendedCell newCell) throws IOException { 657 List<Tag> tags = Lists.newArrayList(); 658 CellVisibility cellVisibility = null; 659 try { 660 cellVisibility = mutation.getCellVisibility(); 661 } catch (DeserializationException e) { 662 throw new IOException(e); 663 } 664 if (cellVisibility == null) { 665 return newCell; 666 } 667 // Prepend new visibility tags to a new list of tags for the cell 668 // Don't check user auths for labels with Mutations when the user is super user 669 boolean authCheck = authorizationEnabled && checkAuths && !(isSystemOrSuperUser()); 670 tags.addAll(this.visibilityLabelService.createVisibilityExpTags(cellVisibility.getExpression(), 671 true, authCheck)); 672 // Carry forward all other tags 673 Iterator<Tag> tagsItr = PrivateCellUtil.tagsIterator(newCell); 674 while (tagsItr.hasNext()) { 675 Tag tag = tagsItr.next(); 676 if ( 677 tag.getType() != TagType.VISIBILITY_TAG_TYPE 678 && tag.getType() != TagType.VISIBILITY_EXP_SERIALIZATION_FORMAT_TAG_TYPE 679 ) { 680 tags.add(tag); 681 } 682 } 683 684 return PrivateCellUtil.createCell(newCell, tags); 685 } 686 687 /****************************** 688 * VisibilityEndpoint service related methods 689 ******************************/ 690 @Override 691 public synchronized void addLabels(RpcController controller, VisibilityLabelsRequest request, 692 RpcCallback<VisibilityLabelsResponse> done) { 693 VisibilityLabelsResponse.Builder response = VisibilityLabelsResponse.newBuilder(); 694 List<VisibilityLabel> visLabels = request.getVisLabelList(); 695 if (!initialized) { 696 setExceptionResults(visLabels.size(), 697 new VisibilityControllerNotReadyException("VisibilityController not yet initialized!"), 698 response); 699 } else { 700 List<byte[]> labels = new ArrayList<>(visLabels.size()); 701 try { 702 if (authorizationEnabled) { 703 checkCallingUserAuth(); 704 } 705 RegionActionResult successResult = RegionActionResult.newBuilder().build(); 706 for (VisibilityLabel visLabel : visLabels) { 707 byte[] label = visLabel.getLabel().toByteArray(); 708 labels.add(label); 709 response.addResult(successResult); // Just mark as success. Later it will get reset 710 // based on the result from 711 // visibilityLabelService.addLabels () 712 } 713 if (!labels.isEmpty()) { 714 OperationStatus[] opStatus = this.visibilityLabelService.addLabels(labels); 715 logResult(true, "addLabels", "Adding labels allowed", null, labels, null); 716 int i = 0; 717 for (OperationStatus status : opStatus) { 718 while (!Objects.equals(response.getResult(i), successResult)) { 719 i++; 720 } 721 if (status.getOperationStatusCode() != SUCCESS) { 722 RegionActionResult.Builder failureResultBuilder = RegionActionResult.newBuilder(); 723 failureResultBuilder 724 .setException(buildException(new DoNotRetryIOException(status.getExceptionMsg()))); 725 response.setResult(i, failureResultBuilder.build()); 726 } 727 i++; 728 } 729 } 730 } catch (AccessDeniedException e) { 731 logResult(false, "addLabels", e.getMessage(), null, labels, null); 732 LOG.error("User is not having required permissions to add labels", e); 733 setExceptionResults(visLabels.size(), e, response); 734 } catch (IOException e) { 735 LOG.error(e.toString(), e); 736 setExceptionResults(visLabels.size(), e, response); 737 } 738 } 739 done.run(response.build()); 740 } 741 742 private void setExceptionResults(int size, IOException e, 743 VisibilityLabelsResponse.Builder response) { 744 RegionActionResult.Builder failureResultBuilder = RegionActionResult.newBuilder(); 745 failureResultBuilder.setException(buildException(e)); 746 RegionActionResult failureResult = failureResultBuilder.build(); 747 for (int i = 0; i < size; i++) { 748 response.addResult(i, failureResult); 749 } 750 } 751 752 @Override 753 public synchronized void setAuths(RpcController controller, SetAuthsRequest request, 754 RpcCallback<VisibilityLabelsResponse> done) { 755 VisibilityLabelsResponse.Builder response = VisibilityLabelsResponse.newBuilder(); 756 List<ByteString> auths = request.getAuthList(); 757 if (!initialized) { 758 setExceptionResults(auths.size(), 759 new VisibilityControllerNotReadyException("VisibilityController not yet initialized!"), 760 response); 761 } else { 762 byte[] user = request.getUser().toByteArray(); 763 List<byte[]> labelAuths = new ArrayList<>(auths.size()); 764 try { 765 if (authorizationEnabled) { 766 checkCallingUserAuth(); 767 } 768 for (ByteString authBS : auths) { 769 labelAuths.add(authBS.toByteArray()); 770 } 771 OperationStatus[] opStatus = this.visibilityLabelService.setAuths(user, labelAuths); 772 logResult(true, "setAuths", "Setting authorization for labels allowed", user, labelAuths, 773 null); 774 RegionActionResult successResult = RegionActionResult.newBuilder().build(); 775 for (OperationStatus status : opStatus) { 776 if (status.getOperationStatusCode() == SUCCESS) { 777 response.addResult(successResult); 778 } else { 779 RegionActionResult.Builder failureResultBuilder = RegionActionResult.newBuilder(); 780 failureResultBuilder 781 .setException(buildException(new DoNotRetryIOException(status.getExceptionMsg()))); 782 response.addResult(failureResultBuilder.build()); 783 } 784 } 785 } catch (AccessDeniedException e) { 786 logResult(false, "setAuths", e.getMessage(), user, labelAuths, null); 787 LOG.error("User is not having required permissions to set authorization", e); 788 setExceptionResults(auths.size(), e, response); 789 } catch (IOException e) { 790 LOG.error(e.toString(), e); 791 setExceptionResults(auths.size(), e, response); 792 } 793 } 794 done.run(response.build()); 795 } 796 797 private void logResult(boolean isAllowed, String request, String reason, byte[] user, 798 List<byte[]> labelAuths, String regex) { 799 if (AUDITLOG.isTraceEnabled()) { 800 // This is more duplicated code! 801 List<String> labelAuthsStr = new ArrayList<>(); 802 if (labelAuths != null) { 803 int labelAuthsSize = labelAuths.size(); 804 labelAuthsStr = new ArrayList<>(labelAuthsSize); 805 for (int i = 0; i < labelAuthsSize; i++) { 806 labelAuthsStr.add(Bytes.toString(labelAuths.get(i))); 807 } 808 } 809 810 User requestingUser = null; 811 try { 812 requestingUser = VisibilityUtils.getActiveUser(); 813 } catch (IOException e) { 814 LOG.warn("Failed to get active system user."); 815 LOG.debug("Details on failure to get active system user.", e); 816 } 817 AUDITLOG.trace("Access " + (isAllowed ? "allowed" : "denied") + " for user " 818 + (requestingUser != null ? requestingUser.getShortName() : "UNKNOWN") + "; reason: " 819 + reason + "; remote address: " 820 + RpcServer.getRemoteAddress().map(InetAddress::toString).orElse("") + "; request: " 821 + request + "; user: " + (user != null ? Bytes.toShort(user) : "null") + "; labels: " 822 + labelAuthsStr + "; regex: " + regex); 823 } 824 } 825 826 @Override 827 public synchronized void getAuths(RpcController controller, GetAuthsRequest request, 828 RpcCallback<GetAuthsResponse> done) { 829 GetAuthsResponse.Builder response = GetAuthsResponse.newBuilder(); 830 if (!initialized) { 831 controller.setFailed("VisibilityController not yet initialized"); 832 } else { 833 byte[] user = request.getUser().toByteArray(); 834 List<String> labels = null; 835 try { 836 // We do ACL check here as we create scanner directly on region. It will not make calls to 837 // AccessController CP methods. 838 if (authorizationEnabled && accessControllerAvailable && !isSystemOrSuperUser()) { 839 User requestingUser = VisibilityUtils.getActiveUser(); 840 throw new AccessDeniedException( 841 "User '" + (requestingUser != null ? requestingUser.getShortName() : "null") 842 + "' is not authorized to perform this action."); 843 } 844 if (AuthUtil.isGroupPrincipal(Bytes.toString(user))) { 845 String group = AuthUtil.getGroupName(Bytes.toString(user)); 846 labels = this.visibilityLabelService.getGroupAuths(new String[] { group }, false); 847 } else { 848 labels = this.visibilityLabelService.getUserAuths(user, false); 849 } 850 logResult(true, "getAuths", "Get authorizations for user allowed", user, null, null); 851 } catch (AccessDeniedException e) { 852 logResult(false, "getAuths", e.getMessage(), user, null, null); 853 CoprocessorRpcUtils.setControllerException(controller, e); 854 } catch (IOException e) { 855 CoprocessorRpcUtils.setControllerException(controller, e); 856 } 857 response.setUser(request.getUser()); 858 if (labels != null) { 859 for (String label : labels) { 860 response.addAuth(ByteString.copyFrom(Bytes.toBytes(label))); 861 } 862 } 863 } 864 done.run(response.build()); 865 } 866 867 @Override 868 public synchronized void clearAuths(RpcController controller, SetAuthsRequest request, 869 RpcCallback<VisibilityLabelsResponse> done) { 870 VisibilityLabelsResponse.Builder response = VisibilityLabelsResponse.newBuilder(); 871 List<ByteString> auths = request.getAuthList(); 872 if (!initialized) { 873 setExceptionResults(auths.size(), 874 new CoprocessorException("VisibilityController not yet initialized"), response); 875 } else { 876 byte[] requestUser = request.getUser().toByteArray(); 877 List<byte[]> labelAuths = new ArrayList<>(auths.size()); 878 try { 879 // When AC is ON, do AC based user auth check 880 if (authorizationEnabled && accessControllerAvailable && !isSystemOrSuperUser()) { 881 User user = VisibilityUtils.getActiveUser(); 882 throw new AccessDeniedException("User '" + (user != null ? user.getShortName() : "null") 883 + " is not authorized to perform this action."); 884 } 885 if (authorizationEnabled) { 886 checkCallingUserAuth(); // When AC is not in place the calling user should have 887 // SYSTEM_LABEL auth to do this action. 888 } 889 for (ByteString authBS : auths) { 890 labelAuths.add(authBS.toByteArray()); 891 } 892 893 OperationStatus[] opStatus = 894 this.visibilityLabelService.clearAuths(requestUser, labelAuths); 895 logResult(true, "clearAuths", "Removing authorization for labels allowed", requestUser, 896 labelAuths, null); 897 RegionActionResult successResult = RegionActionResult.newBuilder().build(); 898 for (OperationStatus status : opStatus) { 899 if (status.getOperationStatusCode() == SUCCESS) { 900 response.addResult(successResult); 901 } else { 902 RegionActionResult.Builder failureResultBuilder = RegionActionResult.newBuilder(); 903 failureResultBuilder 904 .setException(buildException(new DoNotRetryIOException(status.getExceptionMsg()))); 905 response.addResult(failureResultBuilder.build()); 906 } 907 } 908 } catch (AccessDeniedException e) { 909 logResult(false, "clearAuths", e.getMessage(), requestUser, labelAuths, null); 910 LOG.error("User is not having required permissions to clear authorization", e); 911 setExceptionResults(auths.size(), e, response); 912 } catch (IOException e) { 913 LOG.error(e.toString(), e); 914 setExceptionResults(auths.size(), e, response); 915 } 916 } 917 done.run(response.build()); 918 } 919 920 @Override 921 public synchronized void listLabels(RpcController controller, ListLabelsRequest request, 922 RpcCallback<ListLabelsResponse> done) { 923 ListLabelsResponse.Builder response = ListLabelsResponse.newBuilder(); 924 if (!initialized) { 925 controller.setFailed("VisibilityController not yet initialized"); 926 } else { 927 List<String> labels = null; 928 String regex = request.hasRegex() ? request.getRegex() : null; 929 try { 930 // We do ACL check here as we create scanner directly on region. It will not make calls to 931 // AccessController CP methods. 932 if (authorizationEnabled && accessControllerAvailable && !isSystemOrSuperUser()) { 933 User requestingUser = VisibilityUtils.getActiveUser(); 934 throw new AccessDeniedException( 935 "User '" + (requestingUser != null ? requestingUser.getShortName() : "null") 936 + "' is not authorized to perform this action."); 937 } 938 labels = this.visibilityLabelService.listLabels(regex); 939 logResult(true, "listLabels", "Listing labels allowed", null, null, regex); 940 } catch (AccessDeniedException e) { 941 logResult(false, "listLabels", e.getMessage(), null, null, regex); 942 CoprocessorRpcUtils.setControllerException(controller, e); 943 } catch (IOException e) { 944 CoprocessorRpcUtils.setControllerException(controller, e); 945 } 946 if (labels != null && !labels.isEmpty()) { 947 for (String label : labels) { 948 response.addLabel(ByteString.copyFrom(Bytes.toBytes(label))); 949 } 950 } 951 } 952 done.run(response.build()); 953 } 954 955 private void checkCallingUserAuth() throws IOException { 956 if (!authorizationEnabled) { // Redundant, but just in case 957 return; 958 } 959 if (!accessControllerAvailable) { 960 User user = VisibilityUtils.getActiveUser(); 961 if (user == null) { 962 throw new IOException("Unable to retrieve calling user"); 963 } 964 if (!(this.visibilityLabelService.havingSystemAuth(user))) { 965 throw new AccessDeniedException( 966 "User '" + user.getShortName() + "' is not authorized to perform this action."); 967 } 968 } 969 } 970 971 private static class DeleteVersionVisibilityExpressionFilter extends FilterBase { 972 private List<Tag> deleteCellVisTags; 973 private Byte deleteCellVisTagsFormat; 974 975 public DeleteVersionVisibilityExpressionFilter(List<Tag> deleteCellVisTags, 976 Byte deleteCellVisTagsFormat) { 977 this.deleteCellVisTags = deleteCellVisTags; 978 this.deleteCellVisTagsFormat = deleteCellVisTagsFormat; 979 } 980 981 @Override 982 public boolean filterRowKey(Cell cell) throws IOException { 983 // Impl in FilterBase might do unnecessary copy for Off heap backed Cells. 984 return false; 985 } 986 987 @Override 988 public ReturnCode filterCell(final Cell cell) throws IOException { 989 List<Tag> putVisTags = new ArrayList<>(); 990 Byte putCellVisTagsFormat = null; 991 if (cell instanceof ExtendedCell) { 992 putCellVisTagsFormat = 993 VisibilityUtils.extractVisibilityTags((ExtendedCell) cell, putVisTags); 994 } 995 996 if (putVisTags.isEmpty() && deleteCellVisTags.isEmpty()) { 997 // Early out if there are no tags in the cell 998 return ReturnCode.INCLUDE; 999 } 1000 boolean matchFound = 1001 VisibilityLabelServiceManager.getInstance().getVisibilityLabelService().matchVisibility( 1002 putVisTags, putCellVisTagsFormat, deleteCellVisTags, deleteCellVisTagsFormat); 1003 return matchFound ? ReturnCode.INCLUDE : ReturnCode.SKIP; 1004 } 1005 1006 @Override 1007 public boolean equals(Object obj) { 1008 if (!(obj instanceof DeleteVersionVisibilityExpressionFilter)) { 1009 return false; 1010 } 1011 if (this == obj) { 1012 return true; 1013 } 1014 DeleteVersionVisibilityExpressionFilter f = (DeleteVersionVisibilityExpressionFilter) obj; 1015 return this.deleteCellVisTags.equals(f.deleteCellVisTags) 1016 && this.deleteCellVisTagsFormat.equals(f.deleteCellVisTagsFormat); 1017 } 1018 1019 @Override 1020 public int hashCode() { 1021 return Objects.hash(this.deleteCellVisTags, this.deleteCellVisTagsFormat); 1022 } 1023 } 1024 1025 /** Returns NameValuePair of the exception name to stringified version os exception. */ 1026 // Copied from ResponseConverter and made private. Only used in here. 1027 private static NameBytesPair buildException(final Throwable t) { 1028 NameBytesPair.Builder parameterBuilder = NameBytesPair.newBuilder(); 1029 parameterBuilder.setName(t.getClass().getName()); 1030 parameterBuilder.setValue(ByteString.copyFromUtf8(StringUtils.stringifyException(t))); 1031 return parameterBuilder.build(); 1032 } 1033}