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. n * @return Bytes form of labels and their
088   * 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. n * @return Bytes form of user auths
103   * 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). n * @return Labels and their ordinal
121   * details n
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. n * @return User auth details n
140   */
141  public static MultiUserAuthorizations readUserAuthsFromZKData(byte[] data)
142    throws DeserializationException {
143    if (ProtobufUtil.isPBMagicPrefix(data)) {
144      int pblen = ProtobufUtil.lengthOfPBMagic();
145      try {
146        MultiUserAuthorizations.Builder builder = MultiUserAuthorizations.newBuilder();
147        ProtobufUtil.mergeFrom(builder, data, pblen, data.length - pblen);
148        return builder.build();
149      } catch (IOException e) {
150        throw new DeserializationException(e);
151      }
152    }
153    return null;
154  }
155
156  /**
157   * @param conf The configuration to use
158   * @return Stack of ScanLabelGenerator instances. ScanLabelGenerator classes can be specified in
159   *         Configuration as comma separated list using key
160   *         "hbase.regionserver.scan.visibility.label.generator.class" n * when any of the
161   *         specified ScanLabelGenerator class can not be loaded.
162   */
163  public static List<ScanLabelGenerator> getScanLabelGenerators(Configuration conf) {
164    // There can be n SLG specified as comma separated in conf
165    String slgClassesCommaSeparated = conf.get(VISIBILITY_LABEL_GENERATOR_CLASS);
166    // We have only System level SLGs now. The order of execution will be same as the order in the
167    // comma separated config value
168    List<ScanLabelGenerator> slgs = new ArrayList<>();
169    if (StringUtils.isNotEmpty(slgClassesCommaSeparated)) {
170      String[] slgClasses = slgClassesCommaSeparated.split(COMMA);
171      for (String slgClass : slgClasses) {
172        Class<? extends ScanLabelGenerator> slgKlass;
173        try {
174          slgKlass = (Class<? extends ScanLabelGenerator>) conf.getClassByName(slgClass.trim());
175          slgs.add(ReflectionUtils.newInstance(slgKlass, conf));
176        } catch (ClassNotFoundException e) {
177          throw new IllegalArgumentException("Unable to find " + slgClass, e);
178        }
179      }
180    }
181    // If no SLG is specified in conf, by default we'll add two SLGs
182    // 1. FeedUserAuthScanLabelGenerator
183    // 2. DefinedSetFilterScanLabelGenerator
184    // This stacking will achieve the following default behavior:
185    // 1. If there is no Auths in the scan, we will obtain the global defined set for the user
186    // from the labels table.
187    // 2. If there is Auths in the scan, we will examine the passed in Auths and filter out the
188    // labels that the user is not entitled to. Then use the resulting label set.
189    if (slgs.isEmpty()) {
190      slgs.add(ReflectionUtils.newInstance(FeedUserAuthScanLabelGenerator.class, conf));
191      slgs.add(ReflectionUtils.newInstance(DefinedSetFilterScanLabelGenerator.class, conf));
192    }
193    return slgs;
194  }
195
196  /**
197   * Extract the visibility tags of the given Cell into the given List
198   * @param cell - the cell
199   * @param tags - the array that will be populated if visibility tags are present
200   * @return The visibility tags serialization format
201   */
202  public static Byte extractVisibilityTags(Cell cell, List<Tag> tags) {
203    Byte serializationFormat = null;
204    Iterator<Tag> tagsIterator = PrivateCellUtil.tagsIterator(cell);
205    while (tagsIterator.hasNext()) {
206      Tag tag = tagsIterator.next();
207      if (tag.getType() == TagType.VISIBILITY_EXP_SERIALIZATION_FORMAT_TAG_TYPE) {
208        serializationFormat = Tag.getValueAsByte(tag);
209      } else if (tag.getType() == VISIBILITY_TAG_TYPE) {
210        tags.add(tag);
211      }
212    }
213    return serializationFormat;
214  }
215
216  /**
217   * Extracts and partitions the visibility tags and nonVisibility Tags
218   * @param cell       - the cell for which we would extract and partition the visibility and non
219   *                   visibility tags n * - all the visibilty tags of type
220   *                   TagType.VISIBILITY_TAG_TYPE would be added to this list
221   * @param nonVisTags - all the non visibility tags would be added to this list
222   * @return - the serailization format of the tag. Can be null if no tags are found or if there is
223   *         no visibility tag found
224   */
225  public static Byte extractAndPartitionTags(Cell cell, List<Tag> visTags, List<Tag> nonVisTags) {
226    Byte serializationFormat = null;
227    Iterator<Tag> tagsIterator = PrivateCellUtil.tagsIterator(cell);
228    while (tagsIterator.hasNext()) {
229      Tag tag = tagsIterator.next();
230      if (tag.getType() == TagType.VISIBILITY_EXP_SERIALIZATION_FORMAT_TAG_TYPE) {
231        serializationFormat = Tag.getValueAsByte(tag);
232      } else if (tag.getType() == VISIBILITY_TAG_TYPE) {
233        visTags.add(tag);
234      } else {
235        // ignore string encoded visibility expressions, will be added in replication handling
236        nonVisTags.add(tag);
237      }
238    }
239    return serializationFormat;
240  }
241
242  public static boolean isVisibilityTagsPresent(Cell cell) {
243    Iterator<Tag> tagsIterator = PrivateCellUtil.tagsIterator(cell);
244    while (tagsIterator.hasNext()) {
245      Tag tag = tagsIterator.next();
246      if (tag.getType() == VISIBILITY_TAG_TYPE) {
247        return true;
248      }
249    }
250    return false;
251  }
252
253  public static Filter createVisibilityLabelFilter(Region region, Authorizations authorizations)
254    throws IOException {
255    Map<ByteRange, Integer> cfVsMaxVersions = new HashMap<>();
256    for (ColumnFamilyDescriptor hcd : region.getTableDescriptor().getColumnFamilies()) {
257      cfVsMaxVersions.put(new SimpleMutableByteRange(hcd.getName()), hcd.getMaxVersions());
258    }
259    VisibilityLabelService vls =
260      VisibilityLabelServiceManager.getInstance().getVisibilityLabelService();
261    Filter visibilityLabelFilter =
262      new VisibilityLabelFilter(vls.getVisibilityExpEvaluator(authorizations), cfVsMaxVersions);
263    return visibilityLabelFilter;
264  }
265
266  /**
267   * @return User who called RPC method. For non-RPC handling, falls back to system user
268   * @throws IOException When there is IOE in getting the system user (During non-RPC handling).
269   */
270  public static User getActiveUser() throws IOException {
271    Optional<User> optionalUser = RpcServer.getRequestUser();
272    User user;
273    if (optionalUser.isPresent()) {
274      user = optionalUser.get();
275    } else {
276      user = User.getCurrent();
277    }
278    if (LOG.isTraceEnabled()) {
279      LOG.trace("Current active user name is " + user.getShortName());
280    }
281    return user;
282  }
283
284  public static List<Tag> createVisibilityExpTags(String visExpression,
285    boolean withSerializationFormat, boolean checkAuths, Set<Integer> auths,
286    VisibilityLabelOrdinalProvider ordinalProvider) throws IOException {
287    ExpressionNode node = null;
288    try {
289      node = EXP_PARSER.parse(visExpression);
290    } catch (ParseException e) {
291      throw new IOException(e);
292    }
293    node = EXP_EXPANDER.expand(node);
294    List<Tag> tags = new ArrayList<>();
295    ByteArrayOutputStream baos = new ByteArrayOutputStream();
296    DataOutputStream dos = new DataOutputStream(baos);
297    List<Integer> labelOrdinals = new ArrayList<>();
298    // We will be adding this tag before the visibility tags and the presence of this
299    // tag indicates we are supporting deletes with cell visibility
300    if (withSerializationFormat) {
301      tags.add(VisibilityUtils.SORTED_ORDINAL_SERIALIZATION_FORMAT_TAG);
302    }
303    if (node.isSingleNode()) {
304      getLabelOrdinals(node, labelOrdinals, auths, checkAuths, ordinalProvider);
305      writeLabelOrdinalsToStream(labelOrdinals, dos);
306      tags.add(new ArrayBackedTag(VISIBILITY_TAG_TYPE, baos.toByteArray()));
307      baos.reset();
308    } else {
309      NonLeafExpressionNode nlNode = (NonLeafExpressionNode) node;
310      if (nlNode.getOperator() == Operator.OR) {
311        for (ExpressionNode child : nlNode.getChildExps()) {
312          getLabelOrdinals(child, labelOrdinals, auths, checkAuths, ordinalProvider);
313          writeLabelOrdinalsToStream(labelOrdinals, dos);
314          tags.add(new ArrayBackedTag(VISIBILITY_TAG_TYPE, baos.toByteArray()));
315          baos.reset();
316          labelOrdinals.clear();
317        }
318      } else {
319        getLabelOrdinals(nlNode, labelOrdinals, auths, checkAuths, ordinalProvider);
320        writeLabelOrdinalsToStream(labelOrdinals, dos);
321        tags.add(new ArrayBackedTag(VISIBILITY_TAG_TYPE, baos.toByteArray()));
322        baos.reset();
323      }
324    }
325    return tags;
326  }
327
328  private static void getLabelOrdinals(ExpressionNode node, List<Integer> labelOrdinals,
329    Set<Integer> auths, boolean checkAuths, VisibilityLabelOrdinalProvider ordinalProvider)
330    throws IOException, InvalidLabelException {
331    if (node.isSingleNode()) {
332      String identifier = null;
333      int labelOrdinal = 0;
334      if (node instanceof LeafExpressionNode) {
335        identifier = ((LeafExpressionNode) node).getIdentifier();
336        if (LOG.isTraceEnabled()) {
337          LOG.trace("The identifier is " + identifier);
338        }
339        labelOrdinal = ordinalProvider.getLabelOrdinal(identifier);
340        checkAuths(auths, labelOrdinal, identifier, checkAuths);
341      } else {
342        // This is a NOT node.
343        LeafExpressionNode lNode =
344          (LeafExpressionNode) ((NonLeafExpressionNode) node).getChildExps().get(0);
345        identifier = lNode.getIdentifier();
346        labelOrdinal = ordinalProvider.getLabelOrdinal(identifier);
347        checkAuths(auths, labelOrdinal, identifier, checkAuths);
348        labelOrdinal = -1 * labelOrdinal; // Store NOT node as -ve ordinal.
349      }
350      if (labelOrdinal == 0) {
351        throw new InvalidLabelException("Invalid visibility label " + identifier);
352      }
353      labelOrdinals.add(labelOrdinal);
354    } else {
355      List<ExpressionNode> childExps = ((NonLeafExpressionNode) node).getChildExps();
356      for (ExpressionNode child : childExps) {
357        getLabelOrdinals(child, labelOrdinals, auths, checkAuths, ordinalProvider);
358      }
359    }
360  }
361
362  /**
363   * This will sort the passed labels in ascending oder and then will write one after the other to
364   * the passed stream. n * Unsorted label ordinals n * Stream where to write the labels. n * When
365   * IOE during writes to Stream.
366   */
367  private static void writeLabelOrdinalsToStream(List<Integer> labelOrdinals, DataOutputStream dos)
368    throws IOException {
369    Collections.sort(labelOrdinals);
370    for (Integer labelOrdinal : labelOrdinals) {
371      StreamUtils.writeRawVInt32(dos, labelOrdinal);
372    }
373  }
374
375  private static void checkAuths(Set<Integer> auths, int labelOrdinal, String identifier,
376    boolean checkAuths) throws IOException {
377    if (checkAuths) {
378      if (auths == null || (!auths.contains(labelOrdinal))) {
379        throw new AccessDeniedException("Visibility label " + identifier
380          + " not authorized for the user " + VisibilityUtils.getActiveUser().getShortName());
381      }
382    }
383  }
384}