View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.security.visibility;
20  
21  import java.io.IOException;
22  import java.util.ArrayList;
23  import java.util.List;
24  
25  import org.apache.commons.logging.Log;
26  import org.apache.commons.logging.LogFactory;
27  import org.apache.hadoop.hbase.classification.InterfaceAudience;
28  import org.apache.hadoop.hbase.Cell;
29  import org.apache.hadoop.hbase.KeyValue;
30  import org.apache.hadoop.hbase.KeyValue.Type;
31  import org.apache.hadoop.hbase.Tag;
32  import org.apache.hadoop.hbase.regionserver.ScanDeleteTracker;
33  import org.apache.hadoop.hbase.util.Bytes;
34  import org.apache.hadoop.hbase.util.Pair;
35  import org.apache.hadoop.hbase.util.Triple;
36  
37  /**
38   * Similar to ScanDeletTracker but tracks the visibility expression also before
39   * deciding if a Cell can be considered deleted
40   */
41  @InterfaceAudience.Private
42  public class VisibilityScanDeleteTracker extends ScanDeleteTracker {
43  
44    private static final Log LOG = LogFactory.getLog(VisibilityScanDeleteTracker.class);
45  
46    // Its better to track the visibility tags in delete based on each type.  Create individual
47    // data structures for tracking each of them.  This would ensure that there is no tracking based
48    // on time and also would handle all cases where deletefamily or deletecolumns is specified with
49    // Latest_timestamp.  In such cases the ts in the delete marker and the masking
50    // put will not be same. So going with individual data structures for different delete
51    // type would solve this problem and also ensure that the combination of different type
52    // of deletes with diff ts would also work fine
53    // Track per TS
54    private List<Triple<List<Tag>, Byte, Long>> visibilityTagsDeleteFamily =
55        new ArrayList<Triple<List<Tag>, Byte, Long>>();
56    // Delete family version with different ts and different visibility expression could come.
57    // Need to track it per ts.
58    private List<Triple<List<Tag>, Byte, Long>> visibilityTagsDeleteFamilyVersion =
59        new ArrayList<Triple<List<Tag>, Byte, Long>>();
60    private List<Pair<List<Tag>, Byte>> visibilityTagsDeleteColumns;
61    // Tracking as List<List> is to handle same ts cell but different visibility tag. 
62    // TODO : Need to handle puts with same ts but different vis tags.
63    private List<Pair<List<Tag>, Byte>> visiblityTagsDeleteColumnVersion =
64        new ArrayList<Pair<List<Tag>, Byte>>();
65  
66    public VisibilityScanDeleteTracker() {
67      super();
68    }
69  
70    @Override
71    public void add(Cell delCell) {
72      //Cannot call super.add because need to find if the delete needs to be considered
73      long timestamp = delCell.getTimestamp();
74      int qualifierOffset = delCell.getQualifierOffset();
75      int qualifierLength = delCell.getQualifierLength();
76      byte type = delCell.getTypeByte();
77      if (type == KeyValue.Type.DeleteFamily.getCode()) {
78        hasFamilyStamp = true;
79        boolean hasVisTag = extractDeleteCellVisTags(delCell, KeyValue.Type.DeleteFamily);
80        if (!hasVisTag && timestamp > familyStamp) {
81          familyStamp = timestamp;
82        }
83        return;
84      } else if (type == KeyValue.Type.DeleteFamilyVersion.getCode()) {
85        familyVersionStamps.add(timestamp);
86        extractDeleteCellVisTags(delCell, KeyValue.Type.DeleteFamilyVersion);
87        return;
88      }
89      // new column, or more general delete type
90      if (deleteBuffer != null) {
91        if (Bytes.compareTo(deleteBuffer, deleteOffset, deleteLength, delCell.getQualifierArray(),
92            qualifierOffset, qualifierLength) != 0) {
93          // A case where there are deletes for a column qualifier but there are
94          // no corresponding puts for them. Rare case.
95          visibilityTagsDeleteColumns = null;
96          visiblityTagsDeleteColumnVersion = null;
97        } else if (type == KeyValue.Type.Delete.getCode() && (deleteTimestamp != timestamp)) {
98          // there is a timestamp change which means we could clear the list
99          // when ts is same and the vis tags are different we need to collect
100         // them all. Interesting part is that in the normal case of puts if
101         // there are 2 cells with same ts and diff vis tags only one of them is
102         // returned. Handling with a single List<Tag> would mean that only one
103         // of the cell would be considered. Doing this as a precaution.
104         // Rare cases.
105         visiblityTagsDeleteColumnVersion = null;
106       }
107     }
108     deleteBuffer = delCell.getQualifierArray();
109     deleteOffset = qualifierOffset;
110     deleteLength = qualifierLength;
111     deleteType = type;
112     deleteTimestamp = timestamp;
113     extractDeleteCellVisTags(delCell, KeyValue.Type.codeToType(type));
114   }
115 
116   private boolean extractDeleteCellVisTags(Cell delCell, Type type) {
117     // If tag is present in the delete
118     boolean hasVisTag = false;
119     if (delCell.getTagsLength() > 0) {
120       Byte deleteCellVisTagsFormat = null;
121       switch (type) {
122       case DeleteFamily:
123         List<Tag> delTags = new ArrayList<Tag>();
124         if (visibilityTagsDeleteFamily == null) {
125           visibilityTagsDeleteFamily = new ArrayList<Triple<List<Tag>, Byte, Long>>();
126         }
127         deleteCellVisTagsFormat = VisibilityUtils.extractVisibilityTags(delCell, delTags);
128         if (!delTags.isEmpty()) {
129           visibilityTagsDeleteFamily.add(new Triple<List<Tag>, Byte, Long>(delTags,
130               deleteCellVisTagsFormat, delCell.getTimestamp()));
131           hasVisTag = true;
132         }
133         break;
134       case DeleteFamilyVersion:
135         if(visibilityTagsDeleteFamilyVersion == null) {
136           visibilityTagsDeleteFamilyVersion = new ArrayList<Triple<List<Tag>, Byte, Long>>();
137         }
138         delTags = new ArrayList<Tag>();
139         deleteCellVisTagsFormat = VisibilityUtils.extractVisibilityTags(delCell, delTags);
140         if (!delTags.isEmpty()) {
141           visibilityTagsDeleteFamilyVersion.add(new Triple<List<Tag>, Byte, Long>(delTags,
142               deleteCellVisTagsFormat, delCell.getTimestamp()));
143           hasVisTag = true;
144         }
145         break;
146       case DeleteColumn:
147         if (visibilityTagsDeleteColumns == null) {
148           visibilityTagsDeleteColumns = new ArrayList<Pair<List<Tag>, Byte>>();
149         }
150         delTags = new ArrayList<Tag>();
151         deleteCellVisTagsFormat = VisibilityUtils.extractVisibilityTags(delCell, delTags);
152         if (!delTags.isEmpty()) {
153           visibilityTagsDeleteColumns.add(new Pair<List<Tag>, Byte>(delTags,
154               deleteCellVisTagsFormat));
155           hasVisTag = true;
156         }
157         break;
158       case Delete:
159         if (visiblityTagsDeleteColumnVersion == null) {
160           visiblityTagsDeleteColumnVersion = new ArrayList<Pair<List<Tag>, Byte>>();
161         }
162         delTags = new ArrayList<Tag>();
163         deleteCellVisTagsFormat = VisibilityUtils.extractVisibilityTags(delCell, delTags);
164         if (!delTags.isEmpty()) {
165           visiblityTagsDeleteColumnVersion.add(new Pair<List<Tag>, Byte>(delTags,
166               deleteCellVisTagsFormat));
167           hasVisTag = true;
168         }
169         break;
170       default:
171         throw new IllegalArgumentException("Invalid delete type");
172       }
173     }
174     return hasVisTag;
175   }
176 
177   @Override
178   public DeleteResult isDeleted(Cell cell) {
179     long timestamp = cell.getTimestamp();
180     int qualifierOffset = cell.getQualifierOffset();
181     int qualifierLength = cell.getQualifierLength();
182     try {
183       if (hasFamilyStamp) {
184         if (visibilityTagsDeleteFamily != null) {
185           if (!visibilityTagsDeleteFamily.isEmpty()) {
186             for (int i = 0; i < visibilityTagsDeleteFamily.size(); i++) {
187               // visibilityTagsDeleteFamily is ArrayList
188               Triple<List<Tag>, Byte, Long> triple = visibilityTagsDeleteFamily.get(i);
189               if (timestamp <= triple.getThird()) {
190                 List<Tag> putVisTags = new ArrayList<Tag>();
191                 Byte putCellVisTagsFormat = VisibilityUtils.extractVisibilityTags(cell, putVisTags);
192                 boolean matchFound = VisibilityLabelServiceManager.getInstance()
193                     .getVisibilityLabelService().matchVisibility(putVisTags, putCellVisTagsFormat,
194                       triple.getFirst(), triple.getSecond());
195                 if (matchFound) {
196                   // A return type of FAMILY_DELETED will cause skip for all remaining cells from
197                   // this
198                   // family. We would like to match visibility expression on every put cells after
199                   // this and only remove those matching with the family delete visibility. So we
200                   // are
201                   // returning FAMILY_VERSION_DELETED from here.
202                   return DeleteResult.FAMILY_VERSION_DELETED;
203                 }
204               }
205             }
206           } else {
207             if (!VisibilityUtils.isVisibilityTagsPresent(cell) && timestamp <= familyStamp) {
208               // No tags
209               return DeleteResult.FAMILY_VERSION_DELETED;
210             }
211           }
212         } else {
213           if (!VisibilityUtils.isVisibilityTagsPresent(cell) && timestamp <= familyStamp) {
214             // No tags
215             return DeleteResult.FAMILY_VERSION_DELETED;
216           }
217         }
218       }
219       if (familyVersionStamps.contains(Long.valueOf(timestamp))) {
220         if (visibilityTagsDeleteFamilyVersion != null) {
221           if (!visibilityTagsDeleteFamilyVersion.isEmpty()) {
222             for (int i = 0; i < visibilityTagsDeleteFamilyVersion.size(); i++) {
223               // visibilityTagsDeleteFamilyVersion is ArrayList
224               Triple<List<Tag>, Byte, Long> triple = visibilityTagsDeleteFamilyVersion.get(i);
225               if (timestamp == triple.getThird()) {
226                 List<Tag> putVisTags = new ArrayList<Tag>();
227                 Byte putCellVisTagsFormat = VisibilityUtils.extractVisibilityTags(cell, putVisTags);
228                 boolean matchFound = VisibilityLabelServiceManager.getInstance()
229                     .getVisibilityLabelService().matchVisibility(putVisTags, putCellVisTagsFormat,
230                       triple.getFirst(), triple.getSecond());
231                 if (matchFound) {
232                   return DeleteResult.FAMILY_VERSION_DELETED;
233                 }
234               }
235             }
236           } else {
237             if (!VisibilityUtils.isVisibilityTagsPresent(cell)) {
238               // No tags
239               return DeleteResult.FAMILY_VERSION_DELETED;
240             }
241           }
242         } else {
243           if (!VisibilityUtils.isVisibilityTagsPresent(cell)) {
244             // No tags
245             return DeleteResult.FAMILY_VERSION_DELETED;
246           }
247         }
248       }
249       if (deleteBuffer != null) {
250         int ret = Bytes.compareTo(deleteBuffer, deleteOffset, deleteLength,
251             cell.getQualifierArray(), qualifierOffset, qualifierLength);
252 
253         if (ret == 0) {
254           if (deleteType == KeyValue.Type.DeleteColumn.getCode()) {
255             if (visibilityTagsDeleteColumns != null) {
256               if (!visibilityTagsDeleteColumns.isEmpty()) {
257                 for (Pair<List<Tag>, Byte> tags : visibilityTagsDeleteColumns) {
258                   List<Tag> putVisTags = new ArrayList<Tag>();
259                   Byte putCellVisTagsFormat =
260                       VisibilityUtils.extractVisibilityTags(cell, putVisTags);
261                   boolean matchFound = VisibilityLabelServiceManager.getInstance()
262                       .getVisibilityLabelService().matchVisibility(putVisTags, putCellVisTagsFormat,
263                         tags.getFirst(), tags.getSecond());
264                   if (matchFound) {
265                     return DeleteResult.VERSION_DELETED;
266                   }
267                 }
268               } else {
269                 if (!VisibilityUtils.isVisibilityTagsPresent(cell)) {
270                   // No tags
271                   return DeleteResult.VERSION_DELETED;
272                 }
273               }
274             } else {
275               if (!VisibilityUtils.isVisibilityTagsPresent(cell)) {
276                 // No tags
277                 return DeleteResult.VERSION_DELETED;
278               }
279             }
280           }
281           // Delete (aka DeleteVersion)
282           // If the timestamp is the same, keep this one
283           if (timestamp == deleteTimestamp) {
284             if (visiblityTagsDeleteColumnVersion != null) {
285               if (!visiblityTagsDeleteColumnVersion.isEmpty()) {
286                 for (Pair<List<Tag>, Byte> tags : visiblityTagsDeleteColumnVersion) {
287                   List<Tag> putVisTags = new ArrayList<Tag>();
288                   Byte putCellVisTagsFormat =
289                       VisibilityUtils.extractVisibilityTags(cell, putVisTags);
290                   boolean matchFound = VisibilityLabelServiceManager.getInstance()
291                       .getVisibilityLabelService().matchVisibility(putVisTags, putCellVisTagsFormat,
292                         tags.getFirst(), tags.getSecond());
293                   if (matchFound) {
294                     return DeleteResult.VERSION_DELETED;
295                   }
296                 }
297               } else {
298                 if (!VisibilityUtils.isVisibilityTagsPresent(cell)) {
299                   // No tags
300                   return DeleteResult.VERSION_DELETED;
301                 }
302               }
303             } else {
304               if (!VisibilityUtils.isVisibilityTagsPresent(cell)) {
305                 // No tags
306                 return DeleteResult.VERSION_DELETED;
307               }
308             }
309           }
310         } else if (ret < 0) {
311           // Next column case.
312           deleteBuffer = null;
313           // Can nullify this because we are moving to the next column
314           visibilityTagsDeleteColumns = null;
315           visiblityTagsDeleteColumnVersion = null;
316         } else {
317           throw new IllegalStateException("isDeleted failed: deleteBuffer="
318               + Bytes.toStringBinary(deleteBuffer, deleteOffset, deleteLength) + ", qualifier="
319               + Bytes.toStringBinary(cell.getQualifierArray(), qualifierOffset, qualifierLength)
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 }