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