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