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.Collections;
024import java.util.List;
025
026import org.apache.yetus.audience.InterfaceAudience;
027import org.slf4j.Logger;
028import org.slf4j.LoggerFactory;
029import org.apache.hadoop.hbase.regionserver.querymatcher.ScanDeleteTracker;
030import org.apache.hadoop.hbase.Cell;
031import org.apache.hadoop.hbase.CellComparator;
032import org.apache.hadoop.hbase.CellUtil;
033import org.apache.hadoop.hbase.KeyValue;
034import org.apache.hadoop.hbase.KeyValue.Type;
035import org.apache.hadoop.hbase.Tag;
036import org.apache.hadoop.hbase.util.Bytes;
037import org.apache.hadoop.hbase.util.Pair;
038import org.apache.hadoop.hbase.util.Triple;
039
040/**
041 * Similar to ScanDeletTracker but tracks the visibility expression also before
042 * deciding if a Cell can be considered deleted
043 */
044@InterfaceAudience.Private
045public class VisibilityScanDeleteTracker extends ScanDeleteTracker {
046
047  private static final Logger LOG = LoggerFactory.getLogger(VisibilityScanDeleteTracker.class);
048
049  /**
050   * This tag is used for the DELETE cell which has no visibility label.
051   */
052  private static final List<Tag> EMPTY_TAG = Collections.EMPTY_LIST;
053  // Its better to track the visibility tags in delete based on each type.  Create individual
054  // data structures for tracking each of them.  This would ensure that there is no tracking based
055  // on time and also would handle all cases where deletefamily or deletecolumns is specified with
056  // Latest_timestamp.  In such cases the ts in the delete marker and the masking
057  // put will not be same. So going with individual data structures for different delete
058  // type would solve this problem and also ensure that the combination of different type
059  // of deletes with diff ts would also work fine
060  // Track per TS
061  private List<Triple<List<Tag>, Byte, Long>> visibilityTagsDeleteFamily = new ArrayList<>();
062  // Delete family version with different ts and different visibility expression could come.
063  // Need to track it per ts.
064  private List<Triple<List<Tag>, Byte, Long>> visibilityTagsDeleteFamilyVersion = new ArrayList<>();
065  private List<Pair<List<Tag>, Byte>> visibilityTagsDeleteColumns;
066  // Tracking as List<List> is to handle same ts cell but different visibility tag.
067  // TODO : Need to handle puts with same ts but different vis tags.
068  private List<Pair<List<Tag>, Byte>> visiblityTagsDeleteColumnVersion = new ArrayList<>();
069
070  public VisibilityScanDeleteTracker(CellComparator comparator) {
071    super(comparator);
072  }
073
074  @Override
075  public void add(Cell delCell) {
076    //Cannot call super.add because need to find if the delete needs to be considered
077    long timestamp = delCell.getTimestamp();
078    byte type = delCell.getTypeByte();
079    if (type == KeyValue.Type.DeleteFamily.getCode()) {
080      hasFamilyStamp = true;
081      boolean hasVisTag = extractDeleteCellVisTags(delCell, KeyValue.Type.DeleteFamily);
082      if (!hasVisTag && timestamp > familyStamp) {
083        familyStamp = timestamp;
084      }
085      return;
086    } else if (type == KeyValue.Type.DeleteFamilyVersion.getCode()) {
087      familyVersionStamps.add(timestamp);
088      extractDeleteCellVisTags(delCell, KeyValue.Type.DeleteFamilyVersion);
089      return;
090    }
091    // new column, or more general delete type
092    if (deleteCell != null) {
093      if (!(CellUtil.matchingQualifier(delCell, deleteCell))) {
094        // A case where there are deletes for a column qualifier but there are
095        // no corresponding puts for them. Rare case.
096        visibilityTagsDeleteColumns = null;
097        visiblityTagsDeleteColumnVersion = null;
098      } else if (type == KeyValue.Type.Delete.getCode() && (deleteTimestamp != timestamp)) {
099        // there is a timestamp change which means we could clear the list
100        // when ts is same and the vis tags are different we need to collect
101        // them all. Interesting part is that in the normal case of puts if
102        // there are 2 cells with same ts and diff vis tags only one of them is
103        // returned. Handling with a single List<Tag> would mean that only one
104        // of the cell would be considered. Doing this as a precaution.
105        // Rare cases.
106        visiblityTagsDeleteColumnVersion = null;
107      }
108    }
109    deleteCell = delCell;
110    deleteType = type;
111    deleteTimestamp = timestamp;
112    extractDeleteCellVisTags(delCell, KeyValue.Type.codeToType(type));
113  }
114
115  private boolean extractDeleteCellVisTags(Cell delCell, Type type) {
116    // If tag is present in the delete
117    boolean hasVisTag = false;
118    Byte deleteCellVisTagsFormat = null;
119    switch (type) {
120      case DeleteFamily:
121        List<Tag> delTags = new ArrayList<>();
122        if (visibilityTagsDeleteFamily == null) {
123          visibilityTagsDeleteFamily = new ArrayList<>();
124        }
125        deleteCellVisTagsFormat = VisibilityUtils.extractVisibilityTags(delCell, delTags);
126        if (!delTags.isEmpty()) {
127          visibilityTagsDeleteFamily.add(new Triple<>(delTags, deleteCellVisTagsFormat, delCell.getTimestamp()));
128          hasVisTag = true;
129        } else {
130          visibilityTagsDeleteFamily.add(new Triple<>(EMPTY_TAG, deleteCellVisTagsFormat, delCell.getTimestamp()));
131        }
132        break;
133      case DeleteFamilyVersion:
134        if(visibilityTagsDeleteFamilyVersion == null) {
135          visibilityTagsDeleteFamilyVersion = new ArrayList<>();
136        }
137        delTags = new ArrayList<>();
138        deleteCellVisTagsFormat = VisibilityUtils.extractVisibilityTags(delCell, delTags);
139        if (!delTags.isEmpty()) {
140          visibilityTagsDeleteFamilyVersion.add(new Triple<>(delTags, deleteCellVisTagsFormat, delCell.getTimestamp()));
141          hasVisTag = true;
142        } else {
143          visibilityTagsDeleteFamilyVersion.add(new Triple<>(EMPTY_TAG, deleteCellVisTagsFormat, delCell.getTimestamp()));
144        }
145        break;
146      case DeleteColumn:
147        if (visibilityTagsDeleteColumns == null) {
148          visibilityTagsDeleteColumns = new ArrayList<>();
149        }
150        delTags = new ArrayList<>();
151        deleteCellVisTagsFormat = VisibilityUtils.extractVisibilityTags(delCell, delTags);
152        if (!delTags.isEmpty()) {
153          visibilityTagsDeleteColumns.add(new Pair<>(delTags, deleteCellVisTagsFormat));
154          hasVisTag = true;
155        } else {
156          visibilityTagsDeleteColumns.add(new Pair<>(EMPTY_TAG, deleteCellVisTagsFormat));
157        }
158        break;
159      case Delete:
160        if (visiblityTagsDeleteColumnVersion == null) {
161          visiblityTagsDeleteColumnVersion = new ArrayList<>();
162        }
163        delTags = new ArrayList<>();
164        deleteCellVisTagsFormat = VisibilityUtils.extractVisibilityTags(delCell, delTags);
165        if (!delTags.isEmpty()) {
166          visiblityTagsDeleteColumnVersion.add(new Pair<>(delTags, deleteCellVisTagsFormat));
167          hasVisTag = true;
168        } else {
169          visiblityTagsDeleteColumnVersion.add(new Pair<>(EMPTY_TAG, deleteCellVisTagsFormat));
170        }
171        break;
172      default:
173        throw new IllegalArgumentException("Invalid delete type");
174    }
175    return hasVisTag;
176  }
177
178  @Override
179  public DeleteResult isDeleted(Cell cell) {
180    long timestamp = cell.getTimestamp();
181    try {
182      if (hasFamilyStamp) {
183        if (visibilityTagsDeleteFamily != null) {
184          if (!visibilityTagsDeleteFamily.isEmpty()) {
185            for (int i = 0; i < visibilityTagsDeleteFamily.size(); i++) {
186              // visibilityTagsDeleteFamily is ArrayList
187              Triple<List<Tag>, Byte, Long> triple = visibilityTagsDeleteFamily.get(i);
188              if (timestamp <= triple.getThird()) {
189                List<Tag> putVisTags = new ArrayList<>();
190                Byte putCellVisTagsFormat = VisibilityUtils.extractVisibilityTags(cell, putVisTags);
191                boolean matchFound = VisibilityLabelServiceManager.getInstance()
192                    .getVisibilityLabelService().matchVisibility(putVisTags, putCellVisTagsFormat,
193                      triple.getFirst(), triple.getSecond());
194                if (matchFound) {
195                  // A return type of FAMILY_DELETED will cause skip for all remaining cells from
196                  // this
197                  // family. We would like to match visibility expression on every put cells after
198                  // this and only remove those matching with the family delete visibility. So we
199                  // are
200                  // returning FAMILY_VERSION_DELETED from here.
201                  return DeleteResult.FAMILY_VERSION_DELETED;
202                }
203              }
204            }
205          } else {
206            if (!VisibilityUtils.isVisibilityTagsPresent(cell) && timestamp <= familyStamp) {
207              // No tags
208              return DeleteResult.FAMILY_VERSION_DELETED;
209            }
210          }
211        } else {
212          if (!VisibilityUtils.isVisibilityTagsPresent(cell) && timestamp <= familyStamp) {
213            // No tags
214            return DeleteResult.FAMILY_VERSION_DELETED;
215          }
216        }
217      }
218      if (familyVersionStamps.contains(Long.valueOf(timestamp))) {
219        if (visibilityTagsDeleteFamilyVersion != null) {
220          if (!visibilityTagsDeleteFamilyVersion.isEmpty()) {
221            for (int i = 0; i < visibilityTagsDeleteFamilyVersion.size(); i++) {
222              // visibilityTagsDeleteFamilyVersion is ArrayList
223              Triple<List<Tag>, Byte, Long> triple = visibilityTagsDeleteFamilyVersion.get(i);
224              if (timestamp == triple.getThird()) {
225                List<Tag> putVisTags = new ArrayList<>();
226                Byte putCellVisTagsFormat = VisibilityUtils.extractVisibilityTags(cell, putVisTags);
227                boolean matchFound = VisibilityLabelServiceManager.getInstance()
228                    .getVisibilityLabelService().matchVisibility(putVisTags, putCellVisTagsFormat,
229                      triple.getFirst(), triple.getSecond());
230                if (matchFound) {
231                  return DeleteResult.FAMILY_VERSION_DELETED;
232                }
233              }
234            }
235          } else {
236            if (!VisibilityUtils.isVisibilityTagsPresent(cell)) {
237              // No tags
238              return DeleteResult.FAMILY_VERSION_DELETED;
239            }
240          }
241        } else {
242          if (!VisibilityUtils.isVisibilityTagsPresent(cell)) {
243            // No tags
244            return DeleteResult.FAMILY_VERSION_DELETED;
245          }
246        }
247      }
248      if (deleteCell != null) {
249        int ret = comparator.compareQualifiers(cell, deleteCell);
250        if (ret == 0) {
251          if (deleteType == KeyValue.Type.DeleteColumn.getCode()) {
252            if (visibilityTagsDeleteColumns != null) {
253              if (!visibilityTagsDeleteColumns.isEmpty()) {
254                for (Pair<List<Tag>, Byte> tags : visibilityTagsDeleteColumns) {
255                  List<Tag> putVisTags = new ArrayList<>();
256                  Byte putCellVisTagsFormat =
257                      VisibilityUtils.extractVisibilityTags(cell, putVisTags);
258                  boolean matchFound = VisibilityLabelServiceManager.getInstance()
259                      .getVisibilityLabelService().matchVisibility(putVisTags, putCellVisTagsFormat,
260                        tags.getFirst(), tags.getSecond());
261                  if (matchFound) {
262                    return DeleteResult.VERSION_DELETED;
263                  }
264                }
265              } else {
266                if (!VisibilityUtils.isVisibilityTagsPresent(cell)) {
267                  // No tags
268                  return DeleteResult.VERSION_DELETED;
269                }
270              }
271            } else {
272              if (!VisibilityUtils.isVisibilityTagsPresent(cell)) {
273                // No tags
274                return DeleteResult.VERSION_DELETED;
275              }
276            }
277          }
278          // Delete (aka DeleteVersion)
279          // If the timestamp is the same, keep this one
280          if (timestamp == deleteTimestamp) {
281            if (visiblityTagsDeleteColumnVersion != null) {
282              if (!visiblityTagsDeleteColumnVersion.isEmpty()) {
283                for (Pair<List<Tag>, Byte> tags : visiblityTagsDeleteColumnVersion) {
284                  List<Tag> putVisTags = new ArrayList<>();
285                  Byte putCellVisTagsFormat =
286                      VisibilityUtils.extractVisibilityTags(cell, putVisTags);
287                  boolean matchFound = VisibilityLabelServiceManager.getInstance()
288                      .getVisibilityLabelService().matchVisibility(putVisTags, putCellVisTagsFormat,
289                        tags.getFirst(), tags.getSecond());
290                  if (matchFound) {
291                    return DeleteResult.VERSION_DELETED;
292                  }
293                }
294              } else {
295                if (!VisibilityUtils.isVisibilityTagsPresent(cell)) {
296                  // No tags
297                  return DeleteResult.VERSION_DELETED;
298                }
299              }
300            } else {
301              if (!VisibilityUtils.isVisibilityTagsPresent(cell)) {
302                // No tags
303                return DeleteResult.VERSION_DELETED;
304              }
305            }
306          }
307        } else if (ret > 0) {
308          // Next column case.
309          deleteCell = null;
310          // Can nullify this because we are moving to the next column
311          visibilityTagsDeleteColumns = null;
312          visiblityTagsDeleteColumnVersion = null;
313        } else {
314          throw new IllegalStateException("isDeleted failed: deleteBuffer="
315              + Bytes.toStringBinary(deleteCell.getQualifierArray(),
316                    deleteCell.getQualifierOffset(), deleteCell.getQualifierLength())
317              + ", qualifier="
318              + Bytes.toStringBinary(cell.getQualifierArray(), cell.getQualifierOffset(),
319                  cell.getQualifierLength())
320              + ", timestamp=" + timestamp + ", comparison result: " + ret);
321        }
322      }
323    } catch (IOException e) {
324      LOG.error("Error in isDeleted() check! Will treat cell as not deleted", e);
325    }
326    return DeleteResult.NOT_DELETED;
327  }
328
329  @Override
330  public void reset() {
331    super.reset();
332    // clear only here
333    visibilityTagsDeleteColumns = null;
334    visibilityTagsDeleteFamily = null;
335    visibilityTagsDeleteFamilyVersion = null;
336    visiblityTagsDeleteColumnVersion = null;
337  }
338}