001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 * http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018
019package org.apache.hadoop.hbase.security.visibility;
020
021import static org.apache.hadoop.hbase.HConstants.OperationStatusCode.SANITY_CHECK_FAILURE;
022import static org.apache.hadoop.hbase.HConstants.OperationStatusCode.SUCCESS;
023import static org.apache.hadoop.hbase.security.visibility.VisibilityConstants.LABELS_TABLE_FAMILY;
024import static org.apache.hadoop.hbase.security.visibility.VisibilityConstants.LABELS_TABLE_NAME;
025
026import com.google.protobuf.ByteString;
027import com.google.protobuf.RpcCallback;
028import com.google.protobuf.RpcController;
029import com.google.protobuf.Service;
030import java.io.IOException;
031import java.net.InetAddress;
032import java.util.ArrayList;
033import java.util.Collections;
034import java.util.HashMap;
035import java.util.Iterator;
036import java.util.List;
037import java.util.Map;
038import java.util.Objects;
039import java.util.Optional;
040import org.apache.hadoop.conf.Configuration;
041import org.apache.hadoop.hbase.AuthUtil;
042import org.apache.hadoop.hbase.Cell;
043import org.apache.hadoop.hbase.CellScanner;
044import org.apache.hadoop.hbase.CoprocessorEnvironment;
045import org.apache.hadoop.hbase.DoNotRetryIOException;
046import org.apache.hadoop.hbase.HBaseInterfaceAudience;
047import org.apache.hadoop.hbase.PrivateCellUtil;
048import org.apache.hadoop.hbase.TableName;
049import org.apache.hadoop.hbase.Tag;
050import org.apache.hadoop.hbase.TagType;
051import org.apache.hadoop.hbase.client.Admin;
052import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
053import org.apache.hadoop.hbase.client.Delete;
054import org.apache.hadoop.hbase.client.Get;
055import org.apache.hadoop.hbase.client.MasterSwitchType;
056import org.apache.hadoop.hbase.client.Mutation;
057import org.apache.hadoop.hbase.client.Put;
058import org.apache.hadoop.hbase.client.Result;
059import org.apache.hadoop.hbase.client.Scan;
060import org.apache.hadoop.hbase.client.TableDescriptor;
061import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
062import org.apache.hadoop.hbase.constraint.ConstraintException;
063import org.apache.hadoop.hbase.coprocessor.CoprocessorException;
064import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
065import org.apache.hadoop.hbase.coprocessor.CoreCoprocessor;
066import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor;
067import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
068import org.apache.hadoop.hbase.coprocessor.MasterObserver;
069import org.apache.hadoop.hbase.coprocessor.ObserverContext;
070import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
071import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
072import org.apache.hadoop.hbase.coprocessor.RegionObserver;
073import org.apache.hadoop.hbase.exceptions.DeserializationException;
074import org.apache.hadoop.hbase.filter.Filter;
075import org.apache.hadoop.hbase.filter.FilterBase;
076import org.apache.hadoop.hbase.filter.FilterList;
077import org.apache.hadoop.hbase.io.hfile.HFile;
078import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
079import org.apache.hadoop.hbase.ipc.RpcServer;
080import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult;
081import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair;
082import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos;
083import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.GetAuthsRequest;
084import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.GetAuthsResponse;
085import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.ListLabelsRequest;
086import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.ListLabelsResponse;
087import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.SetAuthsRequest;
088import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabel;
089import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsRequest;
090import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse;
091import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsService;
092import org.apache.hadoop.hbase.regionserver.BloomType;
093import org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy;
094import org.apache.hadoop.hbase.regionserver.InternalScanner;
095import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
096import org.apache.hadoop.hbase.regionserver.OperationStatus;
097import org.apache.hadoop.hbase.regionserver.Region;
098import org.apache.hadoop.hbase.regionserver.RegionScanner;
099import org.apache.hadoop.hbase.regionserver.querymatcher.DeleteTracker;
100import org.apache.hadoop.hbase.security.AccessDeniedException;
101import org.apache.hadoop.hbase.security.Superusers;
102import org.apache.hadoop.hbase.security.User;
103import org.apache.hadoop.hbase.security.access.AccessChecker;
104import org.apache.hadoop.hbase.security.access.AccessController;
105import org.apache.hadoop.hbase.util.Bytes;
106import org.apache.hadoop.hbase.util.Pair;
107import org.apache.hadoop.util.StringUtils;
108import org.apache.yetus.audience.InterfaceAudience;
109import org.slf4j.Logger;
110import org.slf4j.LoggerFactory;
111
112import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
113import org.apache.hbase.thirdparty.com.google.common.collect.MapMaker;
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
126  private static final Logger LOG = LoggerFactory.getLogger(VisibilityController.class);
127  private static final Logger AUDITLOG = LoggerFactory.getLogger("SecurityLogger."
128      + VisibilityController.class.getName());
129  // flags if we are running on a region of the 'labels' table
130  private boolean labelsRegion = false;
131  // Flag denoting whether AcessController is available or not.
132  private boolean accessControllerAvailable = false;
133  private Configuration conf;
134  private volatile boolean initialized = false;
135  private boolean checkAuths = false;
136  /** Mapping of scanner instances to the user who created them */
137  private Map<InternalScanner,String> scannerOwners =
138      new MapMaker().weakKeys().makeMap();
139
140  private VisibilityLabelService visibilityLabelService;
141
142  /** if we are active, usually false, only true if "hbase.security.authorization"
143    has been set to true in site configuration */
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 = VisibilityLabelServiceManager.getInstance()
176          .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.singleton(
199        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, TableName tableName)
238      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 = CoprocessorHost.getLoadedCoprocessors()
256          .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 = this.visibilityLabelService.createVisibilityExpTags(labelsExp, true,
331                  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(
372      ObserverContext<RegionCoprocessorEnvironment> ctx, Mutation delete, Cell cell,
373      byte[] byteNow, Get get) throws IOException {
374    // Nothing to do if we are not filtering by visibility
375    if (!authorizationEnabled) {
376      return;
377    }
378
379    CellVisibility cellVisibility = null;
380    try {
381      cellVisibility = delete.getCellVisibility();
382    } catch (DeserializationException de) {
383      throw new IOException("Invalid cell visibility specified " + delete, de);
384    }
385    // The check for checkForReservedVisibilityTagPresence happens in preBatchMutate happens.
386    // It happens for every mutation and that would be enough.
387    List<Tag> visibilityTags = new ArrayList<>();
388    if (cellVisibility != null) {
389      String labelsExp = cellVisibility.getExpression();
390      try {
391        visibilityTags = this.visibilityLabelService.createVisibilityExpTags(labelsExp, false,
392            false);
393      } catch (InvalidLabelException e) {
394        throw new IOException("Invalid cell visibility specified " + labelsExp, e);
395      }
396    }
397    get.setFilter(new DeleteVersionVisibilityExpressionFilter(visibilityTags,
398        VisibilityConstants.SORTED_ORDINAL_SERIALIZATION_FORMAT));
399    List<Cell> result = ctx.getEnvironment().getRegion().get(get, false);
400
401    if (result.size() < get.getMaxVersions()) {
402      // Nothing to delete
403      PrivateCellUtil.updateLatestStamp(cell, byteNow);
404      return;
405    }
406    if (result.size() > get.getMaxVersions()) {
407      throw new RuntimeException("Unexpected size: " + result.size()
408          + ". Results more than the max versions obtained.");
409    }
410    Cell getCell = result.get(get.getMaxVersions() - 1);
411    PrivateCellUtil.setTimestamp(cell, getCell.getTimestamp());
412
413    // We are bypassing here because in the HRegion.updateDeleteLatestVersionTimeStamp we would
414    // update with the current timestamp after again doing a get. As the hook as already determined
415    // the needed timestamp we need to bypass here.
416    // TODO : See if HRegion.updateDeleteLatestVersionTimeStamp() could be
417    // called only if the hook is not called.
418    ctx.bypass();
419  }
420
421  /**
422   * Checks whether cell contains any tag with type as VISIBILITY_TAG_TYPE. This
423   * tag type is reserved and should not be explicitly set by user.
424   *
425   * @param cell The cell under consideration
426   * @param pair An optional pair of type {@code <Boolean, Tag>} which would be reused if already
427   *     set and new one will be created if NULL is passed
428   * @return If the boolean is false then it indicates that the cell has a RESERVERD_VIS_TAG and
429   *     with boolean as true, not null tag indicates that a string modified tag was found.
430   */
431  private Pair<Boolean, Tag> checkForReservedVisibilityTagPresence(Cell cell,
432      Pair<Boolean, Tag> pair) throws IOException {
433    if (pair == null) {
434      pair = new Pair<>(false, null);
435    } else {
436      pair.setFirst(false);
437      pair.setSecond(null);
438    }
439    // Bypass this check when the operation is done by a system/super user.
440    // This is done because, while Replication, the Cells coming to the peer cluster with reserved
441    // typed tags and this is fine and should get added to the peer cluster table
442    if (isSystemOrSuperUser()) {
443      // Does the cell contain special tag which indicates that the replicated
444      // cell visiblilty tags
445      // have been modified
446      Tag modifiedTag = null;
447      Iterator<Tag> tagsIterator = PrivateCellUtil.tagsIterator(cell);
448      while (tagsIterator.hasNext()) {
449        Tag tag = tagsIterator.next();
450        if (tag.getType() == TagType.STRING_VIS_TAG_TYPE) {
451          modifiedTag = tag;
452          break;
453        }
454      }
455      pair.setFirst(true);
456      pair.setSecond(modifiedTag);
457      return pair;
458    }
459    Iterator<Tag> tagsItr = PrivateCellUtil.tagsIterator(cell);
460    while (tagsItr.hasNext()) {
461      if (RESERVED_VIS_TAG_TYPES.contains(tagsItr.next().getType())) {
462        return pair;
463      }
464    }
465    pair.setFirst(true);
466    return pair;
467  }
468
469  private void removeReplicationVisibilityTag(List<Tag> tags) throws IOException {
470    Iterator<Tag> iterator = tags.iterator();
471    while (iterator.hasNext()) {
472      Tag tag = iterator.next();
473      if (tag.getType() == TagType.STRING_VIS_TAG_TYPE) {
474        iterator.remove();
475        break;
476      }
477    }
478  }
479
480  @Override
481  public void preScannerOpen(ObserverContext<RegionCoprocessorEnvironment> e, Scan scan)
482      throws IOException {
483    if (!initialized) {
484      throw new VisibilityControllerNotReadyException("VisibilityController not yet initialized!");
485    }
486    // Nothing to do if authorization is not enabled
487    if (!authorizationEnabled) {
488      return;
489    }
490    Region region = e.getEnvironment().getRegion();
491    Authorizations authorizations = null;
492    try {
493      authorizations = scan.getAuthorizations();
494    } catch (DeserializationException de) {
495      throw new IOException(de);
496    }
497    if (authorizations == null) {
498      // No Authorizations present for this scan/Get!
499      // In case of system tables other than "labels" just scan with out visibility check and
500      // filtering. Checking visibility labels for META and NAMESPACE table is not needed.
501      TableName table = region.getRegionInfo().getTable();
502      if (table.isSystemTable() && !table.equals(LABELS_TABLE_NAME)) {
503        return;
504      }
505    }
506
507    Filter visibilityLabelFilter = VisibilityUtils.createVisibilityLabelFilter(region,
508        authorizations);
509    if (visibilityLabelFilter != null) {
510      Filter filter = scan.getFilter();
511      if (filter != null) {
512        scan.setFilter(new FilterList(filter, visibilityLabelFilter));
513      } else {
514        scan.setFilter(visibilityLabelFilter);
515      }
516    }
517  }
518
519  @Override
520  public DeleteTracker postInstantiateDeleteTracker(
521      ObserverContext<RegionCoprocessorEnvironment> ctx, DeleteTracker delTracker)
522      throws IOException {
523    // Nothing to do if we are not filtering by visibility
524    if (!authorizationEnabled) {
525      return delTracker;
526    }
527    Region region = ctx.getEnvironment().getRegion();
528    TableName table = region.getRegionInfo().getTable();
529    if (table.isSystemTable()) {
530      return delTracker;
531    }
532    // We are creating a new type of delete tracker here which is able to track
533    // the timestamps and also the
534    // visibility tags per cell. The covering cells are determined not only
535    // based on the delete type and ts
536    // but also on the visibility expression matching.
537    return new VisibilityScanDeleteTracker(delTracker.getCellComparator());
538  }
539
540  @Override
541  public RegionScanner postScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
542      final Scan scan, final RegionScanner s) throws IOException {
543    User user = VisibilityUtils.getActiveUser();
544    if (user != null && user.getShortName() != null) {
545      scannerOwners.put(s, user.getShortName());
546    }
547    return s;
548  }
549
550  @Override
551  public boolean preScannerNext(final ObserverContext<RegionCoprocessorEnvironment> c,
552      final InternalScanner s, final List<Result> result, final int limit, final boolean hasNext)
553      throws IOException {
554    requireScannerOwner(s);
555    return hasNext;
556  }
557
558  @Override
559  public void preScannerClose(final ObserverContext<RegionCoprocessorEnvironment> c,
560      final InternalScanner s) throws IOException {
561    requireScannerOwner(s);
562  }
563
564  @Override
565  public void postScannerClose(final ObserverContext<RegionCoprocessorEnvironment> c,
566      final InternalScanner s) throws IOException {
567    // clean up any associated owner mapping
568    scannerOwners.remove(s);
569  }
570
571  /**
572   * Verify, when servicing an RPC, that the caller is the scanner owner. If so, we assume that
573   * access control is correctly enforced based on the checks performed in preScannerOpen()
574   */
575  private void requireScannerOwner(InternalScanner s) throws AccessDeniedException {
576    if (!RpcServer.isInRpcCallContext())
577      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,
587      List<Cell> results) 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 = VisibilityUtils.createVisibilityLabelFilter(e.getEnvironment()
612        .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 (tag.getType() != TagType.VISIBILITY_TAG_TYPE
672          && tag.getType() != TagType.VISIBILITY_EXP_SERIALIZATION_FORMAT_TAG_TYPE) {
673        tags.add(tag);
674      }
675    }
676
677    return PrivateCellUtil.createCell(newCell, tags);
678  }
679
680  @Override
681  public boolean postScannerFilterRow(final ObserverContext<RegionCoprocessorEnvironment> e,
682      final InternalScanner s, final Cell curRowCell, final boolean hasMore) throws IOException {
683    // 'default' in RegionObserver might do unnecessary copy for Off heap backed Cells.
684    return hasMore;
685  }
686
687  /****************************** VisibilityEndpoint service related methods ******************************/
688  @Override
689  public synchronized void addLabels(RpcController controller, VisibilityLabelsRequest request,
690      RpcCallback<VisibilityLabelsResponse> done) {
691    VisibilityLabelsResponse.Builder response = VisibilityLabelsResponse.newBuilder();
692    List<VisibilityLabel> visLabels = request.getVisLabelList();
693    if (!initialized) {
694      setExceptionResults(visLabels.size(),
695        new VisibilityControllerNotReadyException("VisibilityController not yet initialized!"),
696        response);
697    } else {
698      List<byte[]> labels = new ArrayList<>(visLabels.size());
699      try {
700        if (authorizationEnabled) {
701          checkCallingUserAuth();
702        }
703        RegionActionResult successResult = RegionActionResult.newBuilder().build();
704        for (VisibilityLabel visLabel : visLabels) {
705          byte[] label = visLabel.getLabel().toByteArray();
706          labels.add(label);
707          response.addResult(successResult); // Just mark as success. Later it will get reset
708                                             // based on the result from
709                                             // visibilityLabelService.addLabels ()
710        }
711        if (!labels.isEmpty()) {
712          OperationStatus[] opStatus = this.visibilityLabelService.addLabels(labels);
713          logResult(true, "addLabels", "Adding labels allowed", null, labels, null);
714          int i = 0;
715          for (OperationStatus status : opStatus) {
716            while (!Objects.equals(response.getResult(i), successResult)) {
717              i++;
718            }
719            if (status.getOperationStatusCode() != SUCCESS) {
720              RegionActionResult.Builder failureResultBuilder = RegionActionResult.newBuilder();
721              failureResultBuilder.setException(buildException(new DoNotRetryIOException(
722                  status.getExceptionMsg())));
723              response.setResult(i, failureResultBuilder.build());
724            }
725            i++;
726          }
727        }
728      } catch (AccessDeniedException e) {
729        logResult(false, "addLabels", e.getMessage(), null, labels, null);
730        LOG.error("User is not having required permissions to add labels", e);
731        setExceptionResults(visLabels.size(), e, response);
732      } catch (IOException e) {
733        LOG.error(e.toString(), e);
734        setExceptionResults(visLabels.size(), e, response);
735      }
736    }
737    done.run(response.build());
738  }
739
740  private void setExceptionResults(int size, IOException e,
741      VisibilityLabelsResponse.Builder response) {
742    RegionActionResult.Builder failureResultBuilder = RegionActionResult.newBuilder();
743    failureResultBuilder.setException(buildException(e));
744    RegionActionResult failureResult = failureResultBuilder.build();
745    for (int i = 0; i < size; i++) {
746      response.addResult(i, failureResult);
747    }
748  }
749
750  @Override
751  public synchronized void setAuths(RpcController controller, SetAuthsRequest request,
752      RpcCallback<VisibilityLabelsResponse> done) {
753    VisibilityLabelsResponse.Builder response = VisibilityLabelsResponse.newBuilder();
754    List<ByteString> auths = request.getAuthList();
755    if (!initialized) {
756      setExceptionResults(auths.size(),
757        new VisibilityControllerNotReadyException("VisibilityController not yet initialized!"),
758        response);
759    } else {
760      byte[] user = request.getUser().toByteArray();
761      List<byte[]> labelAuths = new ArrayList<>(auths.size());
762      try {
763        if (authorizationEnabled) {
764          checkCallingUserAuth();
765        }
766        for (ByteString authBS : auths) {
767          labelAuths.add(authBS.toByteArray());
768        }
769        OperationStatus[] opStatus = this.visibilityLabelService.setAuths(user, labelAuths);
770        logResult(true, "setAuths", "Setting authorization for labels allowed", user, labelAuths,
771          null);
772        RegionActionResult successResult = RegionActionResult.newBuilder().build();
773        for (OperationStatus status : opStatus) {
774          if (status.getOperationStatusCode() == SUCCESS) {
775            response.addResult(successResult);
776          } else {
777            RegionActionResult.Builder failureResultBuilder = RegionActionResult.newBuilder();
778            failureResultBuilder.setException(buildException(new DoNotRetryIOException(
779                status.getExceptionMsg())));
780            response.addResult(failureResultBuilder.build());
781          }
782        }
783      } catch (AccessDeniedException e) {
784        logResult(false, "setAuths", e.getMessage(), user, labelAuths, null);
785        LOG.error("User is not having required permissions to set authorization", e);
786        setExceptionResults(auths.size(), e, response);
787      } catch (IOException e) {
788        LOG.error(e.toString(), e);
789        setExceptionResults(auths.size(), e, response);
790      }
791    }
792    done.run(response.build());
793  }
794
795  private void logResult(boolean isAllowed, String request, String reason, byte[] user,
796      List<byte[]> labelAuths, String regex) {
797    if (AUDITLOG.isTraceEnabled()) {
798      // This is more duplicated code!
799      List<String> labelAuthsStr = new ArrayList<>();
800      if (labelAuths != null) {
801        int labelAuthsSize = labelAuths.size();
802        labelAuthsStr = new ArrayList<>(labelAuthsSize);
803        for (int i = 0; i < labelAuthsSize; i++) {
804          labelAuthsStr.add(Bytes.toString(labelAuths.get(i)));
805        }
806      }
807
808      User requestingUser = null;
809      try {
810        requestingUser = VisibilityUtils.getActiveUser();
811      } catch (IOException e) {
812        LOG.warn("Failed to get active system user.");
813        LOG.debug("Details on failure to get active system user.", e);
814      }
815      AUDITLOG.trace("Access " + (isAllowed ? "allowed" : "denied") + " for user " +
816          (requestingUser != null ? requestingUser.getShortName() : "UNKNOWN") + "; reason: " +
817          reason + "; remote address: " +
818          RpcServer.getRemoteAddress().map(InetAddress::toString).orElse("") + "; request: " +
819          request + "; user: " + (user != null ? Bytes.toShort(user) : "null") + "; labels: " +
820          labelAuthsStr + "; regex: " + regex);
821    }
822  }
823
824  @Override
825  public synchronized void getAuths(RpcController controller, GetAuthsRequest request,
826      RpcCallback<GetAuthsResponse> done) {
827    GetAuthsResponse.Builder response = GetAuthsResponse.newBuilder();
828    if (!initialized) {
829      controller.setFailed("VisibilityController not yet initialized");
830    } else {
831      byte[] user = request.getUser().toByteArray();
832      List<String> labels = null;
833      try {
834        // We do ACL check here as we create scanner directly on region. It will not make calls to
835        // AccessController CP methods.
836        if (authorizationEnabled && accessControllerAvailable && !isSystemOrSuperUser()) {
837          User requestingUser = VisibilityUtils.getActiveUser();
838          throw new AccessDeniedException("User '"
839              + (requestingUser != null ? requestingUser.getShortName() : "null")
840              + "' is not authorized to perform this action.");
841        }
842        if (AuthUtil.isGroupPrincipal(Bytes.toString(user))) {
843          String group = AuthUtil.getGroupName(Bytes.toString(user));
844          labels = this.visibilityLabelService.getGroupAuths(new String[]{group}, false);
845        }
846        else {
847          labels = this.visibilityLabelService.getUserAuths(user, false);
848        }
849        logResult(true, "getAuths", "Get authorizations for user allowed", user, null, null);
850      } catch (AccessDeniedException e) {
851        logResult(false, "getAuths", e.getMessage(), user, null, null);
852        CoprocessorRpcUtils.setControllerException(controller, e);
853      } catch (IOException e) {
854        CoprocessorRpcUtils.setControllerException(controller, e);
855      }
856      response.setUser(request.getUser());
857      if (labels != null) {
858        for (String label : labels) {
859          response.addAuth(ByteString.copyFrom(Bytes.toBytes(label)));
860        }
861      }
862    }
863    done.run(response.build());
864  }
865
866  @Override
867  public synchronized void clearAuths(RpcController controller, SetAuthsRequest request,
868      RpcCallback<VisibilityLabelsResponse> done) {
869    VisibilityLabelsResponse.Builder response = VisibilityLabelsResponse.newBuilder();
870    List<ByteString> auths = request.getAuthList();
871    if (!initialized) {
872      setExceptionResults(auths.size(), new CoprocessorException(
873          "VisibilityController not yet initialized"), response);
874    } else {
875      byte[] requestUser = request.getUser().toByteArray();
876      List<byte[]> labelAuths = new ArrayList<>(auths.size());
877      try {
878        // When AC is ON, do AC based user auth check
879        if (authorizationEnabled && accessControllerAvailable && !isSystemOrSuperUser()) {
880          User user = VisibilityUtils.getActiveUser();
881          throw new AccessDeniedException("User '" + (user != null ? user.getShortName() : "null")
882              + " is not authorized to perform this action.");
883        }
884        if (authorizationEnabled) {
885          checkCallingUserAuth(); // When AC is not in place the calling user should have
886                                  // SYSTEM_LABEL auth to do this action.
887        }
888        for (ByteString authBS : auths) {
889          labelAuths.add(authBS.toByteArray());
890        }
891
892        OperationStatus[] opStatus =
893            this.visibilityLabelService.clearAuths(requestUser, labelAuths);
894        logResult(true, "clearAuths", "Removing authorization for labels allowed", requestUser,
895          labelAuths, null);
896        RegionActionResult successResult = RegionActionResult.newBuilder().build();
897        for (OperationStatus status : opStatus) {
898          if (status.getOperationStatusCode() == SUCCESS) {
899            response.addResult(successResult);
900          } else {
901            RegionActionResult.Builder failureResultBuilder = RegionActionResult.newBuilder();
902            failureResultBuilder.setException(buildException(new DoNotRetryIOException(
903                status.getExceptionMsg())));
904            response.addResult(failureResultBuilder.build());
905          }
906        }
907      } catch (AccessDeniedException e) {
908        logResult(false, "clearAuths", e.getMessage(), requestUser, labelAuths, null);
909        LOG.error("User is not having required permissions to clear authorization", e);
910        setExceptionResults(auths.size(), e, response);
911      } catch (IOException e) {
912        LOG.error(e.toString(), e);
913        setExceptionResults(auths.size(), e, response);
914      }
915    }
916    done.run(response.build());
917  }
918
919  @Override
920  public synchronized void listLabels(RpcController controller, ListLabelsRequest request,
921      RpcCallback<ListLabelsResponse> done) {
922    ListLabelsResponse.Builder response = ListLabelsResponse.newBuilder();
923    if (!initialized) {
924      controller.setFailed("VisibilityController not yet initialized");
925    } else {
926      List<String> labels = null;
927      String regex = request.hasRegex() ? request.getRegex() : null;
928      try {
929        // We do ACL check here as we create scanner directly on region. It will not make calls to
930        // AccessController CP methods.
931        if (authorizationEnabled && accessControllerAvailable && !isSystemOrSuperUser()) {
932          User requestingUser = VisibilityUtils.getActiveUser();
933          throw new AccessDeniedException("User '"
934              + (requestingUser != null ? requestingUser.getShortName() : "null")
935              + "' is not authorized to perform this action.");
936        }
937        labels = this.visibilityLabelService.listLabels(regex);
938        logResult(false, "listLabels", "Listing labels allowed", null, null, regex);
939      } catch (AccessDeniedException e) {
940        logResult(false, "listLabels", e.getMessage(), null, null, regex);
941        CoprocessorRpcUtils.setControllerException(controller, e);
942      } catch (IOException e) {
943        CoprocessorRpcUtils.setControllerException(controller, e);
944      }
945      if (labels != null && !labels.isEmpty()) {
946        for (String label : labels) {
947          response.addLabel(ByteString.copyFrom(Bytes.toBytes(label)));
948        }
949      }
950    }
951    done.run(response.build());
952  }
953
954  private void checkCallingUserAuth() throws IOException {
955    if (!authorizationEnabled) { // Redundant, but just in case
956      return;
957    }
958    if (!accessControllerAvailable) {
959      User user = VisibilityUtils.getActiveUser();
960      if (user == null) {
961        throw new IOException("Unable to retrieve calling user");
962      }
963      if (!(this.visibilityLabelService.havingSystemAuth(user))) {
964        throw new AccessDeniedException("User '" + user.getShortName()
965            + "' is not authorized to perform this action.");
966      }
967    }
968  }
969
970  private static class DeleteVersionVisibilityExpressionFilter extends FilterBase {
971    private List<Tag> deleteCellVisTags;
972    private Byte deleteCellVisTagsFormat;
973
974    public DeleteVersionVisibilityExpressionFilter(List<Tag> deleteCellVisTags,
975        Byte deleteCellVisTagsFormat) {
976      this.deleteCellVisTags = deleteCellVisTags;
977      this.deleteCellVisTagsFormat = deleteCellVisTagsFormat;
978    }
979
980    @Override
981    public boolean filterRowKey(Cell cell) throws IOException {
982      // Impl in FilterBase might do unnecessary copy for Off heap backed Cells.
983      return false;
984    }
985
986    @Override
987    public ReturnCode filterCell(final Cell cell) throws IOException {
988      List<Tag> putVisTags = new ArrayList<>();
989      Byte putCellVisTagsFormat = VisibilityUtils.extractVisibilityTags(cell, putVisTags);
990      if (putVisTags.isEmpty() && deleteCellVisTags.isEmpty()) {
991        // Early out if there are no tags in the cell
992        return ReturnCode.INCLUDE;
993      }
994      boolean matchFound = VisibilityLabelServiceManager
995          .getInstance().getVisibilityLabelService()
996          .matchVisibility(putVisTags, putCellVisTagsFormat, deleteCellVisTags,
997              deleteCellVisTagsFormat);
998      return matchFound ? ReturnCode.INCLUDE : ReturnCode.SKIP;
999    }
1000
1001    @Override
1002    public boolean equals(Object obj) {
1003      if (!(obj instanceof DeleteVersionVisibilityExpressionFilter)) {
1004        return false;
1005      }
1006      if (this == obj){
1007        return true;
1008      }
1009      DeleteVersionVisibilityExpressionFilter f = (DeleteVersionVisibilityExpressionFilter)obj;
1010      return this.deleteCellVisTags.equals(f.deleteCellVisTags) &&
1011          this.deleteCellVisTagsFormat.equals(f.deleteCellVisTagsFormat);
1012    }
1013
1014    @Override
1015    public int hashCode() {
1016      return Objects.hash(this.deleteCellVisTags, this.deleteCellVisTagsFormat);
1017    }
1018  }
1019
1020  /**
1021   * @param t
1022   * @return NameValuePair of the exception name to stringified version os exception.
1023   */
1024  // Copied from ResponseConverter and made private. Only used in here.
1025  private static NameBytesPair buildException(final Throwable t) {
1026    NameBytesPair.Builder parameterBuilder = NameBytesPair.newBuilder();
1027    parameterBuilder.setName(t.getClass().getName());
1028    parameterBuilder.setValue(
1029      ByteString.copyFromUtf8(StringUtils.stringifyException(t)));
1030    return parameterBuilder.build();
1031  }
1032}