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.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   * 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  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     * Create a new secret manager instance for generating keys.
82     * @param conf Configuration to use
83     * @param zk Connection to zookeeper for handling leader elections
84     * @param keyUpdateInterval Time (in milliseconds) between rolling a new master key for token signing
85     * @param tokenMaxLifetime Maximum age (in milliseconds) before a token expires and is no longer valid
86     */
87    /* TODO: Restrict access to this constructor to make rogues instances more difficult.
88     * For the moment this class is instantiated from
89     * org.apache.hadoop.hbase.ipc.SecureServer so public access is needed.
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       // populate any existing keys
105       this.zkWatcher.start();
106       // try to become leader
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(identifier.getBytes(),
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     // regenerate the password
150     return createPassword(identifier.getBytes(),
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     // ignore zk changes when running as master
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     // update current sequence
188     if (key.getKeyId() > idSeq) {
189       idSeq = key.getKeyId();
190     }
191   }
192 
193   synchronized boolean removeKey(Integer keyId) {
194     // ignore zk changes when running as master
195     if (leaderElector.isMaster()) {
196       if (LOG.isDebugEnabled()) {
197         LOG.debug("Running as master, ignoring removed key "+keyId);
198       }
199       return false;
200     }
201 
202     if (LOG.isDebugEnabled()) {
203       LOG.debug("Removing key "+keyId);
204     }
205 
206     allKeys.remove(keyId);
207     return true;
208   }
209 
210   AuthenticationKey getCurrentKey() {
211     return currentKey;
212   }
213 
214   AuthenticationKey getKey(int keyId) {
215     return allKeys.get(keyId);
216   }
217 
218   synchronized void removeExpiredKeys() {
219     if (!leaderElector.isMaster()) {
220       LOG.info("Skipping removeExpiredKeys() because not running as master.");
221       return;
222     }
223 
224     long now = EnvironmentEdgeManager.currentTimeMillis();
225     Iterator<AuthenticationKey> iter = allKeys.values().iterator();
226     while (iter.hasNext()) {
227       AuthenticationKey key = iter.next();
228       if (key.getExpiration() < now) {
229         if (LOG.isDebugEnabled()) {
230           LOG.debug("Removing expired key "+key.getKeyId());
231         }
232         iter.remove();
233         zkWatcher.removeKeyFromZK(key);
234       }
235     }
236   }
237   
238   synchronized boolean isCurrentKeyRolled() {
239     return currentKey != null;
240   }
241 
242   synchronized void rollCurrentKey() {
243     if (!leaderElector.isMaster()) {
244       LOG.info("Skipping rollCurrentKey() because not running as master.");
245       return;
246     }
247 
248     long now = EnvironmentEdgeManager.currentTimeMillis();
249     AuthenticationKey prev = currentKey;
250     AuthenticationKey newKey = new AuthenticationKey(++idSeq,
251         Long.MAX_VALUE, // don't allow to expire until it's replaced by a new key
252         generateSecret());
253     allKeys.put(newKey.getKeyId(), newKey);
254     currentKey = newKey;
255     zkWatcher.addKeyToZK(newKey);
256     lastKeyUpdate = now;
257 
258     if (prev != null) {
259       // make sure previous key is still stored
260       prev.setExpiration(now + tokenMaxLifetime);
261       allKeys.put(prev.getKeyId(), prev);
262       zkWatcher.updateKeyInZK(prev);
263     }
264   }
265 
266   public static SecretKey createSecretKey(byte[] raw) {
267     return SecretManager.createSecretKey(raw);
268   }
269 
270   private class LeaderElector extends Thread implements Stoppable {
271     private boolean stopped = false;
272     /** Flag indicating whether we're in charge of rolling/expiring keys */
273     private boolean isMaster = false;
274     private ZKLeaderManager zkLeader;
275 
276     public LeaderElector(ZooKeeperWatcher watcher, String serverName) {
277       setDaemon(true);
278       setName("ZKSecretWatcher-leaderElector");
279       zkLeader = new ZKLeaderManager(watcher,
280           ZKUtil.joinZNode(zkWatcher.getRootKeyZNode(), "keymaster"),
281           Bytes.toBytes(serverName), this);
282     }
283 
284     public boolean isMaster() {
285       return isMaster;
286     }
287 
288     @Override
289     public boolean isStopped() {
290       return stopped;
291     }
292 
293     @Override
294     public void stop(String reason) {
295       if (stopped) {
296         return;
297       }
298 
299       stopped = true;
300       // prevent further key generation when stopping
301       if (isMaster) {
302         zkLeader.stepDownAsLeader();
303       }
304       isMaster = false;
305       LOG.info("Stopping leader election, because: "+reason);
306       interrupt();
307     }
308 
309     public void run() {
310       zkLeader.start();
311       zkLeader.waitToBecomeLeader();
312       isMaster = true;
313 
314       while (!stopped) {
315         long now = EnvironmentEdgeManager.currentTimeMillis();
316 
317         // clear any expired
318         removeExpiredKeys();
319 
320         if (lastKeyUpdate + keyUpdateInterval < now) {
321           // roll a new master key
322           rollCurrentKey();
323         }
324 
325         try {
326           Thread.sleep(5000);
327         } catch (InterruptedException ie) {
328           if (LOG.isDebugEnabled()) {
329             LOG.debug("Interrupted waiting for next update", ie);
330           }
331         }
332       }
333     }
334   }
335 }