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