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.TagType.VISIBILITY_TAG_TYPE;
021
022import java.io.ByteArrayOutputStream;
023import java.io.DataOutputStream;
024import java.io.IOException;
025import java.util.ArrayList;
026import java.util.Collections;
027import java.util.HashMap;
028import java.util.Iterator;
029import java.util.List;
030import java.util.Map;
031import java.util.Map.Entry;
032import java.util.Optional;
033import java.util.Set;
034import org.apache.commons.lang3.StringUtils;
035import org.apache.hadoop.conf.Configuration;
036import org.apache.hadoop.hbase.ArrayBackedTag;
037import org.apache.hadoop.hbase.ExtendedCell;
038import org.apache.hadoop.hbase.PrivateCellUtil;
039import org.apache.hadoop.hbase.Tag;
040import org.apache.hadoop.hbase.TagType;
041import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
042import org.apache.hadoop.hbase.exceptions.DeserializationException;
043import org.apache.hadoop.hbase.filter.Filter;
044import org.apache.hadoop.hbase.io.util.StreamUtils;
045import org.apache.hadoop.hbase.ipc.RpcServer;
046import org.apache.hadoop.hbase.regionserver.Region;
047import org.apache.hadoop.hbase.security.AccessDeniedException;
048import org.apache.hadoop.hbase.security.User;
049import org.apache.hadoop.hbase.security.visibility.expression.ExpressionNode;
050import org.apache.hadoop.hbase.security.visibility.expression.LeafExpressionNode;
051import org.apache.hadoop.hbase.security.visibility.expression.NonLeafExpressionNode;
052import org.apache.hadoop.hbase.security.visibility.expression.Operator;
053import org.apache.hadoop.hbase.util.ByteRange;
054import org.apache.hadoop.hbase.util.Bytes;
055import org.apache.hadoop.hbase.util.SimpleMutableByteRange;
056import org.apache.hadoop.util.ReflectionUtils;
057import org.apache.yetus.audience.InterfaceAudience;
058import org.slf4j.Logger;
059import org.slf4j.LoggerFactory;
060
061import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
062
063import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
064import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.MultiUserAuthorizations;
065import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.UserAuthorizations;
066import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.VisibilityLabel;
067import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsRequest;
068
069/**
070 * Utility method to support visibility
071 */
072@InterfaceAudience.Private
073public class VisibilityUtils {
074
075  private static final Logger LOG = LoggerFactory.getLogger(VisibilityUtils.class);
076
077  public static final String VISIBILITY_LABEL_GENERATOR_CLASS =
078    "hbase.regionserver.scan.visibility.label.generator.class";
079  public static final String SYSTEM_LABEL = "system";
080  public static final Tag SORTED_ORDINAL_SERIALIZATION_FORMAT_TAG =
081    new ArrayBackedTag(TagType.VISIBILITY_EXP_SERIALIZATION_FORMAT_TAG_TYPE,
082      VisibilityConstants.SORTED_ORDINAL_SERIALIZATION_FORMAT_TAG_VAL);
083  private static final String COMMA = ",";
084
085  private static final ExpressionParser EXP_PARSER = new ExpressionParser();
086  private static final ExpressionExpander EXP_EXPANDER = new ExpressionExpander();
087
088  /**
089   * Creates the labels data to be written to zookeeper.
090   * @return Bytes form of labels and their ordinal details to be written to zookeeper.
091   */
092  public static byte[] getDataToWriteToZooKeeper(Map<String, Integer> existingLabels) {
093    VisibilityLabelsRequest.Builder visReqBuilder = VisibilityLabelsRequest.newBuilder();
094    for (Entry<String, Integer> entry : existingLabels.entrySet()) {
095      VisibilityLabel.Builder visLabBuilder = VisibilityLabel.newBuilder();
096      visLabBuilder.setLabel(ByteString.copyFrom(Bytes.toBytes(entry.getKey())));
097      visLabBuilder.setOrdinal(entry.getValue());
098      visReqBuilder.addVisLabel(visLabBuilder.build());
099    }
100    return ProtobufUtil.prependPBMagic(visReqBuilder.build().toByteArray());
101  }
102
103  /**
104   * Creates the user auth data to be written to zookeeper.
105   * @return Bytes form of user auths details to be written to zookeeper.
106   */
107  public static byte[] getUserAuthsDataToWriteToZooKeeper(Map<String, List<Integer>> userAuths) {
108    MultiUserAuthorizations.Builder builder = MultiUserAuthorizations.newBuilder();
109    for (Entry<String, List<Integer>> entry : userAuths.entrySet()) {
110      UserAuthorizations.Builder userAuthsBuilder = UserAuthorizations.newBuilder();
111      userAuthsBuilder.setUser(ByteString.copyFrom(Bytes.toBytes(entry.getKey())));
112      for (Integer label : entry.getValue()) {
113        userAuthsBuilder.addAuth(label);
114      }
115      builder.addUserAuths(userAuthsBuilder.build());
116    }
117    return ProtobufUtil.prependPBMagic(builder.build().toByteArray());
118  }
119
120  /**
121   * Reads back from the zookeeper. The data read here is of the form written by
122   * writeToZooKeeper(Map&lt;byte[], Integer&gt; entries).
123   * @return Labels and their ordinal details
124   */
125  public static List<VisibilityLabel> readLabelsFromZKData(byte[] data)
126    throws DeserializationException {
127    if (ProtobufUtil.isPBMagicPrefix(data)) {
128      int pblen = ProtobufUtil.lengthOfPBMagic();
129      try {
130        VisibilityLabelsRequest.Builder builder = VisibilityLabelsRequest.newBuilder();
131        ProtobufUtil.mergeFrom(builder, data, pblen, data.length - pblen);
132        return builder.getVisLabelList();
133      } catch (IOException e) {
134        throw new DeserializationException(e);
135      }
136    }
137    return null;
138  }
139
140  /**
141   * Reads back User auth data written to zookeeper.
142   * @return User auth details
143   */
144  public static MultiUserAuthorizations readUserAuthsFromZKData(byte[] data)
145    throws DeserializationException {
146    if (ProtobufUtil.isPBMagicPrefix(data)) {
147      int pblen = ProtobufUtil.lengthOfPBMagic();
148      try {
149        MultiUserAuthorizations.Builder builder = MultiUserAuthorizations.newBuilder();
150        ProtobufUtil.mergeFrom(builder, data, pblen, data.length - pblen);
151        return builder.build();
152      } catch (IOException e) {
153        throw new DeserializationException(e);
154      }
155    }
156    return null;
157  }
158
159  /**
160   * @param conf The configuration to use
161   * @return Stack of ScanLabelGenerator instances. ScanLabelGenerator classes can be specified in
162   *         Configuration as comma separated list using key
163   *         "hbase.regionserver.scan.visibility.label.generator.class" when any of the specified
164   *         ScanLabelGenerator class can not be loaded.
165   */
166  public static List<ScanLabelGenerator> getScanLabelGenerators(Configuration conf) {
167    // There can be n SLG specified as comma separated in conf
168    String slgClassesCommaSeparated = conf.get(VISIBILITY_LABEL_GENERATOR_CLASS);
169    // We have only System level SLGs now. The order of execution will be same as the order in the
170    // comma separated config value
171    List<ScanLabelGenerator> slgs = new ArrayList<>();
172    if (StringUtils.isNotEmpty(slgClassesCommaSeparated)) {
173      String[] slgClasses = slgClassesCommaSeparated.split(COMMA);
174      for (String slgClass : slgClasses) {
175        Class<? extends ScanLabelGenerator> slgKlass;
176        try {
177          slgKlass = (Class<? extends ScanLabelGenerator>) conf.getClassByName(slgClass.trim());
178          slgs.add(ReflectionUtils.newInstance(slgKlass, conf));
179        } catch (ClassNotFoundException e) {
180          throw new IllegalArgumentException("Unable to find " + slgClass, e);
181        }
182      }
183    }
184    // If no SLG is specified in conf, by default we'll add two SLGs
185    // 1. FeedUserAuthScanLabelGenerator
186    // 2. DefinedSetFilterScanLabelGenerator
187    // This stacking will achieve the following default behavior:
188    // 1. If there is no Auths in the scan, we will obtain the global defined set for the user
189    // from the labels table.
190    // 2. If there is Auths in the scan, we will examine the passed in Auths and filter out the
191    // labels that the user is not entitled to. Then use the resulting label set.
192    if (slgs.isEmpty()) {
193      slgs.add(ReflectionUtils.newInstance(FeedUserAuthScanLabelGenerator.class, conf));
194      slgs.add(ReflectionUtils.newInstance(DefinedSetFilterScanLabelGenerator.class, conf));
195    }
196    return slgs;
197  }
198
199  /**
200   * Extract the visibility tags of the given Cell into the given List
201   * @param cell - the cell
202   * @param tags - the array that will be populated if visibility tags are present
203   * @return The visibility tags serialization format
204   */
205  public static Byte extractVisibilityTags(ExtendedCell cell, List<Tag> tags) {
206    Byte serializationFormat = null;
207    Iterator<Tag> tagsIterator = PrivateCellUtil.tagsIterator(cell);
208    while (tagsIterator.hasNext()) {
209      Tag tag = tagsIterator.next();
210      if (tag.getType() == TagType.VISIBILITY_EXP_SERIALIZATION_FORMAT_TAG_TYPE) {
211        serializationFormat = Tag.getValueAsByte(tag);
212      } else if (tag.getType() == VISIBILITY_TAG_TYPE) {
213        tags.add(tag);
214      }
215    }
216    return serializationFormat;
217  }
218
219  /**
220   * Extracts and partitions the visibility tags and nonVisibility Tags
221   * @param cell       - the cell for which we would extract and partition the visibility and non
222   *                   visibility tags - all the visibilty tags of type TagType.VISIBILITY_TAG_TYPE
223   *                   would be added to this list
224   * @param nonVisTags - all the non visibility tags would be added to this list
225   * @return - the serailization format of the tag. Can be null if no tags are found or if there is
226   *         no visibility tag found
227   */
228  public static Byte extractAndPartitionTags(ExtendedCell cell, List<Tag> visTags,
229    List<Tag> nonVisTags) {
230    Byte serializationFormat = null;
231    Iterator<Tag> tagsIterator = PrivateCellUtil.tagsIterator(cell);
232    while (tagsIterator.hasNext()) {
233      Tag tag = tagsIterator.next();
234      if (tag.getType() == TagType.VISIBILITY_EXP_SERIALIZATION_FORMAT_TAG_TYPE) {
235        serializationFormat = Tag.getValueAsByte(tag);
236      } else if (tag.getType() == VISIBILITY_TAG_TYPE) {
237        visTags.add(tag);
238      } else {
239        // ignore string encoded visibility expressions, will be added in replication handling
240        nonVisTags.add(tag);
241      }
242    }
243    return serializationFormat;
244  }
245
246  public static boolean isVisibilityTagsPresent(ExtendedCell cell) {
247    Iterator<Tag> tagsIterator = PrivateCellUtil.tagsIterator(cell);
248    while (tagsIterator.hasNext()) {
249      Tag tag = tagsIterator.next();
250      if (tag.getType() == VISIBILITY_TAG_TYPE) {
251        return true;
252      }
253    }
254    return false;
255  }
256
257  public static Filter createVisibilityLabelFilter(Region region, Authorizations authorizations)
258    throws IOException {
259    Map<ByteRange, Integer> cfVsMaxVersions = new HashMap<>();
260    for (ColumnFamilyDescriptor hcd : region.getTableDescriptor().getColumnFamilies()) {
261      cfVsMaxVersions.put(new SimpleMutableByteRange(hcd.getName()), hcd.getMaxVersions());
262    }
263    VisibilityLabelService vls =
264      VisibilityLabelServiceManager.getInstance().getVisibilityLabelService();
265    Filter visibilityLabelFilter =
266      new VisibilityLabelFilter(vls.getVisibilityExpEvaluator(authorizations), cfVsMaxVersions);
267    return visibilityLabelFilter;
268  }
269
270  /**
271   * @return User who called RPC method. For non-RPC handling, falls back to system user
272   * @throws IOException When there is IOE in getting the system user (During non-RPC handling).
273   */
274  public static User getActiveUser() throws IOException {
275    Optional<User> optionalUser = RpcServer.getRequestUser();
276    User user;
277    if (optionalUser.isPresent()) {
278      user = optionalUser.get();
279    } else {
280      user = User.getCurrent();
281    }
282    if (LOG.isTraceEnabled()) {
283      LOG.trace("Current active user name is " + user.getShortName());
284    }
285    return user;
286  }
287
288  public static List<Tag> createVisibilityExpTags(String visExpression,
289    boolean withSerializationFormat, boolean checkAuths, Set<Integer> auths,
290    VisibilityLabelOrdinalProvider ordinalProvider) throws IOException {
291    ExpressionNode node = null;
292    try {
293      node = EXP_PARSER.parse(visExpression);
294    } catch (ParseException e) {
295      throw new IOException(e);
296    }
297    node = EXP_EXPANDER.expand(node);
298    List<Tag> tags = new ArrayList<>();
299    ByteArrayOutputStream baos = new ByteArrayOutputStream();
300    DataOutputStream dos = new DataOutputStream(baos);
301    List<Integer> labelOrdinals = new ArrayList<>();
302    // We will be adding this tag before the visibility tags and the presence of this
303    // tag indicates we are supporting deletes with cell visibility
304    if (withSerializationFormat) {
305      tags.add(VisibilityUtils.SORTED_ORDINAL_SERIALIZATION_FORMAT_TAG);
306    }
307    if (node.isSingleNode()) {
308      getLabelOrdinals(node, labelOrdinals, auths, checkAuths, ordinalProvider);
309      writeLabelOrdinalsToStream(labelOrdinals, dos);
310      tags.add(new ArrayBackedTag(VISIBILITY_TAG_TYPE, baos.toByteArray()));
311      baos.reset();
312    } else {
313      NonLeafExpressionNode nlNode = (NonLeafExpressionNode) node;
314      if (nlNode.getOperator() == Operator.OR) {
315        for (ExpressionNode child : nlNode.getChildExps()) {
316          getLabelOrdinals(child, labelOrdinals, auths, checkAuths, ordinalProvider);
317          writeLabelOrdinalsToStream(labelOrdinals, dos);
318          tags.add(new ArrayBackedTag(VISIBILITY_TAG_TYPE, baos.toByteArray()));
319          baos.reset();
320          labelOrdinals.clear();
321        }
322      } else {
323        getLabelOrdinals(nlNode, labelOrdinals, auths, checkAuths, ordinalProvider);
324        writeLabelOrdinalsToStream(labelOrdinals, dos);
325        tags.add(new ArrayBackedTag(VISIBILITY_TAG_TYPE, baos.toByteArray()));
326        baos.reset();
327      }
328    }
329    return tags;
330  }
331
332  private static void getLabelOrdinals(ExpressionNode node, List<Integer> labelOrdinals,
333    Set<Integer> auths, boolean checkAuths, VisibilityLabelOrdinalProvider ordinalProvider)
334    throws IOException, InvalidLabelException {
335    if (node.isSingleNode()) {
336      String identifier = null;
337      int labelOrdinal = 0;
338      if (node instanceof LeafExpressionNode) {
339        identifier = ((LeafExpressionNode) node).getIdentifier();
340        if (LOG.isTraceEnabled()) {
341          LOG.trace("The identifier is " + identifier);
342        }
343        labelOrdinal = ordinalProvider.getLabelOrdinal(identifier);
344        checkAuths(auths, labelOrdinal, identifier, checkAuths);
345      } else {
346        // This is a NOT node.
347        LeafExpressionNode lNode =
348          (LeafExpressionNode) ((NonLeafExpressionNode) node).getChildExps().get(0);
349        identifier = lNode.getIdentifier();
350        labelOrdinal = ordinalProvider.getLabelOrdinal(identifier);
351        checkAuths(auths, labelOrdinal, identifier, checkAuths);
352        labelOrdinal = -1 * labelOrdinal; // Store NOT node as -ve ordinal.
353      }
354      if (labelOrdinal == 0) {
355        throw new InvalidLabelException("Invalid visibility label " + identifier);
356      }
357      labelOrdinals.add(labelOrdinal);
358    } else {
359      List<ExpressionNode> childExps = ((NonLeafExpressionNode) node).getChildExps();
360      for (ExpressionNode child : childExps) {
361        getLabelOrdinals(child, labelOrdinals, auths, checkAuths, ordinalProvider);
362      }
363    }
364  }
365
366  /**
367   * This will sort the passed labels in ascending oder and then will write one after the other to
368   * the passed stream. Unsorted label ordinals Stream where to write the labels. When IOE during
369   * writes to Stream.
370   */
371  private static void writeLabelOrdinalsToStream(List<Integer> labelOrdinals, DataOutputStream dos)
372    throws IOException {
373    Collections.sort(labelOrdinals);
374    for (Integer labelOrdinal : labelOrdinals) {
375      StreamUtils.writeRawVInt32(dos, labelOrdinal);
376    }
377  }
378
379  private static void checkAuths(Set<Integer> auths, int labelOrdinal, String identifier,
380    boolean checkAuths) throws IOException {
381    if (checkAuths) {
382      if (auths == null || (!auths.contains(labelOrdinal))) {
383        throw new AccessDeniedException("Visibility label " + identifier
384          + " not authorized for the user " + VisibilityUtils.getActiveUser().getShortName());
385      }
386    }
387  }
388}