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