1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.security.visibility;
19
20 import java.io.IOException;
21 import java.util.ArrayList;
22 import java.util.Collections;
23 import java.util.Iterator;
24 import java.util.List;
25 import java.util.Map;
26 import java.util.Map.Entry;
27
28 import org.apache.commons.lang.StringUtils;
29 import org.apache.hadoop.classification.InterfaceAudience;
30 import org.apache.hadoop.conf.Configuration;
31 import org.apache.hadoop.hbase.Cell;
32 import org.apache.hadoop.hbase.CellUtil;
33 import org.apache.hadoop.hbase.Tag;
34 import org.apache.hadoop.hbase.TagType;
35 import org.apache.hadoop.hbase.exceptions.DeserializationException;
36 import org.apache.hadoop.hbase.io.util.StreamUtils;
37 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
38 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.MultiUserAuthorizations;
39 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.UserAuthorizations;
40 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabel;
41 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsRequest;
42 import org.apache.hadoop.hbase.util.ByteStringer;
43 import org.apache.hadoop.hbase.util.Bytes;
44 import org.apache.hadoop.hbase.util.Pair;
45 import org.apache.hadoop.util.ReflectionUtils;
46
47 import com.google.protobuf.InvalidProtocolBufferException;
48
49
50
51
52 @InterfaceAudience.Private
53 public class VisibilityUtils {
54
55 public static final String VISIBILITY_LABEL_GENERATOR_CLASS =
56 "hbase.regionserver.scan.visibility.label.generator.class";
57 public static final byte VISIBILITY_TAG_TYPE = TagType.VISIBILITY_TAG_TYPE;
58 public static final byte VISIBILITY_EXP_SERIALIZATION_TAG_TYPE =
59 TagType.VISIBILITY_EXP_SERIALIZATION_TAG_TYPE;
60 public static final String SYSTEM_LABEL = "system";
61 public static final Tag VIS_SERIALIZATION_TAG = new Tag(VISIBILITY_EXP_SERIALIZATION_TAG_TYPE,
62 VisibilityConstants.SORTED_ORDINAL_SERIALIZATION_FORMAT);
63 private static final String COMMA = ",";
64
65
66
67
68
69
70 public static byte[] getDataToWriteToZooKeeper(Map<String, Integer> existingLabels) {
71 VisibilityLabelsRequest.Builder visReqBuilder = VisibilityLabelsRequest.newBuilder();
72 for (Entry<String, Integer> entry : existingLabels.entrySet()) {
73 VisibilityLabel.Builder visLabBuilder = VisibilityLabel.newBuilder();
74 visLabBuilder.setLabel(ByteStringer.wrap(Bytes.toBytes(entry.getKey())));
75 visLabBuilder.setOrdinal(entry.getValue());
76 visReqBuilder.addVisLabel(visLabBuilder.build());
77 }
78 return ProtobufUtil.prependPBMagic(visReqBuilder.build().toByteArray());
79 }
80
81
82
83
84
85
86 public static byte[] getUserAuthsDataToWriteToZooKeeper(Map<String, List<Integer>> userAuths) {
87 MultiUserAuthorizations.Builder builder = MultiUserAuthorizations.newBuilder();
88 for (Entry<String, List<Integer>> entry : userAuths.entrySet()) {
89 UserAuthorizations.Builder userAuthsBuilder = UserAuthorizations.newBuilder();
90 userAuthsBuilder.setUser(ByteStringer.wrap(Bytes.toBytes(entry.getKey())));
91 for (Integer label : entry.getValue()) {
92 userAuthsBuilder.addAuth(label);
93 }
94 builder.addUserAuths(userAuthsBuilder.build());
95 }
96 return ProtobufUtil.prependPBMagic(builder.build().toByteArray());
97 }
98
99
100
101
102
103
104
105
106
107 public static List<VisibilityLabel> readLabelsFromZKData(byte[] data)
108 throws DeserializationException {
109 if (ProtobufUtil.isPBMagicPrefix(data)) {
110 int pblen = ProtobufUtil.lengthOfPBMagic();
111 try {
112 VisibilityLabelsRequest request = VisibilityLabelsRequest.newBuilder()
113 .mergeFrom(data, pblen, data.length - pblen).build();
114 return request.getVisLabelList();
115 } catch (InvalidProtocolBufferException e) {
116 throw new DeserializationException(e);
117 }
118 }
119 return null;
120 }
121
122
123
124
125
126
127
128 public static MultiUserAuthorizations readUserAuthsFromZKData(byte[] data)
129 throws DeserializationException {
130 if (ProtobufUtil.isPBMagicPrefix(data)) {
131 int pblen = ProtobufUtil.lengthOfPBMagic();
132 try {
133 MultiUserAuthorizations multiUserAuths = MultiUserAuthorizations.newBuilder()
134 .mergeFrom(data, pblen, data.length - pblen).build();
135 return multiUserAuths;
136 } catch (InvalidProtocolBufferException e) {
137 throw new DeserializationException(e);
138 }
139 }
140 return null;
141 }
142
143 public static List<ScanLabelGenerator> getScanLabelGenerators(Configuration conf)
144 throws IOException {
145
146 String slgClassesCommaSeparated = conf.get(VISIBILITY_LABEL_GENERATOR_CLASS);
147
148
149 List<ScanLabelGenerator> slgs = new ArrayList<ScanLabelGenerator>();
150 if (StringUtils.isNotEmpty(slgClassesCommaSeparated)) {
151 String[] slgClasses = slgClassesCommaSeparated.split(COMMA);
152 for (String slgClass : slgClasses) {
153 Class<? extends ScanLabelGenerator> slgKlass;
154 try {
155 slgKlass = (Class<? extends ScanLabelGenerator>) conf.getClassByName(slgClass.trim());
156 slgs.add(ReflectionUtils.newInstance(slgKlass, conf));
157 } catch (ClassNotFoundException e) {
158 throw new IOException(e);
159 }
160 }
161 }
162
163
164 if (slgs.isEmpty()) {
165 slgs.add(ReflectionUtils.newInstance(DefaultScanLabelGenerator.class, conf));
166 }
167 return slgs;
168 }
169
170
171
172
173
174
175
176
177 public static boolean getVisibilityTags(Cell cell, List<Tag> tags) {
178 boolean sortedOrder = false;
179 Iterator<Tag> tagsIterator = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
180 cell.getTagsLengthUnsigned());
181 while (tagsIterator.hasNext()) {
182 Tag tag = tagsIterator.next();
183 if (tag.getType() == VisibilityUtils.VISIBILITY_EXP_SERIALIZATION_TAG_TYPE) {
184 int serializationVersion = tag.getBuffer()[tag.getTagOffset()];
185 if (serializationVersion == VisibilityConstants.VISIBILITY_SERIALIZATION_VERSION) {
186 sortedOrder = true;
187 continue;
188 }
189 }
190 if (tag.getType() == VisibilityUtils.VISIBILITY_TAG_TYPE) {
191 tags.add(tag);
192 }
193 }
194 return sortedOrder;
195 }
196
197
198
199
200
201
202 public static boolean isVisibilityTagsPresent(Cell cell) {
203 Iterator<Tag> tagsIterator = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
204 cell.getTagsLengthUnsigned());
205 while (tagsIterator.hasNext()) {
206 Tag tag = tagsIterator.next();
207 if (tag.getType() == VisibilityUtils.VISIBILITY_TAG_TYPE) {
208 return true;
209 }
210 }
211 return false;
212 }
213
214
215
216
217
218
219
220
221
222 public static boolean checkForMatchingVisibilityTags(Cell cell,
223 List<Tag> visibilityTagsInDeleteCell) {
224 List<Tag> tags = new ArrayList<Tag>();
225 boolean sortedTags = getVisibilityTags(cell, tags);
226 if (tags.size() == 0) {
227
228 return false;
229 }
230 if (sortedTags) {
231 return checkForMatchingVisibilityTagsWithSortedOrder(visibilityTagsInDeleteCell, tags);
232 } else {
233 try {
234 return checkForMatchingVisibilityTagsWithOutSortedOrder(cell, visibilityTagsInDeleteCell);
235 } catch (IOException e) {
236
237 throw new RuntimeException("Exception while sorting the tags from the cell", e);
238 }
239 }
240 }
241
242 private static boolean checkForMatchingVisibilityTagsWithOutSortedOrder(Cell cell,
243 List<Tag> visibilityTagsInDeleteCell) throws IOException {
244 List<List<Integer>> sortedDeleteTags = sortTagsBasedOnOrdinal(
245 visibilityTagsInDeleteCell);
246 List<List<Integer>> sortedTags = sortTagsBasedOnOrdinal(cell);
247 return compareTagsOrdinals(sortedDeleteTags, sortedTags);
248 }
249
250 private static boolean checkForMatchingVisibilityTagsWithSortedOrder(
251 List<Tag> visibilityTagsInDeleteCell, List<Tag> tags) {
252 boolean matchFound = false;
253 if ((visibilityTagsInDeleteCell.size()) != tags.size()) {
254
255
256
257 return matchFound;
258 }
259 for (Tag tag : visibilityTagsInDeleteCell) {
260 matchFound = false;
261 for (Tag givenTag : tags) {
262 if (Bytes.equals(tag.getBuffer(), tag.getTagOffset(), tag.getTagLength(),
263 givenTag.getBuffer(), givenTag.getTagOffset(), givenTag.getTagLength())) {
264 matchFound = true;
265 break;
266 }
267 }
268 }
269 return matchFound;
270 }
271
272 private static List<List<Integer>> sortTagsBasedOnOrdinal(Cell cell) throws IOException {
273 Iterator<Tag> tagsItr = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
274 cell.getTagsLengthUnsigned());
275 List<List<Integer>> fullTagsList = new ArrayList<List<Integer>>();
276 while (tagsItr.hasNext()) {
277 Tag tag = tagsItr.next();
278 if (tag.getType() == VisibilityUtils.VISIBILITY_TAG_TYPE) {
279 getSortedTagOrdinals(fullTagsList, tag);
280 }
281 }
282 return fullTagsList;
283 }
284
285 private static List<List<Integer>> sortTagsBasedOnOrdinal(List<Tag> tags) throws IOException {
286 List<List<Integer>> fullTagsList = new ArrayList<List<Integer>>();
287 for (Tag tag : tags) {
288 if (tag.getType() == VisibilityUtils.VISIBILITY_TAG_TYPE) {
289 getSortedTagOrdinals(fullTagsList, tag);
290 }
291 }
292 return fullTagsList;
293 }
294
295 private static void getSortedTagOrdinals(List<List<Integer>> fullTagsList, Tag tag)
296 throws IOException {
297 List<Integer> tagsOrdinalInSortedOrder = new ArrayList<Integer>();
298 int offset = tag.getTagOffset();
299 int endOffset = offset + tag.getTagLength();
300 while (offset < endOffset) {
301 Pair<Integer, Integer> result = StreamUtils.readRawVarint32(tag.getBuffer(), offset);
302 tagsOrdinalInSortedOrder.add(result.getFirst());
303 offset += result.getSecond();
304 }
305 Collections.sort(tagsOrdinalInSortedOrder);
306 fullTagsList.add(tagsOrdinalInSortedOrder);
307 }
308
309 private static boolean compareTagsOrdinals(List<List<Integer>> tagsInDeletes,
310 List<List<Integer>> tags) {
311 boolean matchFound = false;
312 if (tagsInDeletes.size() != tags.size()) {
313 return matchFound;
314 } else {
315 for (List<Integer> deleteTagOrdinals : tagsInDeletes) {
316 matchFound = false;
317 for (List<Integer> tagOrdinals : tags) {
318 if (deleteTagOrdinals.equals(tagOrdinals)) {
319 matchFound = true;
320 break;
321 }
322 }
323 }
324 return matchFound;
325 }
326 }
327 }