1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.security.token;
20
21 import javax.crypto.SecretKey;
22 import java.io.IOException;
23 import java.util.Iterator;
24 import java.util.Map;
25 import java.util.concurrent.ConcurrentHashMap;
26 import java.util.concurrent.atomic.AtomicLong;
27
28 import org.apache.commons.logging.Log;
29 import org.apache.commons.logging.LogFactory;
30 import org.apache.hadoop.hbase.classification.InterfaceAudience;
31 import org.apache.hadoop.conf.Configuration;
32 import org.apache.hadoop.hbase.Stoppable;
33 import org.apache.hadoop.hbase.util.Bytes;
34 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
35 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
36 import org.apache.hadoop.hbase.zookeeper.ZKLeaderManager;
37 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
38 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
39 import org.apache.hadoop.io.Text;
40 import org.apache.hadoop.io.WritableUtils;
41 import org.apache.hadoop.security.token.SecretManager;
42 import org.apache.hadoop.security.token.Token;
43 import org.apache.zookeeper.KeeperException;
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58 @InterfaceAudience.Private
59 public class AuthenticationTokenSecretManager
60 extends SecretManager<AuthenticationTokenIdentifier> {
61
62 static final String NAME_PREFIX = "SecretManager-";
63
64 private static Log LOG = LogFactory.getLog(
65 AuthenticationTokenSecretManager.class);
66
67 private long lastKeyUpdate;
68 private long keyUpdateInterval;
69 private long tokenMaxLifetime;
70 private ZKSecretWatcher zkWatcher;
71 private LeaderElector leaderElector;
72 private ZKClusterId clusterId;
73
74 private Map<Integer,AuthenticationKey> allKeys =
75 new ConcurrentHashMap<Integer, AuthenticationKey>();
76 private AuthenticationKey currentKey;
77
78 private int idSeq;
79 private AtomicLong tokenSeq = new AtomicLong();
80 private String name;
81
82
83
84
85
86
87
88
89
90
91
92
93 public AuthenticationTokenSecretManager(Configuration conf,
94 ZooKeeperWatcher zk, String serverName,
95 long keyUpdateInterval, long tokenMaxLifetime) {
96 this.zkWatcher = new ZKSecretWatcher(conf, zk, this);
97 this.keyUpdateInterval = keyUpdateInterval;
98 this.tokenMaxLifetime = tokenMaxLifetime;
99 this.leaderElector = new LeaderElector(zk, serverName);
100 this.name = NAME_PREFIX+serverName;
101 this.clusterId = new ZKClusterId(zk, zk);
102 }
103
104 public void start() {
105 try {
106
107 this.zkWatcher.start();
108
109 this.leaderElector.start();
110 } catch (KeeperException ke) {
111 LOG.error("Zookeeper initialization failed", ke);
112 }
113 }
114
115 public void stop() {
116 this.leaderElector.stop("SecretManager stopping");
117 }
118
119 public boolean isMaster() {
120 return leaderElector.isMaster();
121 }
122
123 public String getName() {
124 return name;
125 }
126
127 @Override
128 protected byte[] createPassword(AuthenticationTokenIdentifier identifier) {
129 long now = EnvironmentEdgeManager.currentTimeMillis();
130 AuthenticationKey secretKey = currentKey;
131 identifier.setKeyId(secretKey.getKeyId());
132 identifier.setIssueDate(now);
133 identifier.setExpirationDate(now + tokenMaxLifetime);
134 identifier.setSequenceNumber(tokenSeq.getAndIncrement());
135 return createPassword(identifier.getBytes(),
136 secretKey.getKey());
137 }
138
139 @Override
140 public byte[] retrievePassword(AuthenticationTokenIdentifier identifier)
141 throws InvalidToken {
142 long now = EnvironmentEdgeManager.currentTimeMillis();
143 if (identifier.getExpirationDate() < now) {
144 throw new InvalidToken("Token has expired");
145 }
146 AuthenticationKey masterKey = allKeys.get(identifier.getKeyId());
147 if(masterKey == null) {
148 if(zkWatcher.getWatcher().isAborted()) {
149 LOG.error("ZookeeperWatcher is abort");
150 throw new InvalidToken("Token keys could not be sync from zookeeper"
151 + " because of ZookeeperWatcher abort");
152 }
153 synchronized (this) {
154 if (!leaderElector.isAlive() || leaderElector.isStopped()) {
155 LOG.warn("Thread leaderElector[" + leaderElector.getName() + ":"
156 + leaderElector.getId() + "] is stoped or not alive");
157 leaderElector.start();
158 LOG.info("Thread leaderElector [" + leaderElector.getName() + ":"
159 + leaderElector.getId() + "] is started");
160 }
161 }
162 zkWatcher.refreshKeys();
163 if (LOG.isDebugEnabled()) {
164 LOG.debug("Sync token keys from zookeeper");
165 }
166 masterKey = allKeys.get(identifier.getKeyId());
167 }
168 if (masterKey == null) {
169 throw new InvalidToken("Unknown master key for token (id="+
170 identifier.getKeyId()+")");
171 }
172
173 return createPassword(identifier.getBytes(),
174 masterKey.getKey());
175 }
176
177 @Override
178 public AuthenticationTokenIdentifier createIdentifier() {
179 return new AuthenticationTokenIdentifier();
180 }
181
182 public Token<AuthenticationTokenIdentifier> generateToken(String username) {
183 AuthenticationTokenIdentifier ident =
184 new AuthenticationTokenIdentifier(username);
185 Token<AuthenticationTokenIdentifier> token =
186 new Token<AuthenticationTokenIdentifier>(ident, this);
187 if (clusterId.hasId()) {
188 token.setService(new Text(clusterId.getId()));
189 }
190 return token;
191 }
192
193 public synchronized void addKey(AuthenticationKey key) throws IOException {
194
195 if (leaderElector.isMaster()) {
196 if (LOG.isDebugEnabled()) {
197 LOG.debug("Running as master, ignoring new key "+key.getKeyId());
198 }
199 return;
200 }
201
202 if (LOG.isDebugEnabled()) {
203 LOG.debug("Adding key "+key.getKeyId());
204 }
205
206 allKeys.put(key.getKeyId(), key);
207 if (currentKey == null || key.getKeyId() > currentKey.getKeyId()) {
208 currentKey = key;
209 }
210
211 if (key.getKeyId() > idSeq) {
212 idSeq = key.getKeyId();
213 }
214 }
215
216 synchronized boolean removeKey(Integer keyId) {
217
218 if (leaderElector.isMaster()) {
219 if (LOG.isDebugEnabled()) {
220 LOG.debug("Running as master, ignoring removed key "+keyId);
221 }
222 return false;
223 }
224
225 if (LOG.isDebugEnabled()) {
226 LOG.debug("Removing key "+keyId);
227 }
228
229 allKeys.remove(keyId);
230 return true;
231 }
232
233 AuthenticationKey getCurrentKey() {
234 return currentKey;
235 }
236
237 AuthenticationKey getKey(int keyId) {
238 return allKeys.get(keyId);
239 }
240
241 synchronized void removeExpiredKeys() {
242 if (!leaderElector.isMaster()) {
243 LOG.info("Skipping removeExpiredKeys() because not running as master.");
244 return;
245 }
246
247 long now = EnvironmentEdgeManager.currentTimeMillis();
248 Iterator<AuthenticationKey> iter = allKeys.values().iterator();
249 while (iter.hasNext()) {
250 AuthenticationKey key = iter.next();
251 if (key.getExpiration() < now) {
252 if (LOG.isDebugEnabled()) {
253 LOG.debug("Removing expired key "+key.getKeyId());
254 }
255 iter.remove();
256 zkWatcher.removeKeyFromZK(key);
257 }
258 }
259 }
260
261 synchronized boolean isCurrentKeyRolled() {
262 return currentKey != null;
263 }
264
265 synchronized void rollCurrentKey() {
266 if (!leaderElector.isMaster()) {
267 LOG.info("Skipping rollCurrentKey() because not running as master.");
268 return;
269 }
270
271 long now = EnvironmentEdgeManager.currentTimeMillis();
272 AuthenticationKey prev = currentKey;
273 AuthenticationKey newKey = new AuthenticationKey(++idSeq,
274 Long.MAX_VALUE,
275 generateSecret());
276 allKeys.put(newKey.getKeyId(), newKey);
277 currentKey = newKey;
278 zkWatcher.addKeyToZK(newKey);
279 lastKeyUpdate = now;
280
281 if (prev != null) {
282
283 prev.setExpiration(now + tokenMaxLifetime);
284 allKeys.put(prev.getKeyId(), prev);
285 zkWatcher.updateKeyInZK(prev);
286 }
287 }
288
289 synchronized long getLastKeyUpdate() {
290 return lastKeyUpdate;
291 }
292
293 public static SecretKey createSecretKey(byte[] raw) {
294 return SecretManager.createSecretKey(raw);
295 }
296
297 private class LeaderElector extends Thread implements Stoppable {
298 private boolean stopped = false;
299
300 private boolean isMaster = false;
301 private ZKLeaderManager zkLeader;
302
303 public LeaderElector(ZooKeeperWatcher watcher, String serverName) {
304 setDaemon(true);
305 setName("ZKSecretWatcher-leaderElector");
306 zkLeader = new ZKLeaderManager(watcher,
307 ZKUtil.joinZNode(zkWatcher.getRootKeyZNode(), "keymaster"),
308 Bytes.toBytes(serverName), this);
309 }
310
311 public boolean isMaster() {
312 return isMaster;
313 }
314
315 @Override
316 public boolean isStopped() {
317 return stopped;
318 }
319
320 @Override
321 public void stop(String reason) {
322 if (stopped) {
323 return;
324 }
325
326 stopped = true;
327
328 if (isMaster) {
329 zkLeader.stepDownAsLeader();
330 }
331 isMaster = false;
332 LOG.info("Stopping leader election, because: "+reason);
333 interrupt();
334 }
335
336 public void run() {
337 zkLeader.start();
338 zkLeader.waitToBecomeLeader();
339 isMaster = true;
340
341 while (!stopped) {
342 long now = EnvironmentEdgeManager.currentTimeMillis();
343
344
345 removeExpiredKeys();
346 long localLastKeyUpdate = getLastKeyUpdate();
347 if (localLastKeyUpdate + keyUpdateInterval < now) {
348
349 rollCurrentKey();
350 }
351
352 try {
353 Thread.sleep(5000);
354 } catch (InterruptedException ie) {
355 if (LOG.isDebugEnabled()) {
356 LOG.debug("Interrupted waiting for next update", ie);
357 }
358 }
359 }
360 }
361 }
362 }