001/**
002 *
003 * Licensed to the Apache Software Foundation (ASF) under one
004 * or more contributor license agreements.  See the NOTICE file
005 * distributed with this work for additional information
006 * regarding copyright ownership.  The ASF licenses this file
007 * to you under the Apache License, Version 2.0 (the
008 * "License"); you may not use this file except in compliance
009 * with the License.  You may obtain a copy of the License at
010 *
011 *     http://www.apache.org/licenses/LICENSE-2.0
012 *
013 * Unless required by applicable law or agreed to in writing, software
014 * distributed under the License is distributed on an "AS IS" BASIS,
015 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
016 * See the License for the specific language governing permissions and
017 * limitations under the License.
018 */
019package org.apache.hadoop.hbase.security.visibility;
020
021import java.io.IOException;
022import java.util.ArrayList;
023import java.util.HashMap;
024import java.util.List;
025import java.util.Map;
026import java.util.NavigableMap;
027import java.util.NavigableSet;
028import java.util.SortedMap;
029import java.util.SortedSet;
030import java.util.TreeMap;
031import java.util.TreeSet;
032import org.apache.hadoop.hbase.Cell;
033import org.apache.hadoop.hbase.CellComparator;
034import org.apache.hadoop.hbase.KeyValue;
035import org.apache.hadoop.hbase.Tag;
036import org.apache.hadoop.hbase.regionserver.querymatcher.NewVersionBehaviorTracker;
037import org.apache.yetus.audience.InterfaceAudience;
038import org.slf4j.Logger;
039import org.slf4j.LoggerFactory;
040
041/**
042 * Similar to MvccSensitiveTracker but tracks the visibility expression also before
043 * deciding if a Cell can be considered deleted
044 */
045@InterfaceAudience.Private
046public class VisibilityNewVersionBehaivorTracker extends NewVersionBehaviorTracker {
047  private static final Logger LOG =
048      LoggerFactory.getLogger(VisibilityNewVersionBehaivorTracker.class);
049
050  public VisibilityNewVersionBehaivorTracker(NavigableSet<byte[]> columns,
051      CellComparator cellComparator, int minVersion, int maxVersion, int resultMaxVersions,
052      long oldestUnexpiredTS) {
053    super(columns, cellComparator, minVersion, maxVersion, resultMaxVersions, oldestUnexpiredTS);
054  }
055
056  private static class TagInfo {
057    List<Tag> tags;
058    Byte format;
059
060    private TagInfo(Cell c) {
061      tags = new ArrayList<>();
062      format = VisibilityUtils.extractVisibilityTags(c, tags);
063    }
064
065    private TagInfo() {
066      tags = new ArrayList<>();
067    }
068  }
069
070  private class VisibilityDeleteVersionsNode extends DeleteVersionsNode {
071    private TagInfo tagInfo;
072
073    // <timestamp, set<mvcc>>
074    // Key is ts of version deletes, value is its mvccs.
075    // We may delete more than one time for a version.
076    private Map<Long, SortedMap<Long, TagInfo>> deletesMap = new HashMap<>();
077
078    // <mvcc, set<mvcc>>
079    // Key is mvcc of version deletes, value is mvcc of visible puts before the delete effect.
080    private NavigableMap<Long, SortedSet<Long>> mvccCountingMap = new TreeMap<>();
081
082    protected VisibilityDeleteVersionsNode(long ts, long mvcc, TagInfo tagInfo) {
083      this.tagInfo = tagInfo;
084      this.ts = ts;
085      this.mvcc = mvcc;
086      mvccCountingMap.put(Long.MAX_VALUE, new TreeSet<Long>());
087    }
088
089    @Override
090    protected VisibilityDeleteVersionsNode getDeepCopy() {
091      VisibilityDeleteVersionsNode node = new VisibilityDeleteVersionsNode(ts, mvcc, tagInfo);
092      for (Map.Entry<Long, SortedMap<Long, TagInfo>> e : deletesMap.entrySet()) {
093        node.deletesMap.put(e.getKey(), new TreeMap<>(e.getValue()));
094      }
095      for (Map.Entry<Long, SortedSet<Long>> e : mvccCountingMap.entrySet()) {
096        node.mvccCountingMap.put(e.getKey(), new TreeSet<>(e.getValue()));
097      }
098      return node;
099    }
100
101    @Override
102    public void addVersionDelete(Cell cell) {
103      SortedMap<Long, TagInfo> set = deletesMap.get(cell.getTimestamp());
104      if (set == null) {
105        set = new TreeMap<>();
106        deletesMap.put(cell.getTimestamp(), set);
107      }
108      set.put(cell.getSequenceId(), new TagInfo(cell));
109      // The init set should be the puts whose mvcc is smaller than this Delete. Because
110      // there may be some Puts masked by them. The Puts whose mvcc is larger than this Delete can
111      // not be copied to this node because we may delete one version and the oldest put may not be
112      // masked.
113      SortedSet<Long> nextValue = mvccCountingMap.ceilingEntry(cell.getSequenceId()).getValue();
114      SortedSet<Long> thisValue = new TreeSet<>(nextValue.headSet(cell.getSequenceId()));
115      mvccCountingMap.put(cell.getSequenceId(), thisValue);
116    }
117
118  }
119
120  @Override
121  public void add(Cell cell) {
122    prepare(cell);
123    byte type = cell.getTypeByte();
124    switch (KeyValue.Type.codeToType(type)) {
125    // By the order of seen. We put null cq at first.
126    case DeleteFamily: // Delete all versions of all columns of the specified family
127      delFamMap.put(cell.getSequenceId(),
128          new VisibilityDeleteVersionsNode(cell.getTimestamp(), cell.getSequenceId(),
129              new TagInfo(cell)));
130      break;
131    case DeleteFamilyVersion: // Delete all columns of the specified family and specified version
132      delFamMap.ceilingEntry(cell.getSequenceId()).getValue().addVersionDelete(cell);
133      break;
134
135    // These two kinds of markers are mix with Puts.
136    case DeleteColumn: // Delete all versions of the specified column
137      delColMap.put(cell.getSequenceId(),
138          new VisibilityDeleteVersionsNode(cell.getTimestamp(), cell.getSequenceId(),
139              new TagInfo(cell)));
140      break;
141    case Delete: // Delete the specified version of the specified column.
142      delColMap.ceilingEntry(cell.getSequenceId()).getValue().addVersionDelete(cell);
143      break;
144    default:
145      throw new AssertionError("Unknown delete marker type for " + cell);
146    }
147  }
148
149  private boolean tagMatched(Cell put, TagInfo delInfo) throws IOException {
150    List<Tag> putVisTags = new ArrayList<>();
151    Byte putCellVisTagsFormat = VisibilityUtils.extractVisibilityTags(put, putVisTags);
152    return putVisTags.isEmpty() == delInfo.tags.isEmpty() && (
153        (putVisTags.isEmpty() && delInfo.tags.isEmpty()) || VisibilityLabelServiceManager
154            .getInstance().getVisibilityLabelService()
155            .matchVisibility(putVisTags, putCellVisTagsFormat, delInfo.tags, delInfo.format));
156  }
157
158  @Override
159  public DeleteResult isDeleted(Cell cell) {
160    try {
161      long duplicateMvcc = prepare(cell);
162
163      for (Map.Entry<Long, DeleteVersionsNode> e : delColMap.tailMap(cell.getSequenceId())
164          .entrySet()) {
165        VisibilityDeleteVersionsNode node = (VisibilityDeleteVersionsNode) e.getValue();
166        long deleteMvcc = Long.MAX_VALUE;
167        SortedMap<Long, TagInfo> deleteVersionMvccs = node.deletesMap.get(cell.getTimestamp());
168        if (deleteVersionMvccs != null) {
169          SortedMap<Long, TagInfo> tail = deleteVersionMvccs.tailMap(cell.getSequenceId());
170          for (Map.Entry<Long, TagInfo> entry : tail.entrySet()) {
171            if (tagMatched(cell, entry.getValue())) {
172              deleteMvcc = tail.firstKey();
173              break;
174            }
175          }
176        }
177        SortedMap<Long, SortedSet<Long>> subMap = node.mvccCountingMap
178            .subMap(cell.getSequenceId(), true, Math.min(duplicateMvcc, deleteMvcc), true);
179        for (Map.Entry<Long, SortedSet<Long>> seg : subMap.entrySet()) {
180          if (seg.getValue().size() >= maxVersions) {
181            return DeleteResult.VERSION_MASKED;
182          }
183          seg.getValue().add(cell.getSequenceId());
184        }
185        if (deleteMvcc < Long.MAX_VALUE) {
186          return DeleteResult.VERSION_DELETED;
187        }
188
189        if (cell.getTimestamp() <= node.ts && tagMatched(cell, node.tagInfo)) {
190          return DeleteResult.COLUMN_DELETED;
191        }
192      }
193      if (duplicateMvcc < Long.MAX_VALUE) {
194        return DeleteResult.VERSION_MASKED;
195      }
196    } catch (IOException e) {
197      LOG.error("Error in isDeleted() check! Will treat cell as not deleted", e);
198    }
199    return DeleteResult.NOT_DELETED;
200  }
201
202  @Override
203  protected void resetInternal() {
204    delFamMap.put(Long.MAX_VALUE,
205        new VisibilityDeleteVersionsNode(Long.MIN_VALUE, Long.MAX_VALUE, new TagInfo()));
206  }
207}