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