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 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 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     // regenerate the password
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     // ignore zk changes when running as master
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     // update current sequence
189     if (key.getKeyId() > idSeq) {
190       idSeq = key.getKeyId();
191     }
192   }
193 
194   synchronized boolean removeKey(Integer keyId) {
195     // ignore zk changes when running as master
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, // don't allow to expire until it's replaced by a new key
253         generateSecret());
254     allKeys.put(newKey.getKeyId(), newKey);
255     currentKey = newKey;
256     zkWatcher.addKeyToZK(newKey);
257     lastKeyUpdate = now;
258 
259     if (prev != null) {
260       // make sure previous key is still stored
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     /** Flag indicating whether we're in charge of rolling/expiring keys */
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       // prevent further key generation when stopping
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         // clear any expired
319         removeExpiredKeys();
320 
321         if (lastKeyUpdate + keyUpdateInterval < now) {
322           // roll a new master key
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 }