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;
021import static org.apache.hadoop.hbase.security.visibility.VisibilityConstants.LABELS_TABLE_FAMILY;
022import static org.apache.hadoop.hbase.security.visibility.VisibilityConstants.LABELS_TABLE_NAME;
023import static org.apache.hadoop.hbase.security.visibility.VisibilityUtils.SYSTEM_LABEL;
024
025import java.io.ByteArrayOutputStream;
026import java.io.DataOutputStream;
027import java.io.IOException;
028import java.util.ArrayList;
029import java.util.Collections;
030import java.util.HashSet;
031import java.util.Iterator;
032import java.util.List;
033import java.util.Set;
034
035import org.apache.hadoop.conf.Configuration;
036import org.apache.hadoop.hbase.ArrayBackedTag;
037import org.apache.hadoop.hbase.AuthUtil;
038import org.apache.hadoop.hbase.Cell;
039import org.apache.hadoop.hbase.CellBuilder;
040import org.apache.hadoop.hbase.CellBuilderFactory;
041import org.apache.hadoop.hbase.CellBuilderType;
042import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
043import org.apache.hadoop.hbase.PrivateCellUtil;
044import org.apache.hadoop.hbase.Tag;
045import org.apache.hadoop.hbase.TagType;
046import org.apache.hadoop.hbase.client.Connection;
047import org.apache.hadoop.hbase.client.ConnectionFactory;
048import org.apache.hadoop.hbase.client.Delete;
049import org.apache.hadoop.hbase.client.Get;
050import org.apache.hadoop.hbase.client.Put;
051import org.apache.hadoop.hbase.client.Result;
052import org.apache.hadoop.hbase.client.Table;
053import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
054import org.apache.hadoop.hbase.regionserver.OperationStatus;
055import org.apache.hadoop.hbase.regionserver.Region;
056import org.apache.hadoop.hbase.security.Superusers;
057import org.apache.hadoop.hbase.security.User;
058import org.apache.hadoop.hbase.security.visibility.expression.ExpressionNode;
059import org.apache.hadoop.hbase.security.visibility.expression.LeafExpressionNode;
060import org.apache.hadoop.hbase.security.visibility.expression.NonLeafExpressionNode;
061import org.apache.hadoop.hbase.security.visibility.expression.Operator;
062import org.apache.hadoop.hbase.util.ByteBufferUtils;
063import org.apache.hadoop.hbase.util.Bytes;
064import org.apache.yetus.audience.InterfaceAudience;
065import org.slf4j.Logger;
066import org.slf4j.LoggerFactory;
067
068/**
069 * This is a VisibilityLabelService where labels in Mutation's visibility
070 * expression will be persisted as Strings itself rather than ordinals in
071 * 'labels' table. Also there is no need to add labels to the system, prior to
072 * using them in Mutations/Authorizations.
073 */
074@InterfaceAudience.Private
075public class ExpAsStringVisibilityLabelServiceImpl implements VisibilityLabelService {
076  private static final Logger LOG =
077      LoggerFactory.getLogger(ExpAsStringVisibilityLabelServiceImpl.class);
078
079  private static final byte[] DUMMY_VALUE = new byte[0];
080  private static final byte STRING_SERIALIZATION_FORMAT = 2;
081  private static final Tag STRING_SERIALIZATION_FORMAT_TAG = new ArrayBackedTag(
082      TagType.VISIBILITY_EXP_SERIALIZATION_FORMAT_TAG_TYPE,
083      new byte[] { STRING_SERIALIZATION_FORMAT });
084  private final ExpressionParser expressionParser = new ExpressionParser();
085  private final ExpressionExpander expressionExpander = new ExpressionExpander();
086  private Configuration conf;
087  private Region labelsRegion;
088  private List<ScanLabelGenerator> scanLabelGenerators;
089
090  @Override
091  public OperationStatus[] addLabels(List<byte[]> labels) throws IOException {
092    // Not doing specific label add. We will just add labels in Mutation
093    // visibility expression as it
094    // is along with every cell.
095    OperationStatus[] status = new OperationStatus[labels.size()];
096    for (int i = 0; i < labels.size(); i++) {
097      status[i] = new OperationStatus(OperationStatusCode.SUCCESS);
098    }
099    return status;
100  }
101
102  @Override
103  public OperationStatus[] setAuths(byte[] user, List<byte[]> authLabels) throws IOException {
104    assert labelsRegion != null;
105    OperationStatus[] finalOpStatus = new OperationStatus[authLabels.size()];
106    Put p = new Put(user);
107    CellBuilder builder = CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY);
108    for (byte[] auth : authLabels) {
109      p.add(builder.clear()
110          .setRow(p.getRow())
111          .setFamily(LABELS_TABLE_FAMILY)
112          .setQualifier(auth)
113          .setTimestamp(p.getTimestamp())
114          .setType(Cell.Type.Put)
115          .setValue(DUMMY_VALUE)
116          .build());
117    }
118    this.labelsRegion.put(p);
119    // This is a testing impl and so not doing any caching
120    for (int i = 0; i < authLabels.size(); i++) {
121      finalOpStatus[i] = new OperationStatus(OperationStatusCode.SUCCESS);
122    }
123    return finalOpStatus;
124  }
125
126  @Override
127  public OperationStatus[] clearAuths(byte[] user, List<byte[]> authLabels) throws IOException {
128    assert labelsRegion != null;
129    OperationStatus[] finalOpStatus = new OperationStatus[authLabels.size()];
130    List<String> currentAuths;
131    if (AuthUtil.isGroupPrincipal(Bytes.toString(user))) {
132      String group = AuthUtil.getGroupName(Bytes.toString(user));
133      currentAuths = this.getGroupAuths(new String[]{group}, true);
134    }
135    else {
136      currentAuths = this.getUserAuths(user, true);
137    }
138    Delete d = new Delete(user);
139    int i = 0;
140    for (byte[] authLabel : authLabels) {
141      String authLabelStr = Bytes.toString(authLabel);
142      if (currentAuths.contains(authLabelStr)) {
143        d.addColumns(LABELS_TABLE_FAMILY, authLabel);
144      } else {
145        // This label is not set for the user.
146        finalOpStatus[i] = new OperationStatus(OperationStatusCode.FAILURE,
147            new InvalidLabelException("Label '" + authLabelStr + "' is not set for the user "
148                + Bytes.toString(user)));
149      }
150      i++;
151    }
152    this.labelsRegion.delete(d);
153    // This is a testing impl and so not doing any caching
154    for (i = 0; i < authLabels.size(); i++) {
155      if (finalOpStatus[i] == null) {
156        finalOpStatus[i] = new OperationStatus(OperationStatusCode.SUCCESS);
157      }
158    }
159    return finalOpStatus;
160  }
161
162  @Override
163  public List<String> getUserAuths(byte[] user, boolean systemCall) throws IOException {
164    assert (labelsRegion != null || systemCall);
165    List<String> auths = new ArrayList<>();
166    Get get = new Get(user);
167    List<Cell> cells = null;
168    if (labelsRegion == null) {
169      Table table = null;
170      Connection connection = null;
171      try {
172        connection = ConnectionFactory.createConnection(conf);
173        table = connection.getTable(VisibilityConstants.LABELS_TABLE_NAME);
174        Result result = table.get(get);
175        cells = result.listCells();
176      } finally {
177        if (table != null) {
178          table.close();
179        }
180        if (connection != null){
181          connection.close();
182        }
183      }
184    } else {
185      cells = this.labelsRegion.get(get, false);
186    }
187    if (cells != null) {
188      for (Cell cell : cells) {
189        String auth = Bytes.toString(cell.getQualifierArray(), cell.getQualifierOffset(),
190          cell.getQualifierLength());
191        auths.add(auth);
192      }
193    }
194    return auths;
195  }
196
197  @Override
198  public List<String> getGroupAuths(String[] groups, boolean systemCall) throws IOException {
199    assert (labelsRegion != null || systemCall);
200    List<String> auths = new ArrayList<>();
201    if (groups != null && groups.length > 0) {
202      for (String group : groups) {
203        Get get = new Get(Bytes.toBytes(AuthUtil.toGroupEntry(group)));
204        List<Cell> cells = null;
205        if (labelsRegion == null) {
206          Table table = null;
207          Connection connection = null;
208          try {
209            connection = ConnectionFactory.createConnection(conf);
210            table = connection.getTable(VisibilityConstants.LABELS_TABLE_NAME);
211            Result result = table.get(get);
212            cells = result.listCells();
213          } finally {
214            if (table != null) {
215              table.close();
216              connection.close();
217            }
218          }
219        } else {
220          cells = this.labelsRegion.get(get, false);
221        }
222        if (cells != null) {
223          for (Cell cell : cells) {
224            String auth = Bytes.toString(cell.getQualifierArray(), cell.getQualifierOffset(),
225              cell.getQualifierLength());
226            auths.add(auth);
227          }
228        }
229      }
230    }
231    return auths;
232  }
233
234  @Override
235  public List<String> listLabels(String regex) throws IOException {
236    // return an empty list for this implementation.
237    return new ArrayList<>();
238  }
239
240  @Override
241  public List<Tag> createVisibilityExpTags(String visExpression, boolean withSerializationFormat,
242      boolean checkAuths) throws IOException {
243    ExpressionNode node = null;
244    try {
245      node = this.expressionParser.parse(visExpression);
246    } catch (ParseException e) {
247      throw new IOException(e);
248    }
249    node = this.expressionExpander.expand(node);
250    List<Tag> tags = new ArrayList<>();
251    if (withSerializationFormat) {
252      tags.add(STRING_SERIALIZATION_FORMAT_TAG);
253    }
254    if (node instanceof NonLeafExpressionNode
255        && ((NonLeafExpressionNode) node).getOperator() == Operator.OR) {
256      for (ExpressionNode child : ((NonLeafExpressionNode) node).getChildExps()) {
257        tags.add(createTag(child));
258      }
259    } else {
260      tags.add(createTag(node));
261    }
262    return tags;
263  }
264
265  @Override
266  public VisibilityExpEvaluator getVisibilityExpEvaluator(Authorizations authorizations)
267      throws IOException {
268    // If a super user issues a get/scan, he should be able to scan the cells
269    // irrespective of the Visibility labels
270    if (isReadFromSystemAuthUser()) {
271      return new VisibilityExpEvaluator() {
272        @Override
273        public boolean evaluate(Cell cell) throws IOException {
274          return true;
275        }
276      };
277    }
278    List<String> authLabels = null;
279    for (ScanLabelGenerator scanLabelGenerator : scanLabelGenerators) {
280      try {
281        // null authorizations to be handled inside SLG impl.
282        authLabels = scanLabelGenerator.getLabels(VisibilityUtils.getActiveUser(), authorizations);
283        authLabels = (authLabels == null) ? new ArrayList<>() : authLabels;
284        authorizations = new Authorizations(authLabels);
285      } catch (Throwable t) {
286        LOG.error(t.toString(), t);
287        throw new IOException(t);
288      }
289    }
290    final List<String> authLabelsFinal = authLabels;
291    return new VisibilityExpEvaluator() {
292      @Override
293      public boolean evaluate(Cell cell) throws IOException {
294        boolean visibilityTagPresent = false;
295        // Save an object allocation where we can
296        if (cell.getTagsLength() > 0) {
297          Iterator<Tag> tagsItr = PrivateCellUtil.tagsIterator(cell);
298          while (tagsItr.hasNext()) {
299            boolean includeKV = true;
300            Tag tag = tagsItr.next();
301            if (tag.getType() == VISIBILITY_TAG_TYPE) {
302              visibilityTagPresent = true;
303              int offset = tag.getValueOffset();
304              int endOffset = offset + tag.getValueLength();
305              while (offset < endOffset) {
306                short len = getTagValuePartAsShort(tag, offset);
307                offset += 2;
308                if (len < 0) {
309                  // This is a NOT label.
310                  len = (short) (-1 * len);
311                  String label = getTagValuePartAsString(tag, offset, len);
312                  if (authLabelsFinal.contains(label)) {
313                    includeKV = false;
314                    break;
315                  }
316                } else {
317                  String label = getTagValuePartAsString(tag, offset, len);
318                  if (!authLabelsFinal.contains(label)) {
319                    includeKV = false;
320                    break;
321                  }
322                }
323                offset += len;
324              }
325              if (includeKV) {
326                // We got one visibility expression getting evaluated to true.
327                // Good to include this
328                // KV in the result then.
329                return true;
330              }
331            }
332          }
333        }
334        return !(visibilityTagPresent);
335      }
336    };
337  }
338
339  protected boolean isReadFromSystemAuthUser() throws IOException {
340    User user = VisibilityUtils.getActiveUser();
341    return havingSystemAuth(user);
342  }
343
344  private Tag createTag(ExpressionNode node) throws IOException {
345    ByteArrayOutputStream baos = new ByteArrayOutputStream();
346    DataOutputStream dos = new DataOutputStream(baos);
347    List<String> labels = new ArrayList<>();
348    List<String> notLabels = new ArrayList<>();
349    extractLabels(node, labels, notLabels);
350    Collections.sort(labels);
351    Collections.sort(notLabels);
352    // We will write the NOT labels 1st followed by normal labels
353    // Each of the label we will write with label length (as short 1st) followed
354    // by the label bytes.
355    // For a NOT node we will write the label length as -ve.
356    for (String label : notLabels) {
357      byte[] bLabel = Bytes.toBytes(label);
358      short length = (short) bLabel.length;
359      length = (short) (-1 * length);
360      dos.writeShort(length);
361      dos.write(bLabel);
362    }
363    for (String label : labels) {
364      byte[] bLabel = Bytes.toBytes(label);
365      dos.writeShort(bLabel.length);
366      dos.write(bLabel);
367    }
368    return new ArrayBackedTag(VISIBILITY_TAG_TYPE, baos.toByteArray());
369  }
370
371  private void extractLabels(ExpressionNode node, List<String> labels, List<String> notLabels) {
372    if (node.isSingleNode()) {
373      if (node instanceof NonLeafExpressionNode) {
374        // This is a NOT node.
375        LeafExpressionNode lNode = (LeafExpressionNode) ((NonLeafExpressionNode) node)
376            .getChildExps().get(0);
377        notLabels.add(lNode.getIdentifier());
378      } else {
379        labels.add(((LeafExpressionNode) node).getIdentifier());
380      }
381    } else {
382      // A non leaf expression of labels with & operator.
383      NonLeafExpressionNode nlNode = (NonLeafExpressionNode) node;
384      assert nlNode.getOperator() == Operator.AND;
385      List<ExpressionNode> childExps = nlNode.getChildExps();
386      for (ExpressionNode child : childExps) {
387        extractLabels(child, labels, notLabels);
388      }
389    }
390  }
391
392  @Override
393  public Configuration getConf() {
394    return this.conf;
395  }
396
397  @Override
398  public void setConf(Configuration conf) {
399    this.conf = conf;
400  }
401
402  @Override
403  public void init(RegionCoprocessorEnvironment e) throws IOException {
404    this.scanLabelGenerators = VisibilityUtils.getScanLabelGenerators(this.conf);
405    if (e.getRegion().getRegionInfo().getTable().equals(LABELS_TABLE_NAME)) {
406      this.labelsRegion = e.getRegion();
407    }
408  }
409
410  @Override
411  public boolean havingSystemAuth(User user) throws IOException {
412    if (Superusers.isSuperUser(user)) {
413      return true;
414    }
415    Set<String> auths = new HashSet<>();
416    auths.addAll(this.getUserAuths(Bytes.toBytes(user.getShortName()), true));
417    auths.addAll(this.getGroupAuths(user.getGroupNames(), true));
418    return auths.contains(SYSTEM_LABEL);
419  }
420
421  @Override
422  public boolean matchVisibility(List<Tag> putTags, Byte putTagsFormat, List<Tag> deleteTags,
423      Byte deleteTagsFormat) throws IOException {
424    assert putTagsFormat == STRING_SERIALIZATION_FORMAT;
425    assert deleteTagsFormat == STRING_SERIALIZATION_FORMAT;
426    return checkForMatchingVisibilityTagsWithSortedOrder(putTags, deleteTags);
427  }
428
429  private static boolean checkForMatchingVisibilityTagsWithSortedOrder(List<Tag> putVisTags,
430      List<Tag> deleteVisTags) {
431    // Early out if there are no tags in both of cell and delete
432    if (putVisTags.isEmpty() && deleteVisTags.isEmpty()) {
433      return true;
434    }
435    boolean matchFound = false;
436    // If the size does not match. Definitely we are not comparing the equal
437    // tags.
438    if ((deleteVisTags.size()) == putVisTags.size()) {
439      for (Tag tag : deleteVisTags) {
440        matchFound = false;
441        for (Tag givenTag : putVisTags) {
442          if (Tag.matchingValue(tag, givenTag)) {
443            matchFound = true;
444            break;
445          }
446        }
447        if (!matchFound)
448          break;
449      }
450    }
451    return matchFound;
452  }
453
454  @Override
455  public byte[] encodeVisibilityForReplication(final List<Tag> tags, final Byte serializationFormat)
456      throws IOException {
457    if (tags.size() > 0 && (serializationFormat == null
458        || serializationFormat == STRING_SERIALIZATION_FORMAT)) {
459      return createModifiedVisExpression(tags);
460    }
461    return null;
462  }
463
464  /**
465   * @param tags - all the tags associated with the current Cell
466   * @return - the modified visibility expression as byte[]
467   */
468  private byte[] createModifiedVisExpression(final List<Tag> tags)
469      throws IOException {
470    StringBuilder visibilityString = new StringBuilder();
471    for (Tag tag : tags) {
472      if (tag.getType() == TagType.VISIBILITY_TAG_TYPE) {
473        if (visibilityString.length() != 0) {
474          visibilityString.append(VisibilityConstants.CLOSED_PARAN
475              + VisibilityConstants.OR_OPERATOR);
476        }
477        int offset = tag.getValueOffset();
478        int endOffset = offset + tag.getValueLength();
479        boolean expressionStart = true;
480        while (offset < endOffset) {
481          short len = getTagValuePartAsShort(tag, offset);
482          offset += 2;
483          if (len < 0) {
484            len = (short) (-1 * len);
485            String label = getTagValuePartAsString(tag, offset, len);
486            if (expressionStart) {
487              visibilityString.append(VisibilityConstants.OPEN_PARAN
488                  + VisibilityConstants.NOT_OPERATOR + CellVisibility.quote(label));
489            } else {
490              visibilityString.append(VisibilityConstants.AND_OPERATOR
491                  + VisibilityConstants.NOT_OPERATOR + CellVisibility.quote(label));
492            }
493          } else {
494            String label = getTagValuePartAsString(tag, offset, len);
495            if (expressionStart) {
496              visibilityString.append(VisibilityConstants.OPEN_PARAN + CellVisibility.quote(label));
497            } else {
498              visibilityString.append(VisibilityConstants.AND_OPERATOR
499                  + CellVisibility.quote(label));
500            }
501          }
502          expressionStart = false;
503          offset += len;
504        }
505      }
506    }
507    if (visibilityString.length() != 0) {
508      visibilityString.append(VisibilityConstants.CLOSED_PARAN);
509      // Return the string formed as byte[]
510      return Bytes.toBytes(visibilityString.toString());
511    }
512    return null;
513  }
514
515  private static short getTagValuePartAsShort(Tag t, int offset) {
516    if (t.hasArray()) {
517      return Bytes.toShort(t.getValueArray(), offset);
518    }
519    return ByteBufferUtils.toShort(t.getValueByteBuffer(), offset);
520  }
521
522  private static String getTagValuePartAsString(Tag t, int offset, int length) {
523    if (t.hasArray()) {
524      return Bytes.toString(t.getValueArray(), offset, length);
525    }
526    byte[] b = new byte[length];
527    ByteBufferUtils.copyFromBufferToArray(b, t.getValueByteBuffer(), offset, 0, length);
528    return Bytes.toString(b);
529  }
530}