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.hbase.classification.InterfaceAudience;
33 import org.apache.hadoop.conf.Configuration;
34 import org.apache.hadoop.hbase.exceptions.DeserializationException;
35 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.MultiUserAuthorizations;
36 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.UserAuthorizations;
37 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabel;
38 import org.apache.hadoop.hbase.util.Bytes;
39 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
40 import org.apache.zookeeper.KeeperException;
41
42
43
44
45
46
47 @InterfaceAudience.Private
48 public class VisibilityLabelsCache implements VisibilityLabelOrdinalProvider {
49
50 private static final Log LOG = LogFactory.getLog(VisibilityLabelsCache.class);
51 private static final int NON_EXIST_LABEL_ORDINAL = 0;
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
61
62
63 private ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
64
65 private VisibilityLabelsCache(ZooKeeperWatcher watcher, Configuration conf) throws IOException {
66 zkVisibilityWatcher = new ZKVisibilityLabelWatcher(watcher, this, conf);
67 try {
68 zkVisibilityWatcher.start();
69 } catch (KeeperException ke) {
70 LOG.error("ZooKeeper initialization failed", ke);
71 throw new IOException(ke);
72 }
73 }
74
75
76
77
78
79
80
81
82 public synchronized static VisibilityLabelsCache createAndGet(ZooKeeperWatcher watcher,
83 Configuration conf) throws IOException {
84
85
86
87
88
89
90 if (instance == null || watcher != instance.zkVisibilityWatcher.getWatcher()) {
91 instance = new VisibilityLabelsCache(watcher, conf);
92 }
93 return instance;
94 }
95
96
97
98
99
100
101
102 public static VisibilityLabelsCache get() {
103
104
105 if (instance == null) {
106 throw new IllegalStateException("VisibilityLabelsCache not yet instantiated");
107 }
108 return instance;
109 }
110
111 public void refreshLabelsCache(byte[] data) throws IOException {
112 List<VisibilityLabel> visibilityLabels = null;
113 try {
114 visibilityLabels = VisibilityUtils.readLabelsFromZKData(data);
115 } catch (DeserializationException dse) {
116 throw new IOException(dse);
117 }
118 this.lock.writeLock().lock();
119 try {
120 labels.clear();
121 ordinalVsLabels.clear();
122 for (VisibilityLabel visLabel : visibilityLabels) {
123 String label = Bytes.toString(visLabel.getLabel().toByteArray());
124 labels.put(label, visLabel.getOrdinal());
125 ordinalVsLabels.put(visLabel.getOrdinal(), label);
126 }
127 } finally {
128 this.lock.writeLock().unlock();
129 }
130 }
131
132 public void refreshUserAuthsCache(byte[] data) throws IOException {
133 MultiUserAuthorizations multiUserAuths = null;
134 try {
135 multiUserAuths = VisibilityUtils.readUserAuthsFromZKData(data);
136 } catch (DeserializationException dse) {
137 throw new IOException(dse);
138 }
139 this.lock.writeLock().lock();
140 try {
141 this.userAuths.clear();
142 for (UserAuthorizations userAuths : multiUserAuths.getUserAuthsList()) {
143 String user = Bytes.toString(userAuths.getUser().toByteArray());
144 this.userAuths.put(user, new HashSet<Integer>(userAuths.getAuthList()));
145 }
146 } finally {
147 this.lock.writeLock().unlock();
148 }
149 }
150
151
152
153
154
155
156 @Override
157 public int getLabelOrdinal(String label) {
158 Integer ordinal = null;
159 this.lock.readLock().lock();
160 try {
161 ordinal = labels.get(label);
162 } finally {
163 this.lock.readLock().unlock();
164 }
165 if (ordinal != null) {
166 return ordinal.intValue();
167 }
168
169 return NON_EXIST_LABEL_ORDINAL;
170 }
171
172
173
174
175
176
177 public String getLabel(int ordinal) {
178 this.lock.readLock().lock();
179 try {
180 return this.ordinalVsLabels.get(ordinal);
181 } finally {
182 this.lock.readLock().unlock();
183 }
184 }
185
186
187
188
189 public int getLabelsCount() {
190 this.lock.readLock().lock();
191 try {
192 return this.labels.size();
193 } finally {
194 this.lock.readLock().unlock();
195 }
196 }
197
198 public List<String> getAuths(String user) {
199 List<String> auths = EMPTY_LIST;
200 this.lock.readLock().lock();
201 try {
202 Set<Integer> authOrdinals = userAuths.get(user);
203 if (authOrdinals != null) {
204 auths = new ArrayList<String>(authOrdinals.size());
205 for (Integer authOrdinal : authOrdinals) {
206 auths.add(ordinalVsLabels.get(authOrdinal));
207 }
208 }
209 } finally {
210 this.lock.readLock().unlock();
211 }
212 return auths;
213 }
214
215
216
217
218
219
220
221 public Set<Integer> getAuthsAsOrdinals(String user) {
222 this.lock.readLock().lock();
223 try {
224 Set<Integer> auths = userAuths.get(user);
225 return (auths == null) ? EMPTY_SET : auths;
226 } finally {
227 this.lock.readLock().unlock();
228 }
229 }
230
231 public void writeToZookeeper(byte[] data, boolean labelsOrUserAuths) {
232 this.zkVisibilityWatcher.writeToZookeeper(data, labelsOrUserAuths);
233 }
234 }