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.HashMap;
24 import java.util.HashSet;
25 import java.util.List;
26 import java.util.Map;
27 import java.util.Set;
28 import java.util.concurrent.locks.ReentrantReadWriteLock;
29
30 import org.apache.commons.logging.Log;
31 import org.apache.commons.logging.LogFactory;
32 import org.apache.hadoop.conf.Configuration;
33 import org.apache.hadoop.hbase.AuthUtil;
34 import org.apache.hadoop.hbase.classification.InterfaceAudience;
35 import org.apache.hadoop.hbase.exceptions.DeserializationException;
36 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.MultiUserAuthorizations;
37 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.UserAuthorizations;
38 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabel;
39 import org.apache.hadoop.hbase.util.Bytes;
40 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
41 import org.apache.zookeeper.KeeperException;
42
43
44
45
46
47
48 @InterfaceAudience.Private
49 public class VisibilityLabelsCache implements VisibilityLabelOrdinalProvider {
50
51 private static final Log LOG = LogFactory.getLog(VisibilityLabelsCache.class);
52 private static final int NON_EXIST_LABEL_ORDINAL = 0;
53 private static final List<String> EMPTY_LIST = Collections.emptyList();
54 private static final Set<Integer> EMPTY_SET = Collections.emptySet();
55 private static VisibilityLabelsCache instance;
56
57 private ZKVisibilityLabelWatcher zkVisibilityWatcher;
58 private Map<String, Integer> labels = new HashMap<String, Integer>();
59 private Map<Integer, String> ordinalVsLabels = new HashMap<Integer, String>();
60 private Map<String, Set<Integer>> userAuths = new HashMap<String, Set<Integer>>();
61 private Map<String, Set<Integer>> groupAuths = new HashMap<String, Set<Integer>>();
62
63
64
65
66 private ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
67
68 private VisibilityLabelsCache(ZooKeeperWatcher watcher, Configuration conf) throws IOException {
69 zkVisibilityWatcher = new ZKVisibilityLabelWatcher(watcher, this, conf);
70 try {
71 zkVisibilityWatcher.start();
72 } catch (KeeperException ke) {
73 LOG.error("ZooKeeper initialization failed", ke);
74 throw new IOException(ke);
75 }
76 }
77
78
79
80
81
82
83
84
85 public synchronized static VisibilityLabelsCache createAndGet(ZooKeeperWatcher watcher,
86 Configuration conf) throws IOException {
87
88
89
90
91
92
93 if (instance == null || watcher != instance.zkVisibilityWatcher.getWatcher()) {
94 instance = new VisibilityLabelsCache(watcher, conf);
95 }
96 return instance;
97 }
98
99
100
101
102
103
104
105 public static VisibilityLabelsCache get() {
106
107
108 if (instance == null) {
109 throw new IllegalStateException("VisibilityLabelsCache not yet instantiated");
110 }
111 return instance;
112 }
113
114 public void refreshLabelsCache(byte[] data) throws IOException {
115 List<VisibilityLabel> visibilityLabels = null;
116 try {
117 visibilityLabels = VisibilityUtils.readLabelsFromZKData(data);
118 } catch (DeserializationException dse) {
119 throw new IOException(dse);
120 }
121 this.lock.writeLock().lock();
122 try {
123 labels.clear();
124 ordinalVsLabels.clear();
125 for (VisibilityLabel visLabel : visibilityLabels) {
126 String label = Bytes.toString(visLabel.getLabel().toByteArray());
127 labels.put(label, visLabel.getOrdinal());
128 ordinalVsLabels.put(visLabel.getOrdinal(), label);
129 }
130 } finally {
131 this.lock.writeLock().unlock();
132 }
133 }
134
135 public void refreshUserAuthsCache(byte[] data) throws IOException {
136 MultiUserAuthorizations multiUserAuths = null;
137 try {
138 multiUserAuths = VisibilityUtils.readUserAuthsFromZKData(data);
139 } catch (DeserializationException dse) {
140 throw new IOException(dse);
141 }
142 this.lock.writeLock().lock();
143 try {
144 this.userAuths.clear();
145 this.groupAuths.clear();
146 for (UserAuthorizations userAuths : multiUserAuths.getUserAuthsList()) {
147 String user = Bytes.toString(userAuths.getUser().toByteArray());
148 if (AuthUtil.isGroupPrincipal(user)) {
149 this.groupAuths.put(AuthUtil.getGroupName(user),
150 new HashSet<Integer>(userAuths.getAuthList()));
151 } else {
152 this.userAuths.put(user, new HashSet<Integer>(userAuths.getAuthList()));
153 }
154 }
155 } finally {
156 this.lock.writeLock().unlock();
157 }
158 }
159
160
161
162
163
164
165 @Override
166 public int getLabelOrdinal(String label) {
167 Integer ordinal = null;
168 this.lock.readLock().lock();
169 try {
170 ordinal = labels.get(label);
171 } finally {
172 this.lock.readLock().unlock();
173 }
174 if (ordinal != null) {
175 return ordinal.intValue();
176 }
177
178 return NON_EXIST_LABEL_ORDINAL;
179 }
180
181
182
183
184
185
186 @Override
187 public String getLabel(int ordinal) {
188 this.lock.readLock().lock();
189 try {
190 return this.ordinalVsLabels.get(ordinal);
191 } finally {
192 this.lock.readLock().unlock();
193 }
194 }
195
196
197
198
199 public int getLabelsCount() {
200 this.lock.readLock().lock();
201 try {
202 return this.labels.size();
203 } finally {
204 this.lock.readLock().unlock();
205 }
206 }
207
208 public List<String> getUserAuths(String user) {
209 this.lock.readLock().lock();
210 try {
211 List<String> auths = EMPTY_LIST;
212 Set<Integer> authOrdinals = getUserAuthsAsOrdinals(user);
213 if (!authOrdinals.equals(EMPTY_SET)) {
214 auths = new ArrayList<String>(authOrdinals.size());
215 for (Integer authOrdinal : authOrdinals) {
216 auths.add(ordinalVsLabels.get(authOrdinal));
217 }
218 }
219 return auths;
220 } finally {
221 this.lock.readLock().unlock();
222 }
223 }
224
225 public List<String> getGroupAuths(String[] groups) {
226 this.lock.readLock().lock();
227 try {
228 List<String> auths = EMPTY_LIST;
229 Set<Integer> authOrdinals = getGroupAuthsAsOrdinals(groups);
230 if (!authOrdinals.equals(EMPTY_SET)) {
231 auths = new ArrayList<String>(authOrdinals.size());
232 for (Integer authOrdinal : authOrdinals) {
233 auths.add(ordinalVsLabels.get(authOrdinal));
234 }
235 }
236 return auths;
237 } finally {
238 this.lock.readLock().unlock();
239 }
240 }
241
242
243
244
245
246
247
248 public Set<Integer> getUserAuthsAsOrdinals(String user) {
249 this.lock.readLock().lock();
250 try {
251 Set<Integer> auths = userAuths.get(user);
252 return (auths == null) ? EMPTY_SET : auths;
253 } finally {
254 this.lock.readLock().unlock();
255 }
256 }
257
258
259
260
261
262
263
264 public Set<Integer> getGroupAuthsAsOrdinals(String[] groups) {
265 this.lock.readLock().lock();
266 try {
267 Set<Integer> authOrdinals = new HashSet<Integer>();
268 if (groups != null && groups.length > 0) {
269 Set<Integer> groupAuthOrdinals = null;
270 for (String group : groups) {
271 groupAuthOrdinals = groupAuths.get(group);
272 if (groupAuthOrdinals != null && !groupAuthOrdinals.isEmpty()) {
273 authOrdinals.addAll(groupAuthOrdinals);
274 }
275 }
276 }
277 return (authOrdinals.isEmpty()) ? EMPTY_SET : authOrdinals;
278 } finally {
279 this.lock.readLock().unlock();
280 }
281 }
282
283 public void writeToZookeeper(byte[] data, boolean labelsOrUserAuths) throws IOException {
284
285 if (labelsOrUserAuths) {
286
287 this.refreshLabelsCache(data);
288 } else {
289
290 this.refreshUserAuthsCache(data);
291 }
292 this.zkVisibilityWatcher.writeToZookeeper(data, labelsOrUserAuths);
293 }
294 }