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