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}