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.VisibilityConstants.LABEL_QUALIFIER;
024import static org.apache.hadoop.hbase.security.visibility.VisibilityConstants.SORTED_ORDINAL_SERIALIZATION_FORMAT;
025import static org.apache.hadoop.hbase.security.visibility.VisibilityUtils.SYSTEM_LABEL;
026
027import java.io.ByteArrayOutputStream;
028import java.io.DataOutputStream;
029import java.io.IOException;
030import java.util.ArrayList;
031import java.util.Arrays;
032import java.util.BitSet;
033import java.util.Collections;
034import java.util.HashMap;
035import java.util.HashSet;
036import java.util.Iterator;
037import java.util.List;
038import java.util.Map;
039import java.util.Set;
040import java.util.concurrent.atomic.AtomicInteger;
041import java.util.regex.Pattern;
042import org.apache.hadoop.conf.Configuration;
043import org.apache.hadoop.hbase.ArrayBackedTag;
044import org.apache.hadoop.hbase.AuthUtil;
045import org.apache.hadoop.hbase.Cell;
046import org.apache.hadoop.hbase.Cell.Type;
047import org.apache.hadoop.hbase.CellBuilderFactory;
048import org.apache.hadoop.hbase.CellBuilderType;
049import org.apache.hadoop.hbase.CellUtil;
050import org.apache.hadoop.hbase.ExtendedCellBuilder;
051import org.apache.hadoop.hbase.ExtendedCellBuilderFactory;
052import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
053import org.apache.hadoop.hbase.PrivateCellUtil;
054import org.apache.hadoop.hbase.Tag;
055import org.apache.hadoop.hbase.TagType;
056import org.apache.hadoop.hbase.TagUtil;
057import org.apache.hadoop.hbase.client.Delete;
058import org.apache.hadoop.hbase.client.Mutation;
059import org.apache.hadoop.hbase.client.Put;
060import org.apache.hadoop.hbase.client.Scan;
061import org.apache.hadoop.hbase.coprocessor.HasRegionServerServices;
062import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
063import org.apache.hadoop.hbase.filter.Filter;
064import org.apache.hadoop.hbase.io.util.StreamUtils;
065import org.apache.hadoop.hbase.regionserver.OperationStatus;
066import org.apache.hadoop.hbase.regionserver.Region;
067import org.apache.hadoop.hbase.regionserver.RegionScanner;
068import org.apache.hadoop.hbase.security.Superusers;
069import org.apache.hadoop.hbase.security.User;
070import org.apache.hadoop.hbase.util.Bytes;
071import org.apache.hadoop.hbase.util.Pair;
072import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
073import org.apache.yetus.audience.InterfaceAudience;
074import org.slf4j.Logger;
075import org.slf4j.LoggerFactory;
076
077@InterfaceAudience.Private
078public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService {
079  private static final Logger LOG =
080    LoggerFactory.getLogger(DefaultVisibilityLabelServiceImpl.class);
081
082  // "system" label is having an ordinal value 1.
083  private static final int SYSTEM_LABEL_ORDINAL = 1;
084  private static final Tag[] LABELS_TABLE_TAGS = new Tag[1];
085  private static final byte[] DUMMY_VALUE = new byte[0];
086
087  private AtomicInteger ordinalCounter = new AtomicInteger(-1);
088  private Configuration conf;
089  private Region labelsRegion;
090  private VisibilityLabelsCache labelsCache;
091  private List<ScanLabelGenerator> scanLabelGenerators;
092
093  static {
094    ByteArrayOutputStream baos = new ByteArrayOutputStream();
095    DataOutputStream dos = new DataOutputStream(baos);
096    try {
097      StreamUtils.writeRawVInt32(dos, SYSTEM_LABEL_ORDINAL);
098    } catch (IOException e) {
099      // We write to a byte array. No Exception can happen.
100    }
101    LABELS_TABLE_TAGS[0] = new ArrayBackedTag(VISIBILITY_TAG_TYPE, baos.toByteArray());
102  }
103
104  public DefaultVisibilityLabelServiceImpl() {
105
106  }
107
108  @Override
109  public void setConf(Configuration conf) {
110    this.conf = conf;
111  }
112
113  @Override
114  public Configuration getConf() {
115    return this.conf;
116  }
117
118  @Override
119  public void init(RegionCoprocessorEnvironment e) throws IOException {
120    /*
121     * So, presumption that the RegionCE has a ZK Connection is too much. Why would a RCE have a ZK
122     * instance? This is cheating presuming we have access to the RS ZKW. TODO: Fix. And what is
123     * going on here? This ain't even a Coprocessor? And its being passed a CP Env?
124     */
125    // This is a CoreCoprocessor. On creation, we should have gotten an environment that
126    // implements HasRegionServerServices so we can get at RSS. FIX!!!! Integrate this CP as
127    // native service.
128    ZKWatcher zk = ((HasRegionServerServices) e).getRegionServerServices().getZooKeeper();
129    try {
130      labelsCache = VisibilityLabelsCache.createAndGet(zk, this.conf);
131    } catch (IOException ioe) {
132      LOG.error("Error creating VisibilityLabelsCache", ioe);
133      throw ioe;
134    }
135    this.scanLabelGenerators = VisibilityUtils.getScanLabelGenerators(this.conf);
136    if (e.getRegion().getRegionInfo().getTable().equals(LABELS_TABLE_NAME)) {
137      this.labelsRegion = e.getRegion();
138      Pair<Map<String, Integer>, Map<String, List<Integer>>> labelsAndUserAuths =
139        extractLabelsAndAuths(getExistingLabelsWithAuths());
140      Map<String, Integer> labels = labelsAndUserAuths.getFirst();
141      Map<String, List<Integer>> userAuths = labelsAndUserAuths.getSecond();
142      // Add the "system" label if it is not added into the system yet
143      addSystemLabel(this.labelsRegion, labels, userAuths);
144      int ordinal = SYSTEM_LABEL_ORDINAL; // Ordinal 1 is reserved for "system" label.
145      for (Integer i : labels.values()) {
146        if (i > ordinal) {
147          ordinal = i;
148        }
149      }
150      this.ordinalCounter.set(ordinal + 1);
151      if (labels.size() > 0) {
152        // If there is no data need not write to zk
153        byte[] serialized = VisibilityUtils.getDataToWriteToZooKeeper(labels);
154        this.labelsCache.writeToZookeeper(serialized, true);
155        this.labelsCache.refreshLabelsCache(serialized);
156      }
157      if (userAuths.size() > 0) {
158        byte[] serialized = VisibilityUtils.getUserAuthsDataToWriteToZooKeeper(userAuths);
159        this.labelsCache.writeToZookeeper(serialized, false);
160        this.labelsCache.refreshUserAuthsCache(serialized);
161      }
162    }
163  }
164
165  protected List<List<Cell>> getExistingLabelsWithAuths() throws IOException {
166    Scan scan = new Scan();
167    RegionScanner scanner = labelsRegion.getScanner(scan);
168    List<List<Cell>> existingLabels = new ArrayList<>();
169    try {
170      while (true) {
171        List<Cell> cells = new ArrayList<>();
172        scanner.next(cells);
173        if (cells.isEmpty()) {
174          break;
175        }
176        existingLabels.add(cells);
177      }
178    } finally {
179      scanner.close();
180    }
181    return existingLabels;
182  }
183
184  protected Pair<Map<String, Integer>, Map<String, List<Integer>>>
185    extractLabelsAndAuths(List<List<Cell>> labelDetails) {
186    Map<String, Integer> labels = new HashMap<>();
187    Map<String, List<Integer>> userAuths = new HashMap<>();
188    for (List<Cell> cells : labelDetails) {
189      for (Cell cell : cells) {
190        if (CellUtil.matchingQualifier(cell, LABEL_QUALIFIER)) {
191          labels.put(
192            Bytes.toString(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()),
193            PrivateCellUtil.getRowAsInt(cell));
194        } else {
195          // These are user cells who has authorization for this label
196          String user = Bytes.toString(cell.getQualifierArray(), cell.getQualifierOffset(),
197            cell.getQualifierLength());
198          List<Integer> auths = userAuths.get(user);
199          if (auths == null) {
200            auths = new ArrayList<>();
201            userAuths.put(user, auths);
202          }
203          auths.add(PrivateCellUtil.getRowAsInt(cell));
204        }
205      }
206    }
207    return new Pair<>(labels, userAuths);
208  }
209
210  protected void addSystemLabel(Region region, Map<String, Integer> labels,
211    Map<String, List<Integer>> userAuths) throws IOException {
212    if (!labels.containsKey(SYSTEM_LABEL)) {
213      byte[] row = Bytes.toBytes(SYSTEM_LABEL_ORDINAL);
214      Put p = new Put(row);
215      p.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(row)
216        .setFamily(LABELS_TABLE_FAMILY).setQualifier(LABEL_QUALIFIER).setTimestamp(p.getTimestamp())
217        .setType(Type.Put).setValue(Bytes.toBytes(SYSTEM_LABEL)).build());
218      region.put(p);
219      labels.put(SYSTEM_LABEL, SYSTEM_LABEL_ORDINAL);
220    }
221  }
222
223  @Override
224  public OperationStatus[] addLabels(List<byte[]> labels) throws IOException {
225    assert labelsRegion != null;
226    OperationStatus[] finalOpStatus = new OperationStatus[labels.size()];
227    List<Mutation> puts = new ArrayList<>(labels.size());
228    int i = 0;
229    ExtendedCellBuilder builder = ExtendedCellBuilderFactory.create(CellBuilderType.SHALLOW_COPY);
230    for (byte[] label : labels) {
231      String labelStr = Bytes.toString(label);
232      if (this.labelsCache.getLabelOrdinal(labelStr) > 0) {
233        finalOpStatus[i] = new OperationStatus(OperationStatusCode.FAILURE,
234          new LabelAlreadyExistsException("Label '" + labelStr + "' already exists"));
235      } else {
236        byte[] row = Bytes.toBytes(ordinalCounter.get());
237        Put p = new Put(row);
238        p.add(builder.clear().setRow(row).setFamily(LABELS_TABLE_FAMILY)
239          .setQualifier(LABEL_QUALIFIER).setTimestamp(p.getTimestamp()).setType(Type.Put)
240          .setValue(label).setTags(TagUtil.fromList(Arrays.asList(LABELS_TABLE_TAGS))).build());
241        if (LOG.isDebugEnabled()) {
242          LOG.debug("Adding the label " + labelStr);
243        }
244        puts.add(p);
245        ordinalCounter.incrementAndGet();
246      }
247      i++;
248    }
249    if (mutateLabelsRegion(puts, finalOpStatus)) {
250      updateZk(true);
251    }
252    return finalOpStatus;
253  }
254
255  @Override
256  public OperationStatus[] setAuths(byte[] user, List<byte[]> authLabels) throws IOException {
257    assert labelsRegion != null;
258    OperationStatus[] finalOpStatus = new OperationStatus[authLabels.size()];
259    List<Mutation> puts = new ArrayList<>(authLabels.size());
260    int i = 0;
261    ExtendedCellBuilder builder = ExtendedCellBuilderFactory.create(CellBuilderType.SHALLOW_COPY);
262    for (byte[] auth : authLabels) {
263      String authStr = Bytes.toString(auth);
264      int labelOrdinal = this.labelsCache.getLabelOrdinal(authStr);
265      if (labelOrdinal == 0) {
266        // This label is not yet added. 1st this should be added to the system
267        finalOpStatus[i] = new OperationStatus(OperationStatusCode.FAILURE,
268          new InvalidLabelException("Label '" + authStr + "' doesn't exists"));
269      } else {
270        byte[] row = Bytes.toBytes(labelOrdinal);
271        Put p = new Put(row);
272        p.add(builder.clear().setRow(row).setFamily(LABELS_TABLE_FAMILY).setQualifier(user)
273          .setTimestamp(p.getTimestamp()).setType(Cell.Type.Put).setValue(DUMMY_VALUE)
274          .setTags(TagUtil.fromList(Arrays.asList(LABELS_TABLE_TAGS))).build());
275        puts.add(p);
276      }
277      i++;
278    }
279    if (mutateLabelsRegion(puts, finalOpStatus)) {
280      updateZk(false);
281    }
282    return finalOpStatus;
283  }
284
285  @Override
286  public OperationStatus[] clearAuths(byte[] user, List<byte[]> authLabels) throws IOException {
287    assert labelsRegion != null;
288    OperationStatus[] finalOpStatus = new OperationStatus[authLabels.size()];
289    List<String> currentAuths;
290    if (AuthUtil.isGroupPrincipal(Bytes.toString(user))) {
291      String group = AuthUtil.getGroupName(Bytes.toString(user));
292      currentAuths = this.getGroupAuths(new String[] { group }, true);
293    } else {
294      currentAuths = this.getUserAuths(user, true);
295    }
296    List<Mutation> deletes = new ArrayList<>(authLabels.size());
297    int i = 0;
298    for (byte[] authLabel : authLabels) {
299      String authLabelStr = Bytes.toString(authLabel);
300      if (currentAuths.contains(authLabelStr)) {
301        int labelOrdinal = this.labelsCache.getLabelOrdinal(authLabelStr);
302        assert labelOrdinal > 0;
303        Delete d = new Delete(Bytes.toBytes(labelOrdinal));
304        d.addColumns(LABELS_TABLE_FAMILY, user);
305        deletes.add(d);
306      } else {
307        // This label is not set for the user.
308        finalOpStatus[i] =
309          new OperationStatus(OperationStatusCode.FAILURE, new InvalidLabelException(
310            "Label '" + authLabelStr + "' is not set for the user " + Bytes.toString(user)));
311      }
312      i++;
313    }
314    if (mutateLabelsRegion(deletes, finalOpStatus)) {
315      updateZk(false);
316    }
317    return finalOpStatus;
318  }
319
320  /**
321   * Adds the mutations to labels region and set the results to the finalOpStatus. finalOpStatus
322   * might have some entries in it where the OpStatus is FAILURE. We will leave those and set in
323   * others in the order.
324   * @return whether we need a ZK update or not.
325   */
326  private boolean mutateLabelsRegion(List<Mutation> mutations, OperationStatus[] finalOpStatus)
327    throws IOException {
328    OperationStatus[] opStatus =
329      this.labelsRegion.batchMutate(mutations.toArray(new Mutation[mutations.size()]));
330    int i = 0;
331    boolean updateZk = false;
332    for (OperationStatus status : opStatus) {
333      // Update the zk when atleast one of the mutation was added successfully.
334      updateZk = updateZk || (status.getOperationStatusCode() == OperationStatusCode.SUCCESS);
335      for (; i < finalOpStatus.length; i++) {
336        if (finalOpStatus[i] == null) {
337          finalOpStatus[i] = status;
338          break;
339        }
340      }
341    }
342    return updateZk;
343  }
344
345  @Override
346  public List<String> getUserAuths(byte[] user, boolean systemCall) throws IOException {
347    assert (labelsRegion != null || systemCall);
348    if (systemCall || labelsRegion == null) {
349      return this.labelsCache.getUserAuths(Bytes.toString(user));
350    }
351    Scan s = new Scan();
352    if (user != null && user.length > 0) {
353      s.addColumn(LABELS_TABLE_FAMILY, user);
354    }
355    Filter filter = VisibilityUtils.createVisibilityLabelFilter(this.labelsRegion,
356      new Authorizations(SYSTEM_LABEL));
357    s.setFilter(filter);
358    ArrayList<String> auths = new ArrayList<>();
359    RegionScanner scanner = this.labelsRegion.getScanner(s);
360    try {
361      List<Cell> results = new ArrayList<>(1);
362      while (true) {
363        scanner.next(results);
364        if (results.isEmpty()) break;
365        Cell cell = results.get(0);
366        int ordinal = PrivateCellUtil.getRowAsInt(cell);
367        String label = this.labelsCache.getLabel(ordinal);
368        if (label != null) {
369          auths.add(label);
370        }
371        results.clear();
372      }
373    } finally {
374      scanner.close();
375    }
376    return auths;
377  }
378
379  @Override
380  public List<String> getGroupAuths(String[] groups, boolean systemCall) throws IOException {
381    assert (labelsRegion != null || systemCall);
382    if (systemCall || labelsRegion == null) {
383      return this.labelsCache.getGroupAuths(groups);
384    }
385    Scan s = new Scan();
386    if (groups != null && groups.length > 0) {
387      for (String group : groups) {
388        s.addColumn(LABELS_TABLE_FAMILY, Bytes.toBytes(AuthUtil.toGroupEntry(group)));
389      }
390    }
391    Filter filter = VisibilityUtils.createVisibilityLabelFilter(this.labelsRegion,
392      new Authorizations(SYSTEM_LABEL));
393    s.setFilter(filter);
394    Set<String> auths = new HashSet<>();
395    RegionScanner scanner = this.labelsRegion.getScanner(s);
396    try {
397      List<Cell> results = new ArrayList<>(1);
398      while (true) {
399        scanner.next(results);
400        if (results.isEmpty()) break;
401        Cell cell = results.get(0);
402        int ordinal = PrivateCellUtil.getRowAsInt(cell);
403        String label = this.labelsCache.getLabel(ordinal);
404        if (label != null) {
405          auths.add(label);
406        }
407        results.clear();
408      }
409    } finally {
410      scanner.close();
411    }
412    return new ArrayList<>(auths);
413  }
414
415  @Override
416  public List<String> listLabels(String regex) throws IOException {
417    assert (labelsRegion != null);
418    Pair<Map<String, Integer>, Map<String, List<Integer>>> labelsAndUserAuths =
419      extractLabelsAndAuths(getExistingLabelsWithAuths());
420    Map<String, Integer> labels = labelsAndUserAuths.getFirst();
421    labels.remove(SYSTEM_LABEL);
422    if (regex != null) {
423      Pattern pattern = Pattern.compile(regex);
424      ArrayList<String> matchedLabels = new ArrayList<>();
425      for (String label : labels.keySet()) {
426        if (pattern.matcher(label).matches()) {
427          matchedLabels.add(label);
428        }
429      }
430      return matchedLabels;
431    }
432    return new ArrayList<>(labels.keySet());
433  }
434
435  @Override
436  public List<Tag> createVisibilityExpTags(String visExpression, boolean withSerializationFormat,
437    boolean checkAuths) throws IOException {
438    Set<Integer> auths = new HashSet<>();
439    if (checkAuths) {
440      User user = VisibilityUtils.getActiveUser();
441      auths.addAll(this.labelsCache.getUserAuthsAsOrdinals(user.getShortName()));
442      auths.addAll(this.labelsCache.getGroupAuthsAsOrdinals(user.getGroupNames()));
443    }
444    return VisibilityUtils.createVisibilityExpTags(visExpression, withSerializationFormat,
445      checkAuths, auths, labelsCache);
446  }
447
448  protected void updateZk(boolean labelAddition) throws IOException {
449    // We will add to zookeeper here.
450    // TODO we should add the delta only to zk. Else this will be a very heavy op and when there are
451    // so many labels and auth in the system, we will end up adding lots of data to zk. Most
452    // possibly we will exceed zk node data limit!
453    Pair<Map<String, Integer>, Map<String, List<Integer>>> labelsAndUserAuths =
454      extractLabelsAndAuths(getExistingLabelsWithAuths());
455    Map<String, Integer> existingLabels = labelsAndUserAuths.getFirst();
456    Map<String, List<Integer>> userAuths = labelsAndUserAuths.getSecond();
457    if (labelAddition) {
458      byte[] serialized = VisibilityUtils.getDataToWriteToZooKeeper(existingLabels);
459      this.labelsCache.writeToZookeeper(serialized, true);
460    } else {
461      byte[] serialized = VisibilityUtils.getUserAuthsDataToWriteToZooKeeper(userAuths);
462      this.labelsCache.writeToZookeeper(serialized, false);
463    }
464  }
465
466  @Override
467  public VisibilityExpEvaluator getVisibilityExpEvaluator(Authorizations authorizations)
468    throws IOException {
469    // If a super user issues a get/scan, he should be able to scan the cells
470    // irrespective of the Visibility labels
471    if (isReadFromSystemAuthUser()) {
472      return new VisibilityExpEvaluator() {
473        @Override
474        public boolean evaluate(Cell cell) throws IOException {
475          return true;
476        }
477      };
478    }
479    List<String> authLabels = null;
480    for (ScanLabelGenerator scanLabelGenerator : scanLabelGenerators) {
481      try {
482        // null authorizations to be handled inside SLG impl.
483        authLabels = scanLabelGenerator.getLabels(VisibilityUtils.getActiveUser(), authorizations);
484        authLabels = (authLabels == null) ? new ArrayList<>() : authLabels;
485        authorizations = new Authorizations(authLabels);
486      } catch (Throwable t) {
487        LOG.error(t.toString(), t);
488        throw new IOException(t);
489      }
490    }
491    int labelsCount = this.labelsCache.getLabelsCount();
492    final BitSet bs = new BitSet(labelsCount + 1); // ordinal is index 1 based
493    if (authLabels != null) {
494      for (String authLabel : authLabels) {
495        int labelOrdinal = this.labelsCache.getLabelOrdinal(authLabel);
496        if (labelOrdinal != 0) {
497          bs.set(labelOrdinal);
498        }
499      }
500    }
501
502    return new VisibilityExpEvaluator() {
503      @Override
504      public boolean evaluate(Cell cell) throws IOException {
505        boolean visibilityTagPresent = false;
506        Iterator<Tag> tagsItr = PrivateCellUtil.tagsIterator(cell);
507        while (tagsItr.hasNext()) {
508          boolean includeKV = true;
509          Tag tag = tagsItr.next();
510          if (tag.getType() == VISIBILITY_TAG_TYPE) {
511            visibilityTagPresent = true;
512            int offset = tag.getValueOffset();
513            int endOffset = offset + tag.getValueLength();
514            while (offset < endOffset) {
515              Pair<Integer, Integer> result = TagUtil.readVIntValuePart(tag, offset);
516              int currLabelOrdinal = result.getFirst();
517              if (currLabelOrdinal < 0) {
518                // check for the absence of this label in the Scan Auth labels
519                // ie. to check BitSet corresponding bit is 0
520                int temp = -currLabelOrdinal;
521                if (bs.get(temp)) {
522                  includeKV = false;
523                  break;
524                }
525              } else {
526                if (!bs.get(currLabelOrdinal)) {
527                  includeKV = false;
528                  break;
529                }
530              }
531              offset += result.getSecond();
532            }
533            if (includeKV) {
534              // We got one visibility expression getting evaluated to true. Good to include this
535              // KV in the result then.
536              return true;
537            }
538          }
539        }
540        return !(visibilityTagPresent);
541      }
542    };
543  }
544
545  protected boolean isReadFromSystemAuthUser() throws IOException {
546    User user = VisibilityUtils.getActiveUser();
547    return havingSystemAuth(user);
548  }
549
550  @Override
551  public boolean havingSystemAuth(User user) throws IOException {
552    // A super user has 'system' auth.
553    if (Superusers.isSuperUser(user)) {
554      return true;
555    }
556    // A user can also be explicitly granted 'system' auth.
557    List<String> auths = this.getUserAuths(Bytes.toBytes(user.getShortName()), true);
558    if (LOG.isTraceEnabled()) {
559      LOG.trace("The auths for user " + user.getShortName() + " are " + auths);
560    }
561    if (auths.contains(SYSTEM_LABEL)) {
562      return true;
563    }
564    auths = this.getGroupAuths(user.getGroupNames(), true);
565    if (LOG.isTraceEnabled()) {
566      LOG.trace("The auths for groups of user " + user.getShortName() + " are " + auths);
567    }
568    return auths.contains(SYSTEM_LABEL);
569  }
570
571  @Override
572  public boolean matchVisibility(List<Tag> putVisTags, Byte putTagsFormat, List<Tag> deleteVisTags,
573    Byte deleteTagsFormat) throws IOException {
574    // Early out if there are no tags in both of cell and delete
575    if (putVisTags.isEmpty() && deleteVisTags.isEmpty()) {
576      return true;
577    }
578    // Early out if one of the tags is empty
579    if (putVisTags.isEmpty() ^ deleteVisTags.isEmpty()) {
580      return false;
581    }
582    if (
583      (deleteTagsFormat != null && deleteTagsFormat == SORTED_ORDINAL_SERIALIZATION_FORMAT)
584        && (putTagsFormat == null || putTagsFormat == SORTED_ORDINAL_SERIALIZATION_FORMAT)
585    ) {
586      if (putTagsFormat == null) {
587        return matchUnSortedVisibilityTags(putVisTags, deleteVisTags);
588      } else {
589        return matchOrdinalSortedVisibilityTags(putVisTags, deleteVisTags);
590      }
591    }
592    throw new IOException("Unexpected tag format passed for comparison, deleteTagsFormat : "
593      + deleteTagsFormat + ", putTagsFormat : " + putTagsFormat);
594  }
595
596  /**
597   * @param putVisTags    Visibility tags in Put Mutation
598   * @param deleteVisTags Visibility tags in Delete Mutation
599   * @return true when all the visibility tags in Put matches with visibility tags in Delete. This
600   *         is used when, at least one set of tags are not sorted based on the label ordinal.
601   */
602  private static boolean matchUnSortedVisibilityTags(List<Tag> putVisTags, List<Tag> deleteVisTags)
603    throws IOException {
604    return compareTagsOrdinals(sortTagsBasedOnOrdinal(putVisTags),
605      sortTagsBasedOnOrdinal(deleteVisTags));
606  }
607
608  /**
609   * @param putVisTags    Visibility tags in Put Mutation
610   * @param deleteVisTags Visibility tags in Delete Mutation
611   * @return true when all the visibility tags in Put matches with visibility tags in Delete. This
612   *         is used when both the set of tags are sorted based on the label ordinal.
613   */
614  private static boolean matchOrdinalSortedVisibilityTags(List<Tag> putVisTags,
615    List<Tag> deleteVisTags) {
616    boolean matchFound = false;
617    // If the size does not match. Definitely we are not comparing the equal tags.
618    if ((deleteVisTags.size()) == putVisTags.size()) {
619      for (Tag tag : deleteVisTags) {
620        matchFound = false;
621        for (Tag givenTag : putVisTags) {
622          if (Tag.matchingValue(tag, givenTag)) {
623            matchFound = true;
624            break;
625          }
626        }
627        if (!matchFound) break;
628      }
629    }
630    return matchFound;
631  }
632
633  private static List<List<Integer>> sortTagsBasedOnOrdinal(List<Tag> tags) throws IOException {
634    List<List<Integer>> fullTagsList = new ArrayList<>();
635    for (Tag tag : tags) {
636      if (tag.getType() == VISIBILITY_TAG_TYPE) {
637        getSortedTagOrdinals(fullTagsList, tag);
638      }
639    }
640    return fullTagsList;
641  }
642
643  private static void getSortedTagOrdinals(List<List<Integer>> fullTagsList, Tag tag)
644    throws IOException {
645    List<Integer> tagsOrdinalInSortedOrder = new ArrayList<>();
646    int offset = tag.getValueOffset();
647    int endOffset = offset + tag.getValueLength();
648    while (offset < endOffset) {
649      Pair<Integer, Integer> result = TagUtil.readVIntValuePart(tag, offset);
650      tagsOrdinalInSortedOrder.add(result.getFirst());
651      offset += result.getSecond();
652    }
653    Collections.sort(tagsOrdinalInSortedOrder);
654    fullTagsList.add(tagsOrdinalInSortedOrder);
655  }
656
657  /*
658   * @return true when all the visibility tags in Put matches with visibility tags in Delete.
659   */
660  private static boolean compareTagsOrdinals(List<List<Integer>> putVisTags,
661    List<List<Integer>> deleteVisTags) {
662    boolean matchFound = false;
663    if (deleteVisTags.size() == putVisTags.size()) {
664      for (List<Integer> deleteTagOrdinals : deleteVisTags) {
665        matchFound = false;
666        for (List<Integer> tagOrdinals : putVisTags) {
667          if (deleteTagOrdinals.equals(tagOrdinals)) {
668            matchFound = true;
669            break;
670          }
671        }
672        if (!matchFound) break;
673      }
674    }
675    return matchFound;
676  }
677
678  @Override
679  public byte[] encodeVisibilityForReplication(final List<Tag> tags, final Byte serializationFormat)
680    throws IOException {
681    if (
682      tags.size() > 0 && (serializationFormat == null
683        || serializationFormat == SORTED_ORDINAL_SERIALIZATION_FORMAT)
684    ) {
685      return createModifiedVisExpression(tags);
686    }
687    return null;
688  }
689
690  /**
691   * - all the visibility tags associated with the current Cell
692   * @return - the modified visibility expression as byte[]
693   */
694  private byte[] createModifiedVisExpression(final List<Tag> tags) throws IOException {
695    StringBuilder visibilityString = new StringBuilder();
696    for (Tag tag : tags) {
697      if (tag.getType() == TagType.VISIBILITY_TAG_TYPE) {
698        if (visibilityString.length() != 0) {
699          visibilityString.append(VisibilityConstants.CLOSED_PARAN)
700            .append(VisibilityConstants.OR_OPERATOR);
701        }
702        int offset = tag.getValueOffset();
703        int endOffset = offset + tag.getValueLength();
704        boolean expressionStart = true;
705        while (offset < endOffset) {
706          Pair<Integer, Integer> result = TagUtil.readVIntValuePart(tag, offset);
707          int currLabelOrdinal = result.getFirst();
708          if (currLabelOrdinal < 0) {
709            int temp = -currLabelOrdinal;
710            String label = this.labelsCache.getLabel(temp);
711            if (expressionStart) {
712              // Quote every label in case of unicode characters if present
713              visibilityString.append(VisibilityConstants.OPEN_PARAN)
714                .append(VisibilityConstants.NOT_OPERATOR).append(CellVisibility.quote(label));
715            } else {
716              visibilityString.append(VisibilityConstants.AND_OPERATOR)
717                .append(VisibilityConstants.NOT_OPERATOR).append(CellVisibility.quote(label));
718            }
719          } else {
720            String label = this.labelsCache.getLabel(currLabelOrdinal);
721            if (expressionStart) {
722              visibilityString.append(VisibilityConstants.OPEN_PARAN)
723                .append(CellVisibility.quote(label));
724            } else {
725              visibilityString.append(VisibilityConstants.AND_OPERATOR)
726                .append(CellVisibility.quote(label));
727            }
728          }
729          expressionStart = false;
730          offset += result.getSecond();
731        }
732      }
733    }
734    if (visibilityString.length() != 0) {
735      visibilityString.append(VisibilityConstants.CLOSED_PARAN);
736      // Return the string formed as byte[]
737      return Bytes.toBytes(visibilityString.toString());
738    }
739    return null;
740  }
741}