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