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.util.List;
21 import java.util.Map;
22 import java.util.Map.Entry;
23
24 import com.google.protobuf.HBaseZeroCopyByteString;
25 import org.apache.hadoop.classification.InterfaceAudience;
26 import org.apache.hadoop.conf.Configuration;
27 import org.apache.hadoop.hbase.TagType;
28 import org.apache.hadoop.hbase.exceptions.DeserializationException;
29 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
30 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.MultiUserAuthorizations;
31 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.UserAuthorizations;
32 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabel;
33 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsRequest;
34 import org.apache.hadoop.hbase.util.Bytes;
35 import org.apache.hadoop.util.ReflectionUtils;
36
37 import com.google.protobuf.InvalidProtocolBufferException;
38
39
40
41
42 @InterfaceAudience.Private
43 public class VisibilityUtils {
44
45 public static final String VISIBILITY_LABEL_GENERATOR_CLASS =
46 "hbase.regionserver.scan.visibility.label.generator.class";
47 public static final byte VISIBILITY_TAG_TYPE = TagType.VISIBILITY_TAG_TYPE;
48 public static final String SYSTEM_LABEL = "system";
49
50
51
52
53
54
55 public static byte[] getDataToWriteToZooKeeper(Map<String, Integer> existingLabels) {
56 VisibilityLabelsRequest.Builder visReqBuilder = VisibilityLabelsRequest.newBuilder();
57 for (Entry<String, Integer> entry : existingLabels.entrySet()) {
58 VisibilityLabel.Builder visLabBuilder = VisibilityLabel.newBuilder();
59 visLabBuilder.setLabel(HBaseZeroCopyByteString.wrap(Bytes.toBytes(entry.getKey())));
60 visLabBuilder.setOrdinal(entry.getValue());
61 visReqBuilder.addVisLabel(visLabBuilder.build());
62 }
63 return ProtobufUtil.prependPBMagic(visReqBuilder.build().toByteArray());
64 }
65
66
67
68
69
70
71 public static byte[] getUserAuthsDataToWriteToZooKeeper(Map<String, List<Integer>> userAuths) {
72 MultiUserAuthorizations.Builder builder = MultiUserAuthorizations.newBuilder();
73 for (Entry<String, List<Integer>> entry : userAuths.entrySet()) {
74 UserAuthorizations.Builder userAuthsBuilder = UserAuthorizations.newBuilder();
75 userAuthsBuilder.setUser(HBaseZeroCopyByteString.wrap(Bytes.toBytes(entry.getKey())));
76 for (Integer label : entry.getValue()) {
77 userAuthsBuilder.addAuth(label);
78 }
79 builder.addUserAuths(userAuthsBuilder.build());
80 }
81 return ProtobufUtil.prependPBMagic(builder.build().toByteArray());
82 }
83
84
85
86
87
88
89
90
91
92 public static List<VisibilityLabel> readLabelsFromZKData(byte[] data)
93 throws DeserializationException {
94 if (ProtobufUtil.isPBMagicPrefix(data)) {
95 int pblen = ProtobufUtil.lengthOfPBMagic();
96 try {
97 VisibilityLabelsRequest request = VisibilityLabelsRequest.newBuilder()
98 .mergeFrom(data, pblen, data.length - pblen).build();
99 return request.getVisLabelList();
100 } catch (InvalidProtocolBufferException e) {
101 throw new DeserializationException(e);
102 }
103 }
104 return null;
105 }
106
107
108
109
110
111
112
113 public static MultiUserAuthorizations readUserAuthsFromZKData(byte[] data)
114 throws DeserializationException {
115 if (ProtobufUtil.isPBMagicPrefix(data)) {
116 int pblen = ProtobufUtil.lengthOfPBMagic();
117 try {
118 MultiUserAuthorizations multiUserAuths = MultiUserAuthorizations.newBuilder()
119 .mergeFrom(data, pblen, data.length - pblen).build();
120 return multiUserAuths;
121 } catch (InvalidProtocolBufferException e) {
122 throw new DeserializationException(e);
123 }
124 }
125 return null;
126 }
127
128 public static ScanLabelGenerator getScanLabelGenerator(Configuration conf) {
129 Class<? extends ScanLabelGenerator> scanLabelGeneratorKlass = conf
130 .getClass(VISIBILITY_LABEL_GENERATOR_CLASS, DefaultScanLabelGenerator.class,
131 ScanLabelGenerator.class);
132 return ReflectionUtils.newInstance(scanLabelGeneratorKlass, conf);
133 }
134 }