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