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