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}