View Javadoc

1   /*
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
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   * Manages an internal list of secret keys used to sign new authentication
46   * tokens as they are generated, and to valid existing tokens used for
47   * authentication.
48   *
49   * <p>
50   * A single instance of {@code AuthenticationTokenSecretManager} will be
51   * running as the "leader" in a given HBase cluster.  The leader is responsible
52   * for periodically generating new secret keys, which are then distributed to
53   * followers via ZooKeeper, and for expiring previously used secret keys that
54   * are no longer needed (as any tokens using them have expired).
55   * </p>
56   */
57  @InterfaceAudience.Private
58  public class AuthenticationTokenSecretManager
59      extends SecretManager<AuthenticationTokenIdentifier> {
60  
61    static final String NAME_PREFIX = "SecretManager-";
62  
63    private static final 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     * Create a new secret manager instance for generating keys.
83     * @param conf Configuration to use
84     * @param zk Connection to zookeeper for handling leader elections
85     * @param keyUpdateInterval Time (in milliseconds) between rolling a new master key for token signing
86     * @param tokenMaxLifetime Maximum age (in milliseconds) before a token expires and is no longer valid
87     */
88    /* TODO: Restrict access to this constructor to make rogues instances more difficult.
89     * For the moment this class is instantiated from
90     * org.apache.hadoop.hbase.ipc.SecureServer so public access is needed.
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       // populate any existing keys
106       this.zkWatcher.start();
107       // try to become leader
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 synchronized 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       if(zkWatcher.getWatcher().isAborted()) {
148         LOG.error("ZookeeperWatcher is abort");
149         throw new InvalidToken("Token keys could not be sync from zookeeper"
150             + " because of ZookeeperWatcher abort");
151       }
152       synchronized (this) {
153         if (!leaderElector.isAlive() || leaderElector.isStopped()) {
154           LOG.warn("Thread leaderElector[" + leaderElector.getName() + ":"
155               + leaderElector.getId() + "] is stoped or not alive");
156           leaderElector.start();
157           LOG.info("Thread leaderElector [" + leaderElector.getName() + ":"
158               + leaderElector.getId() + "] is started");
159         }
160       }
161       zkWatcher.refreshKeys();
162       if (LOG.isDebugEnabled()) {
163         LOG.debug("Sync token keys from zookeeper");
164       }
165       masterKey = allKeys.get(identifier.getKeyId());
166     }
167     if (masterKey == null) {
168       throw new InvalidToken("Unknown master key for token (id="+
169           identifier.getKeyId()+")");
170     }
171     // regenerate the password
172     return createPassword(identifier.getBytes(),
173         masterKey.getKey());
174   }
175 
176   @Override
177   public AuthenticationTokenIdentifier createIdentifier() {
178     return new AuthenticationTokenIdentifier();
179   }
180 
181   public Token<AuthenticationTokenIdentifier> generateToken(String username) {
182     AuthenticationTokenIdentifier ident =
183         new AuthenticationTokenIdentifier(username);
184     Token<AuthenticationTokenIdentifier> token =
185         new Token<AuthenticationTokenIdentifier>(ident, this);
186     if (clusterId.hasId()) {
187       token.setService(new Text(clusterId.getId()));
188     }
189     return token;
190   }
191 
192   public synchronized void addKey(AuthenticationKey key) throws IOException {
193     // ignore zk changes when running as master
194     if (leaderElector.isMaster()) {
195       if (LOG.isDebugEnabled()) {
196         LOG.debug("Running as master, ignoring new key "+key.getKeyId());
197       }
198       return;
199     }
200 
201     if (LOG.isDebugEnabled()) {
202       LOG.debug("Adding key "+key.getKeyId());
203     }
204 
205     allKeys.put(key.getKeyId(), key);
206     if (currentKey == null || key.getKeyId() > currentKey.getKeyId()) {
207       currentKey = key;
208     }
209     // update current sequence
210     if (key.getKeyId() > idSeq) {
211       idSeq = key.getKeyId();
212     }
213   }
214 
215   synchronized boolean removeKey(Integer keyId) {
216     // ignore zk changes when running as master
217     if (leaderElector.isMaster()) {
218       if (LOG.isDebugEnabled()) {
219         LOG.debug("Running as master, ignoring removed key "+keyId);
220       }
221       return false;
222     }
223 
224     if (LOG.isDebugEnabled()) {
225       LOG.debug("Removing key "+keyId);
226     }
227 
228     allKeys.remove(keyId);
229     return true;
230   }
231 
232   synchronized AuthenticationKey getCurrentKey() {
233     return currentKey;
234   }
235 
236   AuthenticationKey getKey(int keyId) {
237     return allKeys.get(keyId);
238   }
239 
240   synchronized void removeExpiredKeys() {
241     if (!leaderElector.isMaster()) {
242       LOG.info("Skipping removeExpiredKeys() because not running as master.");
243       return;
244     }
245 
246     long now = EnvironmentEdgeManager.currentTime();
247     Iterator<AuthenticationKey> iter = allKeys.values().iterator();
248     while (iter.hasNext()) {
249       AuthenticationKey key = iter.next();
250       if (key.getExpiration() < now) {
251         if (LOG.isDebugEnabled()) {
252           LOG.debug("Removing expired key "+key.getKeyId());
253         }
254         iter.remove();
255         zkWatcher.removeKeyFromZK(key);
256       }
257     }
258   }
259   
260   synchronized boolean isCurrentKeyRolled() {
261     return currentKey != null;
262   }
263 
264   synchronized void rollCurrentKey() {
265     if (!leaderElector.isMaster()) {
266       LOG.info("Skipping rollCurrentKey() because not running as master.");
267       return;
268     }
269 
270     long now = EnvironmentEdgeManager.currentTime();
271     AuthenticationKey prev = currentKey;
272     AuthenticationKey newKey = new AuthenticationKey(++idSeq,
273         Long.MAX_VALUE, // don't allow to expire until it's replaced by a new key
274         generateSecret());
275     allKeys.put(newKey.getKeyId(), newKey);
276     currentKey = newKey;
277     zkWatcher.addKeyToZK(newKey);
278     lastKeyUpdate = now;
279 
280     if (prev != null) {
281       // make sure previous key is still stored
282       prev.setExpiration(now + tokenMaxLifetime);
283       allKeys.put(prev.getKeyId(), prev);
284       zkWatcher.updateKeyInZK(prev);
285     }
286   }
287 
288   synchronized long getLastKeyUpdate() {
289     return lastKeyUpdate;
290   }
291 
292   public static SecretKey createSecretKey(byte[] raw) {
293     return SecretManager.createSecretKey(raw);
294   }
295 
296   private class LeaderElector extends Thread implements Stoppable {
297     private boolean stopped = false;
298     /** Flag indicating whether we're in charge of rolling/expiring keys */
299     private boolean isMaster = false;
300     private ZKLeaderManager zkLeader;
301 
302     public LeaderElector(ZooKeeperWatcher watcher, String serverName) {
303       setDaemon(true);
304       setName("ZKSecretWatcher-leaderElector");
305       zkLeader = new ZKLeaderManager(watcher,
306           ZKUtil.joinZNode(zkWatcher.getRootKeyZNode(), "keymaster"),
307           Bytes.toBytes(serverName), this);
308     }
309 
310     public boolean isMaster() {
311       return isMaster;
312     }
313 
314     @Override
315     public boolean isStopped() {
316       return stopped;
317     }
318 
319     @Override
320     public void stop(String reason) {
321       if (stopped) {
322         return;
323       }
324 
325       stopped = true;
326       // prevent further key generation when stopping
327       if (isMaster) {
328         zkLeader.stepDownAsLeader();
329       }
330       isMaster = false;
331       LOG.info("Stopping leader election, because: "+reason);
332       interrupt();
333     }
334 
335     public void run() {
336       zkLeader.start();
337       zkLeader.waitToBecomeLeader();
338       isMaster = true;
339 
340       while (!stopped) {
341         long now = EnvironmentEdgeManager.currentTime();
342 
343         // clear any expired
344         removeExpiredKeys();
345         long localLastKeyUpdate = getLastKeyUpdate();
346         if (localLastKeyUpdate + keyUpdateInterval < now) {
347           // roll a new master key
348           rollCurrentKey();
349         }
350 
351         try {
352           Thread.sleep(5000);
353         } catch (InterruptedException ie) {
354           if (LOG.isDebugEnabled()) {
355             LOG.debug("Interrupted waiting for next update", ie);
356           }
357         }
358       }
359     }
360   }
361 }