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. nn * @return whether we need a ZK update or not.
324   */
325  private boolean mutateLabelsRegion(List<Mutation> mutations, OperationStatus[] finalOpStatus)
326    throws IOException {
327    OperationStatus[] opStatus =
328      this.labelsRegion.batchMutate(mutations.toArray(new Mutation[mutations.size()]));
329    int i = 0;
330    boolean updateZk = false;
331    for (OperationStatus status : opStatus) {
332      // Update the zk when atleast one of the mutation was added successfully.
333      updateZk = updateZk || (status.getOperationStatusCode() == OperationStatusCode.SUCCESS);
334      for (; i < finalOpStatus.length; i++) {
335        if (finalOpStatus[i] == null) {
336          finalOpStatus[i] = status;
337          break;
338        }
339      }
340    }
341    return updateZk;
342  }
343
344  @Override
345  public List<String> getUserAuths(byte[] user, boolean systemCall) throws IOException {
346    assert (labelsRegion != null || systemCall);
347    if (systemCall || labelsRegion == null) {
348      return this.labelsCache.getUserAuths(Bytes.toString(user));
349    }
350    Scan s = new Scan();
351    if (user != null && user.length > 0) {
352      s.addColumn(LABELS_TABLE_FAMILY, user);
353    }
354    Filter filter = VisibilityUtils.createVisibilityLabelFilter(this.labelsRegion,
355      new Authorizations(SYSTEM_LABEL));
356    s.setFilter(filter);
357    ArrayList<String> auths = new ArrayList<>();
358    RegionScanner scanner = this.labelsRegion.getScanner(s);
359    try {
360      List<Cell> results = new ArrayList<>(1);
361      while (true) {
362        scanner.next(results);
363        if (results.isEmpty()) break;
364        Cell cell = results.get(0);
365        int ordinal = PrivateCellUtil.getRowAsInt(cell);
366        String label = this.labelsCache.getLabel(ordinal);
367        if (label != null) {
368          auths.add(label);
369        }
370        results.clear();
371      }
372    } finally {
373      scanner.close();
374    }
375    return auths;
376  }
377
378  @Override
379  public List<String> getGroupAuths(String[] groups, boolean systemCall) throws IOException {
380    assert (labelsRegion != null || systemCall);
381    if (systemCall || labelsRegion == null) {
382      return this.labelsCache.getGroupAuths(groups);
383    }
384    Scan s = new Scan();
385    if (groups != null && groups.length > 0) {
386      for (String group : groups) {
387        s.addColumn(LABELS_TABLE_FAMILY, Bytes.toBytes(AuthUtil.toGroupEntry(group)));
388      }
389    }
390    Filter filter = VisibilityUtils.createVisibilityLabelFilter(this.labelsRegion,
391      new Authorizations(SYSTEM_LABEL));
392    s.setFilter(filter);
393    Set<String> auths = new HashSet<>();
394    RegionScanner scanner = this.labelsRegion.getScanner(s);
395    try {
396      List<Cell> results = new ArrayList<>(1);
397      while (true) {
398        scanner.next(results);
399        if (results.isEmpty()) break;
400        Cell cell = results.get(0);
401        int ordinal = PrivateCellUtil.getRowAsInt(cell);
402        String label = this.labelsCache.getLabel(ordinal);
403        if (label != null) {
404          auths.add(label);
405        }
406        results.clear();
407      }
408    } finally {
409      scanner.close();
410    }
411    return new ArrayList<>(auths);
412  }
413
414  @Override
415  public List<String> listLabels(String regex) throws IOException {
416    assert (labelsRegion != null);
417    Pair<Map<String, Integer>, Map<String, List<Integer>>> labelsAndUserAuths =
418      extractLabelsAndAuths(getExistingLabelsWithAuths());
419    Map<String, Integer> labels = labelsAndUserAuths.getFirst();
420    labels.remove(SYSTEM_LABEL);
421    if (regex != null) {
422      Pattern pattern = Pattern.compile(regex);
423      ArrayList<String> matchedLabels = new ArrayList<>();
424      for (String label : labels.keySet()) {
425        if (pattern.matcher(label).matches()) {
426          matchedLabels.add(label);
427        }
428      }
429      return matchedLabels;
430    }
431    return new ArrayList<>(labels.keySet());
432  }
433
434  @Override
435  public List<Tag> createVisibilityExpTags(String visExpression, boolean withSerializationFormat,
436    boolean checkAuths) throws IOException {
437    Set<Integer> auths = new HashSet<>();
438    if (checkAuths) {
439      User user = VisibilityUtils.getActiveUser();
440      auths.addAll(this.labelsCache.getUserAuthsAsOrdinals(user.getShortName()));
441      auths.addAll(this.labelsCache.getGroupAuthsAsOrdinals(user.getGroupNames()));
442    }
443    return VisibilityUtils.createVisibilityExpTags(visExpression, withSerializationFormat,
444      checkAuths, auths, labelsCache);
445  }
446
447  protected void updateZk(boolean labelAddition) throws IOException {
448    // We will add to zookeeper here.
449    // TODO we should add the delta only to zk. Else this will be a very heavy op and when there are
450    // so many labels and auth in the system, we will end up adding lots of data to zk. Most
451    // possibly we will exceed zk node data limit!
452    Pair<Map<String, Integer>, Map<String, List<Integer>>> labelsAndUserAuths =
453      extractLabelsAndAuths(getExistingLabelsWithAuths());
454    Map<String, Integer> existingLabels = labelsAndUserAuths.getFirst();
455    Map<String, List<Integer>> userAuths = labelsAndUserAuths.getSecond();
456    if (labelAddition) {
457      byte[] serialized = VisibilityUtils.getDataToWriteToZooKeeper(existingLabels);
458      this.labelsCache.writeToZookeeper(serialized, true);
459    } else {
460      byte[] serialized = VisibilityUtils.getUserAuthsDataToWriteToZooKeeper(userAuths);
461      this.labelsCache.writeToZookeeper(serialized, false);
462    }
463  }
464
465  @Override
466  public VisibilityExpEvaluator getVisibilityExpEvaluator(Authorizations authorizations)
467    throws IOException {
468    // If a super user issues a get/scan, he should be able to scan the cells
469    // irrespective of the Visibility labels
470    if (isReadFromSystemAuthUser()) {
471      return new VisibilityExpEvaluator() {
472        @Override
473        public boolean evaluate(Cell cell) throws IOException {
474          return true;
475        }
476      };
477    }
478    List<String> authLabels = null;
479    for (ScanLabelGenerator scanLabelGenerator : scanLabelGenerators) {
480      try {
481        // null authorizations to be handled inside SLG impl.
482        authLabels = scanLabelGenerator.getLabels(VisibilityUtils.getActiveUser(), authorizations);
483        authLabels = (authLabels == null) ? new ArrayList<>() : authLabels;
484        authorizations = new Authorizations(authLabels);
485      } catch (Throwable t) {
486        LOG.error(t.toString(), t);
487        throw new IOException(t);
488      }
489    }
490    int labelsCount = this.labelsCache.getLabelsCount();
491    final BitSet bs = new BitSet(labelsCount + 1); // ordinal is index 1 based
492    if (authLabels != null) {
493      for (String authLabel : authLabels) {
494        int labelOrdinal = this.labelsCache.getLabelOrdinal(authLabel);
495        if (labelOrdinal != 0) {
496          bs.set(labelOrdinal);
497        }
498      }
499    }
500
501    return new VisibilityExpEvaluator() {
502      @Override
503      public boolean evaluate(Cell cell) throws IOException {
504        boolean visibilityTagPresent = false;
505        Iterator<Tag> tagsItr = PrivateCellUtil.tagsIterator(cell);
506        while (tagsItr.hasNext()) {
507          boolean includeKV = true;
508          Tag tag = tagsItr.next();
509          if (tag.getType() == VISIBILITY_TAG_TYPE) {
510            visibilityTagPresent = true;
511            int offset = tag.getValueOffset();
512            int endOffset = offset + tag.getValueLength();
513            while (offset < endOffset) {
514              Pair<Integer, Integer> result = TagUtil.readVIntValuePart(tag, offset);
515              int currLabelOrdinal = result.getFirst();
516              if (currLabelOrdinal < 0) {
517                // check for the absence of this label in the Scan Auth labels
518                // ie. to check BitSet corresponding bit is 0
519                int temp = -currLabelOrdinal;
520                if (bs.get(temp)) {
521                  includeKV = false;
522                  break;
523                }
524              } else {
525                if (!bs.get(currLabelOrdinal)) {
526                  includeKV = false;
527                  break;
528                }
529              }
530              offset += result.getSecond();
531            }
532            if (includeKV) {
533              // We got one visibility expression getting evaluated to true. Good to include this
534              // KV in the result then.
535              return true;
536            }
537          }
538        }
539        return !(visibilityTagPresent);
540      }
541    };
542  }
543
544  protected boolean isReadFromSystemAuthUser() throws IOException {
545    User user = VisibilityUtils.getActiveUser();
546    return havingSystemAuth(user);
547  }
548
549  @Override
550  public boolean havingSystemAuth(User user) throws IOException {
551    // A super user has 'system' auth.
552    if (Superusers.isSuperUser(user)) {
553      return true;
554    }
555    // A user can also be explicitly granted 'system' auth.
556    List<String> auths = this.getUserAuths(Bytes.toBytes(user.getShortName()), true);
557    if (LOG.isTraceEnabled()) {
558      LOG.trace("The auths for user " + user.getShortName() + " are " + auths);
559    }
560    if (auths.contains(SYSTEM_LABEL)) {
561      return true;
562    }
563    auths = this.getGroupAuths(user.getGroupNames(), true);
564    if (LOG.isTraceEnabled()) {
565      LOG.trace("The auths for groups of user " + user.getShortName() + " are " + auths);
566    }
567    return auths.contains(SYSTEM_LABEL);
568  }
569
570  @Override
571  public boolean matchVisibility(List<Tag> putVisTags, Byte putTagsFormat, List<Tag> deleteVisTags,
572    Byte deleteTagsFormat) throws IOException {
573    // Early out if there are no tags in both of cell and delete
574    if (putVisTags.isEmpty() && deleteVisTags.isEmpty()) {
575      return true;
576    }
577    // Early out if one of the tags is empty
578    if (putVisTags.isEmpty() ^ deleteVisTags.isEmpty()) {
579      return false;
580    }
581    if (
582      (deleteTagsFormat != null && deleteTagsFormat == SORTED_ORDINAL_SERIALIZATION_FORMAT)
583        && (putTagsFormat == null || putTagsFormat == SORTED_ORDINAL_SERIALIZATION_FORMAT)
584    ) {
585      if (putTagsFormat == null) {
586        return matchUnSortedVisibilityTags(putVisTags, deleteVisTags);
587      } else {
588        return matchOrdinalSortedVisibilityTags(putVisTags, deleteVisTags);
589      }
590    }
591    throw new IOException("Unexpected tag format passed for comparison, deleteTagsFormat : "
592      + deleteTagsFormat + ", putTagsFormat : " + putTagsFormat);
593  }
594
595  /**
596   * @param putVisTags    Visibility tags in Put Mutation
597   * @param deleteVisTags Visibility tags in Delete Mutation
598   * @return true when all the visibility tags in Put matches with visibility tags in Delete. This
599   *         is used when, at least one set of tags are not sorted based on the label ordinal.
600   */
601  private static boolean matchUnSortedVisibilityTags(List<Tag> putVisTags, List<Tag> deleteVisTags)
602    throws IOException {
603    return compareTagsOrdinals(sortTagsBasedOnOrdinal(putVisTags),
604      sortTagsBasedOnOrdinal(deleteVisTags));
605  }
606
607  /**
608   * @param putVisTags    Visibility tags in Put Mutation
609   * @param deleteVisTags Visibility tags in Delete Mutation
610   * @return true when all the visibility tags in Put matches with visibility tags in Delete. This
611   *         is used when both the set of tags are sorted based on the label ordinal.
612   */
613  private static boolean matchOrdinalSortedVisibilityTags(List<Tag> putVisTags,
614    List<Tag> deleteVisTags) {
615    boolean matchFound = false;
616    // If the size does not match. Definitely we are not comparing the equal tags.
617    if ((deleteVisTags.size()) == putVisTags.size()) {
618      for (Tag tag : deleteVisTags) {
619        matchFound = false;
620        for (Tag givenTag : putVisTags) {
621          if (Tag.matchingValue(tag, givenTag)) {
622            matchFound = true;
623            break;
624          }
625        }
626        if (!matchFound) break;
627      }
628    }
629    return matchFound;
630  }
631
632  private static List<List<Integer>> sortTagsBasedOnOrdinal(List<Tag> tags) throws IOException {
633    List<List<Integer>> fullTagsList = new ArrayList<>();
634    for (Tag tag : tags) {
635      if (tag.getType() == VISIBILITY_TAG_TYPE) {
636        getSortedTagOrdinals(fullTagsList, tag);
637      }
638    }
639    return fullTagsList;
640  }
641
642  private static void getSortedTagOrdinals(List<List<Integer>> fullTagsList, Tag tag)
643    throws IOException {
644    List<Integer> tagsOrdinalInSortedOrder = new ArrayList<>();
645    int offset = tag.getValueOffset();
646    int endOffset = offset + tag.getValueLength();
647    while (offset < endOffset) {
648      Pair<Integer, Integer> result = TagUtil.readVIntValuePart(tag, offset);
649      tagsOrdinalInSortedOrder.add(result.getFirst());
650      offset += result.getSecond();
651    }
652    Collections.sort(tagsOrdinalInSortedOrder);
653    fullTagsList.add(tagsOrdinalInSortedOrder);
654  }
655
656  /*
657   * @return true when all the visibility tags in Put matches with visibility tags in Delete.
658   */
659  private static boolean compareTagsOrdinals(List<List<Integer>> putVisTags,
660    List<List<Integer>> deleteVisTags) {
661    boolean matchFound = false;
662    if (deleteVisTags.size() == putVisTags.size()) {
663      for (List<Integer> deleteTagOrdinals : deleteVisTags) {
664        matchFound = false;
665        for (List<Integer> tagOrdinals : putVisTags) {
666          if (deleteTagOrdinals.equals(tagOrdinals)) {
667            matchFound = true;
668            break;
669          }
670        }
671        if (!matchFound) break;
672      }
673    }
674    return matchFound;
675  }
676
677  @Override
678  public byte[] encodeVisibilityForReplication(final List<Tag> tags, final Byte serializationFormat)
679    throws IOException {
680    if (
681      tags.size() > 0 && (serializationFormat == null
682        || serializationFormat == SORTED_ORDINAL_SERIALIZATION_FORMAT)
683    ) {
684      return createModifiedVisExpression(tags);
685    }
686    return null;
687  }
688
689  /**
690   * n * - all the visibility tags associated with the current Cell
691   * @return - the modified visibility expression as byte[]
692   */
693  private byte[] createModifiedVisExpression(final List<Tag> tags) throws IOException {
694    StringBuilder visibilityString = new StringBuilder();
695    for (Tag tag : tags) {
696      if (tag.getType() == TagType.VISIBILITY_TAG_TYPE) {
697        if (visibilityString.length() != 0) {
698          visibilityString.append(VisibilityConstants.CLOSED_PARAN)
699            .append(VisibilityConstants.OR_OPERATOR);
700        }
701        int offset = tag.getValueOffset();
702        int endOffset = offset + tag.getValueLength();
703        boolean expressionStart = true;
704        while (offset < endOffset) {
705          Pair<Integer, Integer> result = TagUtil.readVIntValuePart(tag, offset);
706          int currLabelOrdinal = result.getFirst();
707          if (currLabelOrdinal < 0) {
708            int temp = -currLabelOrdinal;
709            String label = this.labelsCache.getLabel(temp);
710            if (expressionStart) {
711              // Quote every label in case of unicode characters if present
712              visibilityString.append(VisibilityConstants.OPEN_PARAN)
713                .append(VisibilityConstants.NOT_OPERATOR).append(CellVisibility.quote(label));
714            } else {
715              visibilityString.append(VisibilityConstants.AND_OPERATOR)
716                .append(VisibilityConstants.NOT_OPERATOR).append(CellVisibility.quote(label));
717            }
718          } else {
719            String label = this.labelsCache.getLabel(currLabelOrdinal);
720            if (expressionStart) {
721              visibilityString.append(VisibilityConstants.OPEN_PARAN)
722                .append(CellVisibility.quote(label));
723            } else {
724              visibilityString.append(VisibilityConstants.AND_OPERATOR)
725                .append(CellVisibility.quote(label));
726            }
727          }
728          expressionStart = false;
729          offset += result.getSecond();
730        }
731      }
732    }
733    if (visibilityString.length() != 0) {
734      visibilityString.append(VisibilityConstants.CLOSED_PARAN);
735      // Return the string formed as byte[]
736      return Bytes.toBytes(visibilityString.toString());
737    }
738    return null;
739  }
740}