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