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.Cell;
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(Cell 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(Cell cell, List<Tag> visTags, List<Tag> nonVisTags) {
229    Byte serializationFormat = null;
230    Iterator<Tag> tagsIterator = PrivateCellUtil.tagsIterator(cell);
231    while (tagsIterator.hasNext()) {
232      Tag tag = tagsIterator.next();
233      if (tag.getType() == TagType.VISIBILITY_EXP_SERIALIZATION_FORMAT_TAG_TYPE) {
234        serializationFormat = Tag.getValueAsByte(tag);
235      } else if (tag.getType() == VISIBILITY_TAG_TYPE) {
236        visTags.add(tag);
237      } else {
238        // ignore string encoded visibility expressions, will be added in replication handling
239        nonVisTags.add(tag);
240      }
241    }
242    return serializationFormat;
243  }
244
245  public static boolean isVisibilityTagsPresent(Cell cell) {
246    Iterator<Tag> tagsIterator = PrivateCellUtil.tagsIterator(cell);
247    while (tagsIterator.hasNext()) {
248      Tag tag = tagsIterator.next();
249      if (tag.getType() == VISIBILITY_TAG_TYPE) {
250        return true;
251      }
252    }
253    return false;
254  }
255
256  public static Filter createVisibilityLabelFilter(Region region, Authorizations authorizations)
257    throws IOException {
258    Map<ByteRange, Integer> cfVsMaxVersions = new HashMap<>();
259    for (ColumnFamilyDescriptor hcd : region.getTableDescriptor().getColumnFamilies()) {
260      cfVsMaxVersions.put(new SimpleMutableByteRange(hcd.getName()), hcd.getMaxVersions());
261    }
262    VisibilityLabelService vls =
263      VisibilityLabelServiceManager.getInstance().getVisibilityLabelService();
264    Filter visibilityLabelFilter =
265      new VisibilityLabelFilter(vls.getVisibilityExpEvaluator(authorizations), cfVsMaxVersions);
266    return visibilityLabelFilter;
267  }
268
269  /**
270   * @return User who called RPC method. For non-RPC handling, falls back to system user
271   * @throws IOException When there is IOE in getting the system user (During non-RPC handling).
272   */
273  public static User getActiveUser() throws IOException {
274    Optional<User> optionalUser = RpcServer.getRequestUser();
275    User user;
276    if (optionalUser.isPresent()) {
277      user = optionalUser.get();
278    } else {
279      user = User.getCurrent();
280    }
281    if (LOG.isTraceEnabled()) {
282      LOG.trace("Current active user name is " + user.getShortName());
283    }
284    return user;
285  }
286
287  public static List<Tag> createVisibilityExpTags(String visExpression,
288    boolean withSerializationFormat, boolean checkAuths, Set<Integer> auths,
289    VisibilityLabelOrdinalProvider ordinalProvider) throws IOException {
290    ExpressionNode node = null;
291    try {
292      node = EXP_PARSER.parse(visExpression);
293    } catch (ParseException e) {
294      throw new IOException(e);
295    }
296    node = EXP_EXPANDER.expand(node);
297    List<Tag> tags = new ArrayList<>();
298    ByteArrayOutputStream baos = new ByteArrayOutputStream();
299    DataOutputStream dos = new DataOutputStream(baos);
300    List<Integer> labelOrdinals = new ArrayList<>();
301    // We will be adding this tag before the visibility tags and the presence of this
302    // tag indicates we are supporting deletes with cell visibility
303    if (withSerializationFormat) {
304      tags.add(VisibilityUtils.SORTED_ORDINAL_SERIALIZATION_FORMAT_TAG);
305    }
306    if (node.isSingleNode()) {
307      getLabelOrdinals(node, labelOrdinals, auths, checkAuths, ordinalProvider);
308      writeLabelOrdinalsToStream(labelOrdinals, dos);
309      tags.add(new ArrayBackedTag(VISIBILITY_TAG_TYPE, baos.toByteArray()));
310      baos.reset();
311    } else {
312      NonLeafExpressionNode nlNode = (NonLeafExpressionNode) node;
313      if (nlNode.getOperator() == Operator.OR) {
314        for (ExpressionNode child : nlNode.getChildExps()) {
315          getLabelOrdinals(child, labelOrdinals, auths, checkAuths, ordinalProvider);
316          writeLabelOrdinalsToStream(labelOrdinals, dos);
317          tags.add(new ArrayBackedTag(VISIBILITY_TAG_TYPE, baos.toByteArray()));
318          baos.reset();
319          labelOrdinals.clear();
320        }
321      } else {
322        getLabelOrdinals(nlNode, labelOrdinals, auths, checkAuths, ordinalProvider);
323        writeLabelOrdinalsToStream(labelOrdinals, dos);
324        tags.add(new ArrayBackedTag(VISIBILITY_TAG_TYPE, baos.toByteArray()));
325        baos.reset();
326      }
327    }
328    return tags;
329  }
330
331  private static void getLabelOrdinals(ExpressionNode node, List<Integer> labelOrdinals,
332    Set<Integer> auths, boolean checkAuths, VisibilityLabelOrdinalProvider ordinalProvider)
333    throws IOException, InvalidLabelException {
334    if (node.isSingleNode()) {
335      String identifier = null;
336      int labelOrdinal = 0;
337      if (node instanceof LeafExpressionNode) {
338        identifier = ((LeafExpressionNode) node).getIdentifier();
339        if (LOG.isTraceEnabled()) {
340          LOG.trace("The identifier is " + identifier);
341        }
342        labelOrdinal = ordinalProvider.getLabelOrdinal(identifier);
343        checkAuths(auths, labelOrdinal, identifier, checkAuths);
344      } else {
345        // This is a NOT node.
346        LeafExpressionNode lNode =
347          (LeafExpressionNode) ((NonLeafExpressionNode) node).getChildExps().get(0);
348        identifier = lNode.getIdentifier();
349        labelOrdinal = ordinalProvider.getLabelOrdinal(identifier);
350        checkAuths(auths, labelOrdinal, identifier, checkAuths);
351        labelOrdinal = -1 * labelOrdinal; // Store NOT node as -ve ordinal.
352      }
353      if (labelOrdinal == 0) {
354        throw new InvalidLabelException("Invalid visibility label " + identifier);
355      }
356      labelOrdinals.add(labelOrdinal);
357    } else {
358      List<ExpressionNode> childExps = ((NonLeafExpressionNode) node).getChildExps();
359      for (ExpressionNode child : childExps) {
360        getLabelOrdinals(child, labelOrdinals, auths, checkAuths, ordinalProvider);
361      }
362    }
363  }
364
365  /**
366   * This will sort the passed labels in ascending oder and then will write one after the other to
367   * the passed stream. Unsorted label ordinals Stream where to write the labels. When IOE during
368   * writes to Stream.
369   */
370  private static void writeLabelOrdinalsToStream(List<Integer> labelOrdinals, DataOutputStream dos)
371    throws IOException {
372    Collections.sort(labelOrdinals);
373    for (Integer labelOrdinal : labelOrdinals) {
374      StreamUtils.writeRawVInt32(dos, labelOrdinal);
375    }
376  }
377
378  private static void checkAuths(Set<Integer> auths, int labelOrdinal, String identifier,
379    boolean checkAuths) throws IOException {
380    if (checkAuths) {
381      if (auths == null || (!auths.contains(labelOrdinal))) {
382        throw new AccessDeniedException("Visibility label " + identifier
383          + " not authorized for the user " + VisibilityUtils.getActiveUser().getShortName());
384      }
385    }
386  }
387}